From 6dde85d87581cac4344ad702055b872946fc56b1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 5 Nov 2019 18:05:18 -0800 Subject: [PATCH 001/291] Initial commit --- README.md | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 README.md diff --git a/README.md b/README.md new file mode 100644 index 00000000000..05889859f72 --- /dev/null +++ b/README.md @@ -0,0 +1,2 @@ +# connectors +Connectors for Delta Lake From 638ce0b8daa1ae58198f2921f482853ca08a7e65 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 5 Nov 2019 18:06:40 -0800 Subject: [PATCH 002/291] Added core files --- build.sbt | 100 +++++++++++++++ build/sbt | 183 ++++++++++++++++++++++++++++ build/sbt-config/repositories | 11 ++ build/sbt-launch-lib.bash | 189 +++++++++++++++++++++++++++++ coreTest/src/main/scala/Test.scala | 38 ++++++ project/build.properties | 36 ++++++ project/plugins.sbt | 35 ++++++ 7 files changed, 592 insertions(+) create mode 100644 build.sbt create mode 100755 build/sbt create mode 100644 build/sbt-config/repositories create mode 100755 build/sbt-launch-lib.bash create mode 100644 coreTest/src/main/scala/Test.scala create mode 100644 project/build.properties create mode 100644 project/plugins.sbt diff --git a/build.sbt b/build.sbt new file mode 100644 index 00000000000..a62f1e70f46 --- /dev/null +++ b/build.sbt @@ -0,0 +1,100 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +lazy val commonSettings = Seq( + version := "0.4.0", + organization := "io.delta", + scalaVersion := "2.12.8", + fork := true +) + +lazy val core = (project in file("core")) + .settings( + name := "delta-connector-core", + libraryDependencies ++= Seq( + "io.delta" %% "delta-core" % "0.4.0" excludeAll (ExclusionRule("org.apache.hadoop")), + "org.apache.spark" %% "spark-sql" % "2.4.2" excludeAll (ExclusionRule("org.apache.hadoop")), + "org.apache.hadoop" % "hadoop-client" % "2.6.5" % "provided" + ), + packageBin in Compile := assembly.value, + commonSettings, + assemblySettings + ) + +lazy val coreTest = (project in file("coreTest")) + .settings( + commonSettings, + libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "2.6.5", + unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, + autoScalaLibrary := false, + + // Ensures that the connector core jar is compiled before compiling this project + (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value + ) + + +lazy val assemblySettings = Seq( + test in assembly := {}, + assemblyMergeStrategy in assembly := { + case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard + case "log4j.properties" => MergeStrategy.discard + case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines + case "reference.conf" => MergeStrategy.concat + case _ => MergeStrategy.first + }, + + assemblyShadeRules in assembly := Seq( + /* + All org.apache.* before shading: + arrow, avro, commons, curator, ivy, jute, log4j, orc, oro, parquet, spark, xbean, zookeeper + */ + + // Packages to exclude from shading because they are not happy when shaded + ShadeRule.rename("javax.**" -> "@0").inAll, + ShadeRule.rename("com.sun.**" -> "@0").inAll, + ShadeRule.rename("com.fasterxml.**" -> "@0").inAll, // Scala reflect trigger via catalyst fails when package changed + ShadeRule.rename("org.apache.hadoop.**" -> "@0").inAll, // Do not change any references to hadoop classes as they will be provided + ShadeRule.rename("org.apache.spark.**" -> "@0").inAll, // Scala package object does not resolve correctly when package changed + ShadeRule.rename("org.apache.log4j.**" -> "@0").inAll, // Initialization via reflection fails when package changed + ShadeRule.rename("org.apache.commons.**" -> "@0").inAll, // Initialization via reflection fails when package changed + ShadeRule.rename("org.xerial.snappy.*Native*" -> "@0").inAll, // JNI class fails to resolve native code when package changed + ShadeRule.rename("com.databricks.**" -> "@0").inAll, // Scala package object does not resolve correctly when package changed + + // Shade everything else + ShadeRule.rename("com.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("org.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("io.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("net.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("avro.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("codegen.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("jersey.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("javassist.**" -> "shadedelta.@0").inAll, + + /* + All top level dirs left in the jar after shading: + aix, assets, com, darwin, delta, fr, include, javax, linux, org, scala, shaded, shadedelta, win + */ + + // Remove things we know are not needed + ShadeRule.zap("py4j**").inAll, + ShadeRule.zap("webapps**").inAll, + ShadeRule.zap("delta**").inAll + ), + + logLevel in assembly := Level.Debug +) + diff --git a/build/sbt b/build/sbt new file mode 100755 index 00000000000..53dc447fede --- /dev/null +++ b/build/sbt @@ -0,0 +1,183 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# This file contains code from the Apache Spark project (original license above). +# It contains modifications, which are licensed as follows: +# + +# +# Copyright 2019 Databricks, Inc. +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so +# that we can run Hive to generate the golden answer. This is not required for normal development +# or testing. +if [ -n "$HIVE_HOME" ]; then + for i in "$HIVE_HOME"/lib/* + do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i" + done + export HADOOP_CLASSPATH +fi + +realpath () { +( + TARGET_FILE="$1" + + cd "$(dirname "$TARGET_FILE")" + TARGET_FILE="$(basename "$TARGET_FILE")" + + COUNT=0 + while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] + do + TARGET_FILE="$(readlink "$TARGET_FILE")" + cd $(dirname "$TARGET_FILE") + TARGET_FILE="$(basename $TARGET_FILE)" + COUNT=$(($COUNT + 1)) + done + + echo "$(pwd -P)/"$TARGET_FILE"" +) +} + +if [[ "$JENKINS_URL" != "" ]]; then + # Make Jenkins use Google Mirror first as Maven Central may ban us + SBT_REPOSITORIES_CONFIG="$(dirname "$(realpath "$0")")/sbt-config/repositories" + export SBT_OPTS="-Dsbt.override.build.repos=true -Dsbt.repository.config=$SBT_REPOSITORIES_CONFIG" +fi + +. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash + + +declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" +declare -r sbt_opts_file=".sbtopts" +declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" + +usage() { + cat < path to global settings/plugins directory (default: ~/.sbt) + -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) + -ivy path to local Ivy repository (default: ~/.ivy2) + -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) + -no-share use all local caches; no sharing + -no-global uses global caches, but does not use global ~/.sbt directory. + -jvm-debug Turn on JVM debugging, open at the given port. + -batch Disable interactive mode + + # sbt version (default: from project/build.properties if present, else latest release) + -sbt-version use the specified version of sbt + -sbt-jar use the specified jar as the sbt launcher + -sbt-rc use an RC version of sbt + -sbt-snapshot use a snapshot version of sbt + + # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) + -java-home alternate JAVA_HOME + + # jvm options and output control + JAVA_OPTS environment variable, if unset uses "$java_opts" + SBT_OPTS environment variable, if unset uses "$default_sbt_opts" + .sbtopts if this file exists in the current directory, it is + prepended to the runner args + /etc/sbt/sbtopts if this file exists, it is prepended to the runner args + -Dkey=val pass -Dkey=val directly to the java runtime + -J-X pass option -X directly to the java runtime + (-J is stripped) + -S-X add -X to sbt's scalacOptions (-S is stripped) + -PmavenProfiles Enable a maven profile for the build. + +In the case of duplicated or conflicting options, the order above +shows precedence: JAVA_OPTS lowest, command line options highest. +EOM +} + +process_my_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; + -no-share) addJava "$noshare_opts" && shift ;; + -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; + -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; + -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; + -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; + -batch) exec /dev/null) + if [[ ! $? ]]; then + saved_stty="" + fi +} + +saveSttySettings +trap onExit INT + +run "$@" + +exit_status=$? +onExit diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories new file mode 100644 index 00000000000..dcac6f66c19 --- /dev/null +++ b/build/sbt-config/repositories @@ -0,0 +1,11 @@ +[repositories] + local + local-preloaded-ivy: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/}, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext] + local-preloaded: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/} + gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/repos/central/data/ + maven-central + typesafe-ivy-releases: https://repo.typesafe.com/typesafe/ivy-releases/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly + sbt-ivy-snapshots: https://repo.scala-sbt.org/scalasbt/ivy-snapshots/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly + sbt-plugin-releases: https://repo.scala-sbt.org/scalasbt/sbt-plugin-releases/, [organization]/[module]/(scala_[scalaVersion]/)(sbt_[sbtVersion]/)[revision]/[type]s/[artifact](-[classifier]).[ext] + bintray-spark-packages: https://dl.bintray.com/spark-packages/maven/ + typesafe-releases: http://repo.typesafe.com/typesafe/releases/ diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash new file mode 100755 index 00000000000..44d37fbf3d0 --- /dev/null +++ b/build/sbt-launch-lib.bash @@ -0,0 +1,189 @@ +#!/usr/bin/env bash +# + +# A library to simplify using the SBT launcher from other packages. +# Note: This should be used by tools like giter8/conscript etc. + +# TODO - Should we merge the main SBT script with this library? + +if test -z "$HOME"; then + declare -r script_dir="$(dirname "$script_path")" +else + declare -r script_dir="$HOME/.sbt" +fi + +declare -a residual_args +declare -a java_args +declare -a scalac_args +declare -a sbt_commands +declare -a maven_profiles + +if test -x "$JAVA_HOME/bin/java"; then + echo -e "Using $JAVA_HOME as default JAVA_HOME." + echo "Note, this will be overridden by -java-home if it is set." + declare java_cmd="$JAVA_HOME/bin/java" +else + declare java_cmd=java +fi + +echoerr () { + echo 1>&2 "$@" +} +vlog () { + [[ $verbose || $debug ]] && echoerr "$@" +} +dlog () { + [[ $debug ]] && echoerr "$@" +} + +acquire_sbt_jar () { + SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` + URL1=https://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + JAR=build/sbt-launch-${SBT_VERSION}.jar + + sbt_jar=$JAR + + if [[ ! -f "$sbt_jar" ]]; then + # Download sbt launch jar if it hasn't been downloaded yet + if [ ! -f "${JAR}" ]; then + # Download + printf "Attempting to fetch sbt\n" + JAR_DL="${JAR}.part" + if [ $(command -v curl) ]; then + curl --fail --location --silent ${URL1} > "${JAR_DL}" &&\ + mv "${JAR_DL}" "${JAR}" + elif [ $(command -v wget) ]; then + wget --quiet ${URL1} -O "${JAR_DL}" &&\ + mv "${JAR_DL}" "${JAR}" + else + printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 + fi + fi + if [ ! -f "${JAR}" ]; then + # We failed to download + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + exit -1 + fi + printf "Launching sbt from ${JAR}\n" + fi +} + +execRunner () { + # print the arguments one to a line, quoting any containing spaces + [[ $verbose || $debug ]] && echo "# Executing command line:" && { + for arg; do + if printf "%s\n" "$arg" | grep -q ' '; then + printf "\"%s\"\n" "$arg" + else + printf "%s\n" "$arg" + fi + done + echo "" + } + + "$@" +} + +addJava () { + dlog "[addJava] arg = '$1'" + java_args=( "${java_args[@]}" "$1" ) +} + +enableProfile () { + dlog "[enableProfile] arg = '$1'" + maven_profiles=( "${maven_profiles[@]}" "$1" ) + export SBT_MAVEN_PROFILES="${maven_profiles[@]}" +} + +addSbt () { + dlog "[addSbt] arg = '$1'" + sbt_commands=( "${sbt_commands[@]}" "$1" ) +} +addResidual () { + dlog "[residual] arg = '$1'" + residual_args=( "${residual_args[@]}" "$1" ) +} +addDebugger () { + addJava "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=$1" +} + +# a ham-fisted attempt to move some memory settings in concert +# so they need not be dicked around with individually. +get_mem_opts () { + local mem=${1:-1000} + local perm=$(( $mem / 4 )) + (( $perm > 256 )) || perm=256 + (( $perm < 4096 )) || perm=4096 + local codecache=$(( $perm / 2 )) + + echo "-Xms${mem}m -Xmx${mem}m -XX:ReservedCodeCacheSize=${codecache}m" +} + +require_arg () { + local type="$1" + local opt="$2" + local arg="$3" + if [[ -z "$arg" ]] || [[ "${arg:0:1}" == "-" ]]; then + echo "$opt requires <$type> argument" 1>&2 + exit 1 + fi +} + +is_function_defined() { + declare -f "$1" > /dev/null +} + +process_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -h|-help) usage; exit 1 ;; + -v|-verbose) verbose=1 && shift ;; + -d|-debug) debug=1 && shift ;; + + -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; + -mem) require_arg integer "$1" "$2" && sbt_mem="$2" && shift 2 ;; + -jvm-debug) require_arg port "$1" "$2" && addDebugger $2 && shift 2 ;; + -batch) exec Date: Tue, 5 Nov 2019 19:02:35 -0800 Subject: [PATCH 003/291] Improved sbt --- build.sbt | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/build.sbt b/build.sbt index a62f1e70f46..8046fb650ff 100644 --- a/build.sbt +++ b/build.sbt @@ -1,3 +1,4 @@ + /* * Copyright 2019 Databricks, Inc. * @@ -23,26 +24,37 @@ lazy val commonSettings = Seq( lazy val core = (project in file("core")) .settings( - name := "delta-connector-core", + name := "delta-core-shaded", libraryDependencies ++= Seq( "io.delta" %% "delta-core" % "0.4.0" excludeAll (ExclusionRule("org.apache.hadoop")), "org.apache.spark" %% "spark-sql" % "2.4.2" excludeAll (ExclusionRule("org.apache.hadoop")), "org.apache.hadoop" % "hadoop-client" % "2.6.5" % "provided" ), + + // Make the 'compile' invoke the 'assembly' task to generate the uber jar. packageBin in Compile := assembly.value, + commonSettings, assemblySettings ) lazy val coreTest = (project in file("coreTest")) .settings( - commonSettings, - libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "2.6.5", + // Add the uber jar as unmanaged library so that we don't bring in the transitive dependencies unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, + + // Only dependency not in the uber jar + libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "2.6.5", + autoScalaLibrary := false, - // Ensures that the connector core jar is compiled before compiling this project - (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value + // Ensure that the uber jar is compiled before compiling this project + (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, + + // Make 'test' invoke 'runMain' + test in Test := (runMain in Runtime).toTask(" test.Test").value, + + commonSettings ) From 16b6d4b694e8cca269b48643a95b17309f899f2f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 5 Nov 2019 19:10:21 -0800 Subject: [PATCH 004/291] Added log4j.properties to make tests less verbose --- coreTest/src/main/resources/log4j.properties | 48 ++++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 coreTest/src/main/resources/log4j.properties diff --git a/coreTest/src/main/resources/log4j.properties b/coreTest/src/main/resources/log4j.properties new file mode 100644 index 00000000000..10887f221ce --- /dev/null +++ b/coreTest/src/main/resources/log4j.properties @@ -0,0 +1,48 @@ +# +# Copyright 2019 Databricks, Inc. +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN From 958251f9fa578c9bc0307fc1f2e1769f8a55cf13 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 5 Nov 2019 19:35:40 -0800 Subject: [PATCH 005/291] Added circleci --- .circleci/config.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 .circleci/config.yml diff --git a/.circleci/config.yml b/.circleci/config.yml new file mode 100644 index 00000000000..73ca026392d --- /dev/null +++ b/.circleci/config.yml @@ -0,0 +1,11 @@ +version: 2 +jobs: + build: + docker: + - image: circleci/openjdk:8u181-jdk # java 8 + steps: + - checkout + - run: + name: Run tests + command: | + build/sbt test From 5d9a4c93617a9fa4170249a9dbd4c4dbae268ced Mon Sep 17 00:00:00 2001 From: windpiger Date: Wed, 6 Nov 2019 10:48:48 -0800 Subject: [PATCH 006/291] hive connector initial commit --- .circleci/config.yml | 2 +- .gitignore | 104 ++++++++++++++ build.sbt | 48 ++++++- hive/pom.xml | 134 ++++++++++++++++++ .../io/delta/hive/DeltaInputFormat.scala | 71 ++++++++++ .../apache/spark/sql/delta/DeltaHelper.scala | 38 +++++ hive/src/test/resources/log4j.properties | 48 +++++++ .../io/delta/hive/HiveConnectorSuite.scala | 127 +++++++++++++++++ .../scala/io/delta/hive/test/HiveTest.scala | 84 +++++++++++ .../io/delta/hive/test/HiveTestSuite.scala | 16 +++ 10 files changed, 669 insertions(+), 3 deletions(-) create mode 100644 .gitignore create mode 100644 hive/pom.xml create mode 100644 hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala create mode 100644 hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala create mode 100644 hive/src/test/resources/log4j.properties create mode 100644 hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala create mode 100644 hive/src/test/scala/io/delta/hive/test/HiveTest.scala create mode 100644 hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala diff --git a/.circleci/config.yml b/.circleci/config.yml index 73ca026392d..9466cb1ec54 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -8,4 +8,4 @@ jobs: - run: name: Run tests command: | - build/sbt test + build/sbt coreTest/test hive/test diff --git a/.gitignore b/.gitignore new file mode 100644 index 00000000000..2fc8a9053a6 --- /dev/null +++ b/.gitignore @@ -0,0 +1,104 @@ +*#*# +*.#* +*.iml +*.ipr +*.iws +*.pyc +*.pyo +*.swp +*~ +.DS_Store +.cache +.classpath +.ensime +.ensime_cache/ +.ensime_lucene +.generated-mima* +.idea/ +.idea_modules/ +.project +.pydevproject +.scala_dependencies +.settings +/lib/ +R-unit-tests.log +R/unit-tests.out +R/cran-check.out +R/pkg/vignettes/sparkr-vignettes.html +R/pkg/tests/fulltests/Rplots.pdf +build/*.jar +build/apache-maven* +build/scala* +build/zinc* +cache +checkpoint +conf/*.cmd +conf/*.conf +conf/*.properties +conf/*.sh +conf/*.xml +conf/java-opts +conf/slaves +dependency-reduced-pom.xml +derby.log +dev/create-release/*final +dev/create-release/*txt +dev/pr-deps/ +dist/ +docs/_site +docs/api +sql/docs +sql/site +lib_managed/ +lint-r-report.log +log/ +logs/ +out/ +project/boot/ +project/build/target/ +project/plugins/lib_managed/ +project/plugins/project/build.properties +project/plugins/src_managed/ +project/plugins/target/ +python/lib/pyspark.zip +python/deps +docs/python/_static/ +docs/python/_templates/ +docs/python/_build/ +python/test_coverage/coverage_data +python/test_coverage/htmlcov +python/pyspark/python +reports/ +scalastyle-on-compile.generated.xml +scalastyle-output.xml +scalastyle.txt +spark-*-bin-*.tgz +spark-tests.log +src_managed/ +streaming-tests.log +target/ +unit-tests.log +work/ +docs/.jekyll-metadata + +# For Hive +TempStatsStore/ +metastore/ +metastore_db/ +sql/hive-thriftserver/test_warehouses +warehouse/ +spark-warehouse/ + +# For R session data +.RData +.RHistory +.Rhistory +*.Rproj +*.Rproj.* + +.Rproj.user + +**/src/main/resources/js + +# For SBT +.jvmopts diff --git a/build.sbt b/build.sbt index 8046fb650ff..94ceb2a34f5 100644 --- a/build.sbt +++ b/build.sbt @@ -15,11 +15,23 @@ * limitations under the License. */ +parallelExecution in ThisBuild := false + lazy val commonSettings = Seq( version := "0.4.0", organization := "io.delta", scalaVersion := "2.12.8", - fork := true + fork := true, + // Configurations to speed up tests and reduce memory footprint + javaOptions in Test ++= Seq( + "-Dspark.ui.enabled=false", + "-Dspark.ui.showConsoleProgress=false", + "-Dspark.databricks.delta.snapshotPartitions=2", + "-Dspark.sql.shuffle.partitions=5", + "-Ddelta.log.cacheSize=3", + "-Dspark.sql.sources.parallelPartitionDiscovery.parallelism=5", + "-Xmx1024m" + ) ) lazy val core = (project in file("core")) @@ -107,6 +119,38 @@ lazy val assemblySettings = Seq( ShadeRule.zap("delta**").inAll ), - logLevel in assembly := Level.Debug + logLevel in assembly := Level.Info ) +lazy val hive = (project in file("hive")) settings ( + scalaVersion := "2.12.8", + name := "hive-delta", + commonSettings, + unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, + autoScalaLibrary := false, + + // Ensures that the connector core jar is compiled before compiling this project + (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, + + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % "2.7.0" % "provided", + "org.apache.hive" % "hive-exec" % "2.3.3" % "provided" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") + ), + "org.apache.hadoop" % "hadoop-common" % "2.7.0" % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-mapreduce-client-hs" % "2.7.0" % "test", + "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % "2.7.0" % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-yarn-server-tests" % "2.7.0" % "test" classifier "tests", + "org.apache.hive" % "hive-cli" % "2.3.3" % "test" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("ch.qos.logback", "logback-classic"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") + ), + // TODO Figure out how this fixes some bad dependency + "org.apache.spark" %% "spark-core" % "2.4.2" % "test" classifier "tests", + "org.scalatest" %% "scalatest" % "3.0.5" % "test" + ) +) diff --git a/hive/pom.xml b/hive/pom.xml new file mode 100644 index 00000000000..3acc72ad453 --- /dev/null +++ b/hive/pom.xml @@ -0,0 +1,134 @@ + + + 4.0.0 + + io.delta + hive-delta + 1.0-SNAPSHOT + + + 2.11 + + + + + io.delta + delta-core_2.11 + provided + 0.3.1-SNAPSHOT + + + + org.apache.hive + hive-exec + 2.3.3 + provided + + + org.apache.commons + * + + + org.codehaus.janino + * + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + 2.7.2 + provided + + + + org.apache.spark + spark-sql_2.11 + 2.4.3 + + + org.apache.hadoop + * + + + + + + + + + org.scala-tools + maven-scala-plugin + 2.15.0 + + + + compile + testCompile + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.1 + + false + ${project.build.directory}/shaded/delta-hive_${scala.binary.version}-${project.version}.jar + + + org.apache.hadoop:* + + + + + + package + + shade + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + org.codehaus.janino + org.codehaus.janino.shaded + + + org.codehaus.commons.compiler + org.codehaus.commons.compiler.shaded + + + org.apache.commons.lang3.time + org.apache.commons.lang3.time.shaded + + + org.antlr.v4.runtime + org.antlr.v4.runtime.shaded + + + + + + + + + \ No newline at end of file diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala new file mode 100644 index 00000000000..497432d4655 --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -0,0 +1,71 @@ +package io.delta.hive + +import java.io.IOException + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat +import org.apache.hadoop.mapred.FileInputFormat._ +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.security.TokenCache +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.actions.{AddFile, SingleAction} +import org.apache.spark.sql.delta.util.DeltaFileOperations +import org.apache.spark.sql.delta.{DeltaHelper, DeltaLog, DeltaTableUtils} + +class DeltaInputFormat extends MapredParquetInputFormat { + + import DeltaInputFormat._ + + @throws[IOException] + override def listStatus(job: JobConf): Array[FileStatus] = try { + val dirs = getInputPaths(job) + if (dirs.isEmpty) { + throw new IOException("No input paths specified in job") + } else { + TokenCache.obtainTokensForNamenodes(job.getCredentials, dirs, job) + + // find delta root path + val rootPath = DeltaTableUtils.findDeltaTableRoot(spark, dirs.head).get + val deltaLog = DeltaLog.forTable(spark, rootPath) + // get the snapshot of the version + val snapshotToUse = deltaLog.snapshot + + val fs = rootPath.getFileSystem(job) + + // get partition filters + val partitionFilters = if (rootPath != dirs.head) { + val partitionFragments = dirs.map { dir => + val relativePath = DeltaFileOperations.tryRelativizePath(fs, rootPath, dir) + assert( + !relativePath.isAbsolute, + s"Fail to relativize path $dir against base path $rootPath.") + relativePath.toUri.toString + } + DeltaHelper.resolvePathFilters(snapshotToUse, partitionFragments) + } else { + assert(dirs.length == 1, "Not-partitioned table should only have one input dir.") + Nil + } + + // selected files to Hive to be processed + DeltaLog.filterFileList( + snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), partitionFilters) + .as[AddFile](SingleAction.addFileEncoder) + .collect() + .map { file => + fs.getFileStatus(new Path(rootPath, file.path)) + } + } + } catch { + case e: Throwable => + e.printStackTrace() + throw e + } +} + +object DeltaInputFormat { + def spark: SparkSession = SparkSession.builder() + .master("local[*]") + .appName("HiveOnDelta Get Files") + .getOrCreate() +} diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala new file mode 100644 index 00000000000..06c2524446c --- /dev/null +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -0,0 +1,38 @@ +package org.apache.spark.sql.delta + +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{And, EqualTo, Expression, Literal} +import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.util.PartitionUtils + +object DeltaHelper { + + def resolvePathFilters(snapshot: Snapshot, partitionFragments: Seq[String]): Seq[Expression] = { + val metadata = snapshot.metadata + val partitionFilters = partitionFragments.map { fragment => + val partitions = try { + PartitionUtils.parsePathFragmentAsSeq(fragment) + } catch { + case _: ArrayIndexOutOfBoundsException => + throw DeltaErrors.partitionPathParseException(fragment) + } + + val badColumns = partitions.map(_._1).filterNot(metadata.partitionColumns.contains) + if (badColumns.nonEmpty) { + throw DeltaErrors.partitionPathInvolvesNonPartitionColumnException(badColumns, fragment) + } + + partitions.map { case (key, value) => + EqualTo(UnresolvedAttribute(key), Literal(value)) + }.reduce(And) + } + + import org.apache.spark.sql.delta.actions.SingleAction._ + val files = DeltaLog.filterFileList( + metadata.partitionColumns, snapshot.allFiles.toDF(), partitionFilters).as[AddFile].collect() + if (files.length == 0) { + throw DeltaErrors.pathNotExistsException(partitionFragments.mkString(",")) + } + partitionFilters + } +} diff --git a/hive/src/test/resources/log4j.properties b/hive/src/test/resources/log4j.properties new file mode 100644 index 00000000000..10887f221ce --- /dev/null +++ b/hive/src/test/resources/log4j.properties @@ -0,0 +1,48 @@ +# +# Copyright 2019 Databricks, Inc. +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala new file mode 100644 index 00000000000..1cdfce6b635 --- /dev/null +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -0,0 +1,127 @@ +package io.delta.hive + +import java.io.File +import java.nio.file.Files + +import io.delta.hive.test.HiveTest +import org.apache.spark.SparkConf +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.DeltaLog +import org.scalatest.BeforeAndAfterEach + +class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + DeltaLog.clearCache() + } + + override def afterEach(): Unit = { + DeltaLog.clearCache() + } + + test("read a non-partitioned table") { + // Create a Delta table + val tempPath = Files.createTempDirectory("testdata").toFile + try { + val conf = new SparkConf() + val spark = SparkSession.builder() + .appName("HiveConnectorSuite") + .master("local[2]") + .getOrCreate() + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta").save(tempPath.getCanonicalPath) + // Clean up resources so that we can use new DeltaLog and SparkSession + spark.stop() + DeltaLog.clearCache() + runQuery( + s""" + |create external table deltatesttable(c1 INT, c2 STRING) + |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |stored as inputformat 'io.delta.hive.DeltaInputFormat' + |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + |location '${tempPath.getCanonicalPath}' + """.stripMargin) + assert(runQuery( + "select * from deltatesttable").sorted === testData.map(r => s"${r._1}\t${r._2}").sorted) + runQuery("drop table deltatesttable") + } finally { + JavaUtils.deleteRecursively(tempPath) + } + } + + test("read a partitioned table") { + // Create a Delta table + val tempPath = Files.createTempDirectory("testdata").toFile + try { + val conf = new SparkConf() + val spark = SparkSession.builder() + .appName("HiveConnectorSuite") + .master("local[2]") + .getOrCreate() + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta") + .partitionBy("c2").save(tempPath.getCanonicalPath) + // Clean up resources so that we can use new DeltaLog and SparkSession + spark.stop() + DeltaLog.clearCache() + runQuery( + s""" + |create external table deltatesttable(c1 INT) + |partitioned by(c2 STRING) -- TODO Remove this. This should be read from Delta's metadata + |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |stored as inputformat 'io.delta.hive.DeltaInputFormat' + |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + |location '${tempPath.getCanonicalPath}' + """.stripMargin) + // TODO Remove this. We should discover partitions automatically + runQuery("msck repair table deltatesttable") + assert(runQuery( + "select * from deltatesttable").sorted === testData.map(r => s"${r._1}\t${r._2}").sorted) + runQuery("drop table deltatesttable") + } finally { + JavaUtils.deleteRecursively(tempPath) + } + } + + test("read a partitioned table with a partition filter") { + // Create a Delta table + val tempPath = Files.createTempDirectory("testdata").toFile + try { + val conf = new SparkConf() + val spark = SparkSession.builder() + .appName("HiveConnectorSuite") + .master("local[2]") + .getOrCreate() + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta") + .partitionBy("c2").save(tempPath.getCanonicalPath) + // Clean up resources so that we can use new DeltaLog and SparkSession + spark.stop() + DeltaLog.clearCache() + runQuery( + s""" + |create external table deltatesttable(c1 INT) + |partitioned by(c2 STRING) -- TODO Remove this. This should be read from Delta's metadata + |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |stored as inputformat 'io.delta.hive.DeltaInputFormat' + |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + |location '${tempPath.getCanonicalPath}' + """.stripMargin) + // TODO Remove this. We should discover partitions automatically + runQuery("msck repair table deltatesttable") + // Delete the partition not needed in the below query to verify the partition pruning works + JavaUtils.deleteRecursively(new File(tempPath, "c2=foo1")) + assert(tempPath.listFiles.map(_.getName).sorted === Seq("_delta_log", "c2=foo0").sorted) + assert(runQuery( + "select * from deltatesttable where c2 = 'foo0'").sorted === + testData.filter(_._2 == "foo0").map(r => s"${r._1}\t${r._2}").sorted) + runQuery("drop table deltatesttable") + } finally { + JavaUtils.deleteRecursively(tempPath) + } + } +} \ No newline at end of file diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala new file mode 100644 index 00000000000..1520228b7da --- /dev/null +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -0,0 +1,84 @@ +package io.delta.hive.test + +import java.io.File +import java.nio.file.Files + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.hive.cli.CliSessionState +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.mapred.{JobConf, MiniMRCluster} +import org.apache.hadoop.mapreduce.MRJobConfig +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.network.util.JavaUtils + +// TODO Yarn is using log4j2. Disable its verbose logs. +trait HiveTest extends FunSuite with BeforeAndAfterAll { + private val tempPath = Files.createTempDirectory(this.getClass.getSimpleName).toFile + + private var driver: Driver = _ + private var mr: MiniMRCluster = _ + + override def beforeAll(): Unit = { + super.beforeAll() + val warehouseDir = new File(tempPath, "warehouse") + val metastoreDir = new File(tempPath, "metastore_db") + val conf = new HiveConf() + + // Disable schema verification and allow schema auto-creation in the + // Derby database, in case the config for the metastore is set otherwise. + // Without these settings, starting the client fails with + // MetaException(message:Version information not found in metastore.)t + conf.set("hive.metastore.schema.verification", "false") + conf.set("datanucleus.schema.autoCreateAll", "true") + conf.set( + "javax.jdo.option.ConnectionURL", + s"jdbc:derby:memory:;databaseName=${metastoreDir.getCanonicalPath};create=true") + conf.set("hive.metastore.warehouse.dir", warehouseDir.getCanonicalPath) + val fs = FileSystem.getLocal(conf) + val jConf = new JobConf(conf) + jConf.set("yarn.scheduler.capacity.root.queues", "default") + jConf.set("yarn.scheduler.capacity.root.default.capacity", "100") + jConf.setInt(MRJobConfig.MAP_MEMORY_MB, 512) + jConf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 512) + jConf.setInt(MRJobConfig.MR_AM_VMEM_MB, 128) + jConf.setInt(YarnConfiguration.YARN_MINICLUSTER_NM_PMEM_MB, 512) + jConf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128) + jConf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 512) + mr = new MiniMRCluster(2, fs.getUri.toString, 1, null, null, jConf) + + val db = Hive.get(conf) + SessionState.start(new CliSessionState(conf)) + driver = new Driver(conf) + } + + override def afterAll() { + if (mr != null) { + mr.shutdown() + } + driver.close() + driver.destroy() + JavaUtils.deleteRecursively(tempPath) + // TODO Remove leaked "target/MiniMRCluster-XXX" directories + super.afterAll() + } + + def runQuery(query: String): Seq[String] = { + val response = driver.run(query) + if (response.getResponseCode != 0) { + throw new Exception(s"failed to run '$query': ${response.getErrorMessage}") + } + val result = new java.util.ArrayList[String]() + if (driver.getResults(result)) { + result.asScala + } else { + Nil + } + } +} diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala b/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala new file mode 100644 index 00000000000..c454a783e80 --- /dev/null +++ b/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala @@ -0,0 +1,16 @@ +package io.delta.hive.test + +import org.scalatest.Ignore + +// TODO Figure out why running this test will cause other tests fail. Probably due to some unknonw +// dependency conflicts. +@Ignore +class HiveTestSuite extends HiveTest { + + test("basic hive query") { + runQuery("create database testdb1") + runQuery("use testdb1") + runQuery("create table testtbl1 as select 'foo' as key") + assert(runQuery("select * from testdb1.testtbl1") === Seq("foo")) + } +} From a2f62149af3b8e9c4b1a6d3016c88bf5f5e4e59e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 12 Nov 2019 17:16:48 -0800 Subject: [PATCH 007/291] Add files for the open source project (#4) - Updated README - Added LICENSE, CONTRIBUTING, NOTICE --- CONTRIBUTING.md | 1 + LICENSE.txt | 388 ++++++++++++++++++++++++++++++++++++++++++++++++ NOTICE.txt | 24 +++ README.md | 47 +++++- 4 files changed, 458 insertions(+), 2 deletions(-) create mode 100644 CONTRIBUTING.md create mode 100644 LICENSE.txt create mode 100644 NOTICE.txt diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 00000000000..9477ac6dbef --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1 @@ +We happily welcome contributions to Delta Lake Connectors. We use [GitHub Issues](/../../issues/) to track community reported issues and [GitHub Pull Requests ](/../../pulls/) for accepting changes. diff --git a/LICENSE.txt b/LICENSE.txt new file mode 100644 index 00000000000..472fbf91414 --- /dev/null +++ b/LICENSE.txt @@ -0,0 +1,388 @@ +Copyright 2019 Databricks, Inc. All rights reserved. + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + + +------------------------------------------------------------------------- +This project includes code derived from the Apache Spark project. +The individual files containing this code carry the original Apache Spark +license, which is reproduced here as well: + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/NOTICE.txt b/NOTICE.txt new file mode 100644 index 00000000000..f2744176c88 --- /dev/null +++ b/NOTICE.txt @@ -0,0 +1,24 @@ +Delta Lake Connectors +Copyright 2019 Databricks, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +This project includes software licensed by the Apache Software Foundation (Apache 2.0) +from the Apache Spark project (www.github.com/apache/spark) + +---------------------------------------------------------- +Apache Spark +Copyright 2014 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/README.md b/README.md index 05889859f72..3074bd112ba 100644 --- a/README.md +++ b/README.md @@ -1,2 +1,45 @@ -# connectors -Connectors for Delta Lake +# Delta Lake Logo Connectors + +[![CircleCI](https://circleci.com/gh/delta-io/connectors/tree/master.svg?style=svg)](https://circleci.com/gh/delta-io/connectors/tree/master) + +We are building connectors to bring [Delta Lake](https://delta.io) to popular big-data engines outside [Apache Spark](https://spark.apache.org) (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)). + +# Introduction + +This is the repository for Delta Lake Connectors. It includes a library for querying Delta Lake metadata and connectors to popular big-data engines (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)). Please refer to the main [Delta Lake](https://github.com/delta-io/delta) repository if you want to learn more about the Delta Lake project. + +# Building + +The project is compiled using [SBT](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html). + +To compile, run + + build/sbt compile + +To generate artifacts, run + + build/sbt package + +To execute tests, run + + build/sbt test + +Refer to [SBT docs](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html) for more commands. + +# Reporting issues + +We use [GitHub Issues](https://github.com/delta-io/connectors/issues) to track community reported issues. You can also [contact](#community) the community for getting answers. + +# Contributing + +We welcome contributions to Delta Lake Connectors repository. We use [GitHub Pull Requests](https://github.com/delta-io/connectors/pulls) for accepting changes. + +# Community + +There are two mediums of communication within the Delta Lake community. + +- Public Slack Channel + - [Register here](https://join.slack.com/t/delta-users/shared_invite/enQtNTY1NDg0ODcxOTI1LWJkZGU3ZmQ3MjkzNmY2ZDM0NjNlYjE4MWIzYjg2OWM1OTBmMWIxZTllMjg3ZmJkNjIwZmE1ZTZkMmQ0OTk5ZjA) + - [Login here](https://delta-users.slack.com/) + +- Public [Mailing list](https://groups.google.com/forum/#!forum/delta-users) From 794fc8d8aa3f1ea9c98ff7c7894ed8780ee781a3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 20 Nov 2019 14:52:43 -0800 Subject: [PATCH 008/291] Updated README.md with more detailed instructions --- README.md | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 3074bd112ba..c24f0ec1c94 100644 --- a/README.md +++ b/README.md @@ -10,19 +10,20 @@ This is the repository for Delta Lake Connectors. It includes a library for quer # Building -The project is compiled using [SBT](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html). - -To compile, run - - build/sbt compile - -To generate artifacts, run - - build/sbt package - -To execute tests, run - - build/sbt test +The project is compiled using [SBT](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html). It has the following subprojects. + +1. Delta uber jar +This project generates a single uber jar containing Delta Lake and all it transitive dependencies (except Hadoop and its dependencies). +- Most of the dependencies are shaded to avoid version conflicts. See the file build.sbt for details on what are not shaded. +- Hadoop and its dependencies is not included in the jar because they are expected to be present in the deployment environment. +- To generate the uber jar, run `build/sbt core/compile` +- To test the uber jar, run `build/sbt coreTest/test` + +2. Hive connector (WIP) +This project contains all the code needed to make Hive read Delta Lake tables. +- To compile the project, run `build/sbt hive/compile` +- To test the project, run `build/sbt hive/test` +- To generate the connector jar run `bulld/sbt hive/package` Refer to [SBT docs](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html) for more commands. From 02802b4f7657cce16e92f079b2e8c6b1d46325e6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 20 Nov 2019 14:53:36 -0800 Subject: [PATCH 009/291] Updated README.md once again --- README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c24f0ec1c94..016f683d01a 100644 --- a/README.md +++ b/README.md @@ -12,14 +12,15 @@ This is the repository for Delta Lake Connectors. It includes a library for quer The project is compiled using [SBT](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html). It has the following subprojects. -1. Delta uber jar +## 1. Delta uber jar + This project generates a single uber jar containing Delta Lake and all it transitive dependencies (except Hadoop and its dependencies). - Most of the dependencies are shaded to avoid version conflicts. See the file build.sbt for details on what are not shaded. - Hadoop and its dependencies is not included in the jar because they are expected to be present in the deployment environment. - To generate the uber jar, run `build/sbt core/compile` - To test the uber jar, run `build/sbt coreTest/test` -2. Hive connector (WIP) +## 2. Hive connector (WIP) This project contains all the code needed to make Hive read Delta Lake tables. - To compile the project, run `build/sbt hive/compile` - To test the project, run `build/sbt hive/test` From afbe153c4c7f33bc36ebd816093c0b2ca55882eb Mon Sep 17 00:00:00 2001 From: windpiger Date: Tue, 26 Nov 2019 06:37:59 +0800 Subject: [PATCH 010/291] Add DeltaStorageHandler for Hive (#6) Implement HiveOnDelta with StorageHandler **DDL:** ``` create external table deltaTbl(a string, b int) stored by 'io.delta.hive.DeltaStorageHandler' location '/test/delta' ``` - must be external table - must not be a Hive partition table - if DeltaTable is a partitionTable, then the partition column should be after data column when creating Hive table - Hive's schema should be match with the under delta'schema ,including column number &column name - the delta.table.path should be existed **Read:** `set hive.input.format = io.delta.hive.HiveInputFormat` - support read a non-partition or a partition table - support push down filter with delta's partition column, currently support predicate (=,!=,>,>=,<,<=,in,like) - auto-detected delta's partition change **Unit Tests:** - Added(`build/sbt clean test`) - `build/sbt clean package` test ok in real Hive Cluster using delta-core-shaded-assembly-0.4.0.jar and hive-delta_2.12-0.4.0.jar --- build.sbt | 37 +- hive/pom.xml | 134 ------ .../java/io/delta/hive/DeltaInputFormat.java | 60 +++ .../delta/hive/DeltaRecordReaderWrapper.java | 106 ++++ .../io/delta/hive/DeltaStorageHandler.java | 202 ++++++++ .../java/io/delta/hive/HiveInputFormat.java | 15 + .../io/delta/hive/IndexPredicateAnalyzer.java | 322 +++++++++++++ .../io/delta/hive/DeltaInputFormat.scala | 71 --- .../apache/spark/sql/delta/DeltaHelper.scala | 143 ++++-- .../spark/sql/delta/DeltaPushFilter.scala | 98 ++++ .../io/delta/hive/HiveConnectorSuite.scala | 454 ++++++++++++++---- .../scala/io/delta/hive/test/HiveTest.scala | 49 +- 12 files changed, 1347 insertions(+), 344 deletions(-) delete mode 100644 hive/pom.xml create mode 100644 hive/src/main/java/io/delta/hive/DeltaInputFormat.java create mode 100644 hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java create mode 100644 hive/src/main/java/io/delta/hive/DeltaStorageHandler.java create mode 100644 hive/src/main/java/io/delta/hive/HiveInputFormat.java create mode 100644 hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java delete mode 100644 hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala create mode 100644 hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala diff --git a/build.sbt b/build.sbt index 94ceb2a34f5..94d42058fbb 100644 --- a/build.sbt +++ b/build.sbt @@ -17,6 +17,10 @@ parallelExecution in ThisBuild := false +val sparkVersion = "2.4.3" +val hadoopVersion = "2.7.2" +val hiveVersion = "2.3.3" + lazy val commonSettings = Seq( version := "0.4.0", organization := "io.delta", @@ -38,9 +42,9 @@ lazy val core = (project in file("core")) .settings( name := "delta-core-shaded", libraryDependencies ++= Seq( - "io.delta" %% "delta-core" % "0.4.0" excludeAll (ExclusionRule("org.apache.hadoop")), - "org.apache.spark" %% "spark-sql" % "2.4.2" excludeAll (ExclusionRule("org.apache.hadoop")), - "org.apache.hadoop" % "hadoop-client" % "2.6.5" % "provided" + "io.delta" %% "delta-core" % "0.4.0" excludeAll ExclusionRule("org.apache.hadoop"), + "org.apache.spark" %% "spark-sql" % sparkVersion excludeAll ExclusionRule("org.apache.hadoop"), + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" ), // Make the 'compile' invoke the 'assembly' task to generate the uber jar. @@ -56,7 +60,8 @@ lazy val coreTest = (project in file("coreTest")) unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, // Only dependency not in the uber jar - libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "2.6.5", + libraryDependencies += "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll + ExclusionRule("org.slf4j", "slf4j-log4j12"), autoScalaLibrary := false, @@ -69,7 +74,6 @@ lazy val coreTest = (project in file("coreTest")) commonSettings ) - lazy val assemblySettings = Seq( test in assembly := {}, assemblyMergeStrategy in assembly := { @@ -87,6 +91,8 @@ lazy val assemblySettings = Seq( arrow, avro, commons, curator, ivy, jute, log4j, orc, oro, parquet, spark, xbean, zookeeper */ + ShadeRule.rename("org.apache.commons.lang3.time.**" -> "shadedelta.@0").inAll, + // Packages to exclude from shading because they are not happy when shaded ShadeRule.rename("javax.**" -> "@0").inAll, ShadeRule.rename("com.sun.**" -> "@0").inAll, @@ -94,6 +100,7 @@ lazy val assemblySettings = Seq( ShadeRule.rename("org.apache.hadoop.**" -> "@0").inAll, // Do not change any references to hadoop classes as they will be provided ShadeRule.rename("org.apache.spark.**" -> "@0").inAll, // Scala package object does not resolve correctly when package changed ShadeRule.rename("org.apache.log4j.**" -> "@0").inAll, // Initialization via reflection fails when package changed + ShadeRule.rename("org.slf4j.**" -> "@0").inAll, // Initialization via reflection fails when package changed ShadeRule.rename("org.apache.commons.**" -> "@0").inAll, // Initialization via reflection fails when package changed ShadeRule.rename("org.xerial.snappy.*Native*" -> "@0").inAll, // JNI class fails to resolve native code when package changed ShadeRule.rename("com.databricks.**" -> "@0").inAll, // Scala package object does not resolve correctly when package changed @@ -123,7 +130,6 @@ lazy val assemblySettings = Seq( ) lazy val hive = (project in file("hive")) settings ( - scalaVersion := "2.12.8", name := "hive-delta", commonSettings, unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, @@ -133,24 +139,25 @@ lazy val hive = (project in file("hive")) settings ( (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, libraryDependencies ++= Seq( - "org.apache.hadoop" % "hadoop-client" % "2.7.0" % "provided", - "org.apache.hive" % "hive-exec" % "2.3.3" % "provided" excludeAll( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") ), - "org.apache.hadoop" % "hadoop-common" % "2.7.0" % "test" classifier "tests", - "org.apache.hadoop" % "hadoop-mapreduce-client-hs" % "2.7.0" % "test", - "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % "2.7.0" % "test" classifier "tests", - "org.apache.hadoop" % "hadoop-yarn-server-tests" % "2.7.0" % "test" classifier "tests", - "org.apache.hive" % "hive-cli" % "2.3.3" % "test" excludeAll( + "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-mapreduce-client-hs" % hadoopVersion % "test", + "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-yarn-server-tests" % hadoopVersion % "test" classifier "tests", + "org.apache.hive" % "hive-cli" % hiveVersion % "test" excludeAll( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), ExclusionRule("ch.qos.logback", "logback-classic"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") ), // TODO Figure out how this fixes some bad dependency - "org.apache.spark" %% "spark-core" % "2.4.2" % "test" classifier "tests", - "org.scalatest" %% "scalatest" % "3.0.5" % "test" + "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", + "org.scalatest" %% "scalatest" % "3.0.5" % "test", + "io.delta" %% "delta-core" % "0.4.0" % "test" excludeAll ExclusionRule("org.apache.hadoop") ) ) diff --git a/hive/pom.xml b/hive/pom.xml deleted file mode 100644 index 3acc72ad453..00000000000 --- a/hive/pom.xml +++ /dev/null @@ -1,134 +0,0 @@ - - - 4.0.0 - - io.delta - hive-delta - 1.0-SNAPSHOT - - - 2.11 - - - - - io.delta - delta-core_2.11 - provided - 0.3.1-SNAPSHOT - - - - org.apache.hive - hive-exec - 2.3.3 - provided - - - org.apache.commons - * - - - org.codehaus.janino - * - - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - 2.7.2 - provided - - - - org.apache.spark - spark-sql_2.11 - 2.4.3 - - - org.apache.hadoop - * - - - - - - - - - org.scala-tools - maven-scala-plugin - 2.15.0 - - - - compile - testCompile - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.2.1 - - false - ${project.build.directory}/shaded/delta-hive_${scala.binary.version}-${project.version}.jar - - - org.apache.hadoop:* - - - - - - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - - org.codehaus.janino - org.codehaus.janino.shaded - - - org.codehaus.commons.compiler - org.codehaus.commons.compiler.shaded - - - org.apache.commons.lang3.time - org.apache.commons.lang3.time.shaded - - - org.antlr.v4.runtime - org.antlr.v4.runtime.shaded - - - - - - - - - \ No newline at end of file diff --git a/hive/src/main/java/io/delta/hive/DeltaInputFormat.java b/hive/src/main/java/io/delta/hive/DeltaInputFormat.java new file mode 100644 index 00000000000..406e91c96cd --- /dev/null +++ b/hive/src/main/java/io/delta/hive/DeltaInputFormat.java @@ -0,0 +1,60 @@ +package io.delta.hive; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.*; +import org.apache.hadoop.mapreduce.security.TokenCache; +import org.apache.parquet.hadoop.ParquetInputFormat; +import org.apache.spark.sql.delta.DeltaHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DeltaInputFormat extends FileInputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(DeltaInputFormat.class); + private final ParquetInputFormat realInput; + + public DeltaInputFormat() { + this(new ParquetInputFormat(DataWritableReadSupport.class)); + } + + protected DeltaInputFormat(ParquetInputFormat inputFormat) { + this.realInput = inputFormat; + } + + @Override + public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException { + try { + if (Utilities.getUseVectorizedInputFileFormat(job)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Using vectorized record reader"); + } + throw new IOException("Currently not support Delta VectorizedReader"); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Using row-mode record reader"); + } + return new DeltaRecordReaderWrapper(this.realInput, split, job, reporter); + } + + } catch (InterruptedException var5) { + throw new RuntimeException("Cannot create a RecordReaderWrapper", var5); + } + } + + @Override + protected FileStatus[] listStatus(JobConf job) throws IOException { + Path deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)); + TokenCache.obtainTokensForNamenodes(job.getCredentials(), new Path[]{deltaRootPath}, job); + + List filteredDeltaFiles = DeltaHelper.listDeltaFiles(deltaRootPath, job); + return filteredDeltaFiles.toArray(new FileStatus[filteredDeltaFiles.size()]); + } +} diff --git a/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java b/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java new file mode 100644 index 00000000000..9392dd5fee3 --- /dev/null +++ b/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java @@ -0,0 +1,106 @@ +package io.delta.hive; + +import com.google.common.base.Joiner; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import scala.collection.JavaConverters; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.parquet.hadoop.ParquetInputFormat; +import org.apache.spark.sql.delta.DeltaHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DeltaRecordReaderWrapper extends ParquetRecordReaderWrapper { + + private static final Logger LOG = LoggerFactory.getLogger(DeltaRecordReaderWrapper.class); + + private Writable[] partitionWritable = null; + + public DeltaRecordReaderWrapper(ParquetInputFormat newInputFormat, InputSplit oldSplit, JobConf oldJobConf, Reporter reporter) throws IOException, InterruptedException { + super(newInputFormat, oldSplit, oldJobConf, reporter); + + if (!(oldSplit instanceof FileSplit)) { + throw new IllegalArgumentException("Unknown split type: " + oldSplit); + } else { + final String columnNameProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES); + final String columnTypeProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES); + LOG.info("Delta partition cols: " + columnNameProperty + " with types: " + columnTypeProperty); + + if (columnNameProperty == null || columnNameProperty.trim().length() == 0 + || columnTypeProperty == null || columnTypeProperty.trim().length() == 0) { + LOG.info("No partition info is provided..."); + } else { + // generate partition writale values which will be appended after data values from parquet + final List columnNames; + final List columnTypes; + if (columnNameProperty.length() == 0) { + columnNames = new ArrayList(); + } else { + columnNames = Arrays.asList(columnNameProperty.split(",")); + } + if (columnTypeProperty.length() == 0) { + columnTypes = new ArrayList(); + } else { + columnTypes = Arrays.asList(columnTypeProperty.split(":")); + } + + Path filePath = ((FileSplit)oldSplit).getPath(); + + Map parsedPartitions = JavaConverters.mapAsJavaMap( + DeltaHelper.parsePathPartition(filePath, JavaConverters.asScalaBufferConverter(columnNames).asScala().toSeq())); + + partitionWritable = new Writable[columnNames.size()]; + // inspect partition values + for(int i=0; i < columnNames.size(); i++) { + ObjectInspector oi = PrimitiveObjectInspectorFactory + .getPrimitiveWritableObjectInspector(TypeInfoFactory + .getPrimitiveTypeInfo(columnTypes.get(i))); + + partitionWritable[i] = (Writable)ObjectInspectorConverters.getConverter( + PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi).convert(parsedPartitions.get(columnNames.get(i))); + } + LOG.info("Parsed partition values from " + filePath.toString() + " list: " + Joiner.on(",").withKeyValueSeparator("=").join(parsedPartitions) + + ", partitionWritable length:" + partitionWritable.length); + } + } + } + + @Override + public boolean next(NullWritable key, ArrayWritable value) throws IOException { + boolean hasNext = super.next(key, value); + if (partitionWritable != null && partitionWritable.length != 0) { + // append partition values to data values + for (int i=0; i < partitionWritable.length; i++) { + value.get()[value.get().length - partitionWritable.length + i] = partitionWritable[i]; + } + } + return hasNext; + } + + @Override + public ArrayWritable createValue() { + ArrayWritable value = super.createValue(); + if (partitionWritable != null && partitionWritable.length != 0) { + for (int i=0; i < partitionWritable.length; i++) { + value.get()[value.get().length - partitionWritable.length + i] = partitionWritable[i]; + } + } + return value; + } +} diff --git a/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java b/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java new file mode 100644 index 00000000000..61d9826c926 --- /dev/null +++ b/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java @@ -0,0 +1,202 @@ +package io.delta.hive; + +import com.google.common.base.Joiner; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import scala.collection.JavaConverters; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler; +import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.AbstractSerDe; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.StringUtils; +import org.apache.spark.sql.delta.DeltaHelper; +import org.apache.spark.sql.delta.DeltaPushFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DeltaStorageHandler extends DefaultStorageHandler + implements HiveMetaHook, HiveStoragePredicateHandler { + + private static final Logger LOG = LoggerFactory.getLogger(DeltaStorageHandler.class); + + public static final String DELTA_TABLE_PATH = "delta.table.path"; + public static final String DELTA_PARTITION_COLS_NAMES = "delta.partition.columns"; + public static final String DELTA_PARTITION_COLS_TYPES = "delta.partition.columns.types"; + + @Override + public Class getInputFormatClass() { + return DeltaInputFormat.class; + } + + @Override + public Class getSerDeClass() { + return ParquetHiveSerDe.class; + } + + @Override + public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { + super.configureInputJobProperties(tableDesc, jobProperties); + jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)); + jobProperties.put(DELTA_PARTITION_COLS_NAMES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_NAMES)); + jobProperties.put(DELTA_PARTITION_COLS_TYPES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_TYPES)); + } + + @Override + public DecomposedPredicate decomposePredicate( + JobConf jobConf, Deserializer deserializer, + ExprNodeDesc predicate) { + // Get the delta root path + String deltaRootPath = jobConf.get(DELTA_TABLE_PATH); + // Get the partitionColumns of Delta + List partitionColumns = JavaConverters.seqAsJavaList( + DeltaHelper.getPartitionCols(new Path(deltaRootPath))); + LOG.info("delta partitionColumns is " + Joiner.on(",").join(partitionColumns)); + + IndexPredicateAnalyzer analyzer = newIndexPredicateAnalyzer(partitionColumns); + + List conditions = new ArrayList(); + ExprNodeGenericFuncDesc pushedPredicate = null; + ExprNodeGenericFuncDesc residualPredicate = + (ExprNodeGenericFuncDesc) analyzer.analyzePredicate(predicate, conditions); + + for (List searchConditions : decompose(conditions).values()) { + // still push back the pushedPredicate to residualPredicate + residualPredicate = + extractResidualCondition(analyzer, searchConditions, residualPredicate); + pushedPredicate = + extractStorageHandlerCondition(analyzer, searchConditions, pushedPredicate); + } + + LOG.info("pushedPredicate:" + (pushedPredicate == null? "null":pushedPredicate.getExprString()) + + ",residualPredicate" + residualPredicate); + + DecomposedPredicate decomposedPredicate = new DecomposedPredicate(); + decomposedPredicate.pushedPredicate = pushedPredicate; + decomposedPredicate.residualPredicate = residualPredicate; + return decomposedPredicate; + } + + private IndexPredicateAnalyzer newIndexPredicateAnalyzer( + List partitionColumns) { + IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer(); + for (String col : partitionColumns) { + // Supported filter exprs on partition column to be pushed down to delta + analyzer.addComparisonOp(col, DeltaPushFilter.supportedPushDownUDFs()); + } + return analyzer; + } + + private static Map> decompose( + List searchConditions) { + Map> result = + new HashMap>(); + for (IndexSearchCondition condition : searchConditions) { + List conditions = result.get(condition.getColumnDesc().getColumn()); + if (conditions == null) { + conditions = new ArrayList(); + result.put(condition.getColumnDesc().getColumn(), conditions); + } + conditions.add(condition); + } + return result; + } + + private static ExprNodeGenericFuncDesc extractResidualCondition( + IndexPredicateAnalyzer analyzer, + List searchConditions, ExprNodeGenericFuncDesc inputExpr) { + if (inputExpr == null) { + return analyzer.translateOriginalConditions(searchConditions); + } + List children = new ArrayList(); + children.add(analyzer.translateOriginalConditions(searchConditions)); + children.add(inputExpr); + return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), children); + } + + private static ExprNodeGenericFuncDesc extractStorageHandlerCondition( + IndexPredicateAnalyzer analyzer, + List searchConditions, ExprNodeGenericFuncDesc inputExpr) { + if (inputExpr == null) { + return analyzer.translateSearchConditions(searchConditions); + } + List children = new ArrayList(); + children.add(analyzer.translateSearchConditions(searchConditions)); + children.add(inputExpr); + return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), children); + } + + public HiveMetaHook getMetaHook() { + return this; + } + + public void preCreateTable(Table tbl) throws MetaException { + boolean isExternal = MetaStoreUtils.isExternalTable(tbl); + if (!isExternal) { + throw new MetaException("HiveOnDelta should be an external table."); + } else if (tbl.getPartitionKeysSize() > 0) { + throw new MetaException("HiveOnDelta does not support to create a partition hive table"); + } + + String deltaRootString = tbl.getSd().getLocation(); + try { + if (deltaRootString == null || deltaRootString.trim().length() == 0) { + throw new MetaException("table location should be set when creating table"); + } else { + Path deltaPath = new Path(deltaRootString); + FileSystem fs = deltaPath.getFileSystem(getConf()); + if (!fs.exists(deltaPath)) { + throw new MetaException("delta.table.path(" + deltaRootString + ") does not exist..."); + } else { + Map partitionProps = JavaConverters.mapAsJavaMap( + DeltaHelper.checkHiveColsInDelta(deltaPath, tbl.getSd().getCols())); + tbl.getSd().getSerdeInfo().getParameters().putAll(partitionProps); + tbl.getSd().getSerdeInfo().getParameters().put(DELTA_TABLE_PATH, deltaRootString); + LOG.info("write partition cols/types to table properties " + + Joiner.on(",").withKeyValueSeparator("=").join(partitionProps)); + } + } + } catch (Exception e) { + throw new MetaException(StringUtils.stringifyException(e)); + } + } + + public void rollbackCreateTable(Table table) throws MetaException { + // nothing to do + } + + public void commitCreateTable(Table table) throws MetaException { + // nothing to do + } + + public void preDropTable(Table table) throws MetaException { + // nothing to do + } + + public void rollbackDropTable(Table table) throws MetaException { + // nothing to do + } + + public void commitDropTable(Table tbl, boolean deleteData) throws MetaException { + // nothing to do + } +} diff --git a/hive/src/main/java/io/delta/hive/HiveInputFormat.java b/hive/src/main/java/io/delta/hive/HiveInputFormat.java new file mode 100644 index 00000000000..fb27cbe7205 --- /dev/null +++ b/hive/src/main/java/io/delta/hive/HiveInputFormat.java @@ -0,0 +1,15 @@ +package io.delta.hive; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; + +public class HiveInputFormat extends org.apache.hadoop.hive.ql.io.HiveInputFormat { + @Override + protected void pushProjectionsAndFilters(JobConf jobConf, Class inputFormatClass, Path splitPath, boolean nonNative) { + if (inputFormatClass == DeltaInputFormat.class ) { + super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, false); + } else { + super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, nonNative); + } + } +} diff --git a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java new file mode 100644 index 00000000000..46635b97f35 --- /dev/null +++ b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java @@ -0,0 +1,322 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.delta.hive; + +import java.util.*; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.index.IndexSearchCondition; +import org.apache.hadoop.hive.ql.lib.*; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.*; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Copy from Hive org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer + * IndexPredicateAnalyzer decomposes predicates, separating the parts + * which can be satisfied by an index from the parts which cannot. + * Currently, it only supports pure conjunctions over binary expressions + * comparing a column reference with a constant value. It is assumed + * that all column aliases encountered refer to the same table. + */ +public class IndexPredicateAnalyzer { + + private static final Logger LOG = LoggerFactory.getLogger(IndexPredicateAnalyzer.class); + + private final Set udfNames; + private final Map> columnToUDFs; + + public IndexPredicateAnalyzer() { + udfNames = new HashSet(); + columnToUDFs = new HashMap>(); + } + + /** + * Registers a comparison operator as one which can be satisfied + * by an index search. Unless this is called, analyzePredicate + * will never find any indexable conditions. + * + * @param udfName name of comparison operator as returned + * by either {@link GenericUDFBridge#getUdfName} (for simple UDF's) + * or udf.getClass().getName() (for generic UDF's). + */ + public void addComparisonOp(String udfName) { + udfNames.add(udfName); + } + + /** + * Clears the set of column names allowed in comparisons. (Initially, all + * column names are allowed.) + */ + public void clearAllowedColumnNames() { + columnToUDFs.clear(); + } + + /** + * Adds a column name to the set of column names allowed. + * + * @param columnName name of column to be allowed + */ + public void allowColumnName(String columnName) { + columnToUDFs.put(columnName, udfNames); + } + + /** + * add allowed functions per column + * @param columnName + * @param udfs + */ + public void addComparisonOp(String columnName, String... udfs) { + Set allowed = columnToUDFs.get(columnName); + if (allowed == null || allowed == udfNames) { + // override + columnToUDFs.put(columnName, new HashSet(Arrays.asList(udfs))); + } else { + allowed.addAll(Arrays.asList(udfs)); + } + } + + /** + * Analyzes a predicate. + * + * @param predicate predicate to be analyzed + * + * @param searchConditions receives conditions produced by analysis + * + * @return residual predicate which could not be translated to + * searchConditions + */ + public ExprNodeDesc analyzePredicate( + ExprNodeDesc predicate, + final List searchConditions) { + + Map opRules = new LinkedHashMap(); + NodeProcessor nodeProcessor = new NodeProcessor() { + public Object process(Node nd, Stack stack, + NodeProcessorCtx procCtx, Object... nodeOutputs) + throws SemanticException { + + // We can only push down stuff which appears as part of + // a pure conjunction: reject OR, CASE, etc. + for (Node ancestor : stack) { + if (nd == ancestor) { + break; + } + if (!FunctionRegistry.isOpAnd((ExprNodeDesc) ancestor)) { + return nd; + } + } + + return analyzeExpr((ExprNodeGenericFuncDesc) nd, searchConditions, nodeOutputs); + } + }; + + Dispatcher disp = new DefaultRuleDispatcher( + nodeProcessor, opRules, null); + GraphWalker ogw = new DefaultGraphWalker(disp); + ArrayList topNodes = new ArrayList(); + topNodes.add(predicate); + HashMap nodeOutput = new HashMap(); + try { + ogw.startWalking(topNodes, nodeOutput); + } catch (SemanticException ex) { + throw new RuntimeException(ex); + } + ExprNodeDesc residualPredicate = (ExprNodeDesc) nodeOutput.get(predicate); + return residualPredicate; + } + + //Check if ExprNodeColumnDesc is wrapped in expr. + //If so, peel off. Otherwise return itself. + private ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { + if (expr instanceof ExprNodeColumnDesc) { + return expr; + } + ExprNodeGenericFuncDesc funcDesc = null; + if (expr instanceof ExprNodeGenericFuncDesc) { + funcDesc = (ExprNodeGenericFuncDesc) expr; + } + if (null == funcDesc) { + return expr; + } + GenericUDF udf = funcDesc.getGenericUDF(); + // check if its a simple cast expression. + if ((udf instanceof GenericUDFBridge || udf instanceof GenericUDFToBinary + || udf instanceof GenericUDFToChar || udf instanceof GenericUDFToVarchar + || udf instanceof GenericUDFToDecimal || udf instanceof GenericUDFToDate + || udf instanceof GenericUDFToUnixTimeStamp || udf instanceof GenericUDFToUtcTimestamp) + && funcDesc.getChildren().size() == 1 + && funcDesc.getChildren().get(0) instanceof ExprNodeColumnDesc) { + return expr.getChildren().get(0); + } + return expr; + } + + private ExprNodeDesc analyzeExpr( + ExprNodeGenericFuncDesc expr, + List searchConditions, + Object... nodeOutputs) throws SemanticException { + + if (FunctionRegistry.isOpAnd(expr)) { + assert(nodeOutputs.length >= 2); + List residuals = new ArrayList(); + for (Object residual : nodeOutputs) { + if (null != residual) { + residuals.add((ExprNodeDesc)residual); + } + } + if (residuals.size() == 0) { + return null; + } else if (residuals.size() == 1) { + return residuals.get(0); + } else { + return new ExprNodeGenericFuncDesc( + TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), + residuals); + } + } + + GenericUDF genericUDF = expr.getGenericUDF(); + + ExprNodeDesc[] peelOffExprs = new ExprNodeDesc[nodeOutputs.length]; + List exprNodeColDescs = new ArrayList(); + List exprConstantColDescs = new ArrayList(); + + for (int i = 0; i < nodeOutputs.length; i++) { + // We may need to peel off the GenericUDFBridge that is added by CBO or user + ExprNodeDesc peelOffExpr = getColumnExpr((ExprNodeDesc)nodeOutputs[i]); + if (peelOffExpr instanceof ExprNodeColumnDesc) { + exprNodeColDescs.add((ExprNodeColumnDesc)peelOffExpr); + } else if (peelOffExpr instanceof ExprNodeConstantDesc) { + exprConstantColDescs.add((ExprNodeConstantDesc)peelOffExpr); + } + + peelOffExprs[i] = peelOffExpr; + } + + if (exprNodeColDescs.size() != 1) { + LOG.info("Pushed down expr should only have one column, while it is " + StringUtils.join(exprNodeColDescs.toArray())); + return expr; + } + + ExprNodeColumnDesc columnDesc = exprNodeColDescs.get(0); + + Set allowed = columnToUDFs.get(columnDesc.getColumn()); + if (allowed == null) { + LOG.info("This column " + columnDesc.getColumn() + " is not allowed to pushed down to delta..."); + return expr; + } + + String udfClassName = genericUDF.getUdfName(); + if (genericUDF instanceof GenericUDFBridge) { + udfClassName = ((GenericUDFBridge) genericUDF).getUdfClassName(); + } + if (!allowed.contains(udfClassName)) { + LOG.info("This udf " + genericUDF.getUdfName() + " is not allowed to pushed down to delta..."); + return expr; + } + + if (!udfClassName.equals("org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn") + && exprConstantColDescs.size() > 1) { + LOG.info("There should be one constant in this udf(" + udfClassName + ") except UDFIn"); + return expr; + } + + // We also need to update the expr so that the index query can be generated. + // Note that, hive does not support UDFToDouble etc in the query text. + ExprNodeGenericFuncDesc indexExpr = + new ExprNodeGenericFuncDesc(expr.getTypeInfo(), expr.getGenericUDF(), Arrays.asList(peelOffExprs)); + + searchConditions.add( + new IndexSearchCondition( + columnDesc, + udfClassName, + null, + indexExpr, + expr, + null)); + + // we converted the expression to a search condition, so + // remove it from the residual predicate + return null; + } + + /** + * Translates search conditions back to ExprNodeDesc form (as + * a left-deep conjunction). + * + * @param searchConditions (typically produced by analyzePredicate) + * + * @return ExprNodeGenericFuncDesc form of search conditions + */ + public ExprNodeGenericFuncDesc translateSearchConditions( + List searchConditions) { + + ExprNodeGenericFuncDesc expr = null; + for (IndexSearchCondition searchCondition : searchConditions) { + if (expr == null) { + expr = searchCondition.getIndexExpr(); + continue; + } + List children = new ArrayList(); + children.add(expr); + children.add(searchCondition.getIndexExpr()); + expr = new ExprNodeGenericFuncDesc( + TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), + children); + } + return expr; + } + + /** + * Translates original conditions back to ExprNodeDesc form (as + * a left-deep conjunction). + * + * @param searchConditions (typically produced by analyzePredicate) + * + * @return ExprNodeGenericFuncDesc form of search conditions + */ + public ExprNodeGenericFuncDesc translateOriginalConditions( + List searchConditions) { + + ExprNodeGenericFuncDesc expr = null; + for (IndexSearchCondition searchCondition : searchConditions) { + if (expr == null) { + expr = searchCondition.getOriginalExpr(); + continue; + } + List children = new ArrayList(); + children.add(expr); + children.add(searchCondition.getOriginalExpr()); + expr = new ExprNodeGenericFuncDesc( + TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), + children); + } + return expr; + } +} diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala deleted file mode 100644 index 497432d4655..00000000000 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ /dev/null @@ -1,71 +0,0 @@ -package io.delta.hive - -import java.io.IOException - -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat -import org.apache.hadoop.mapred.FileInputFormat._ -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.security.TokenCache -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.delta.actions.{AddFile, SingleAction} -import org.apache.spark.sql.delta.util.DeltaFileOperations -import org.apache.spark.sql.delta.{DeltaHelper, DeltaLog, DeltaTableUtils} - -class DeltaInputFormat extends MapredParquetInputFormat { - - import DeltaInputFormat._ - - @throws[IOException] - override def listStatus(job: JobConf): Array[FileStatus] = try { - val dirs = getInputPaths(job) - if (dirs.isEmpty) { - throw new IOException("No input paths specified in job") - } else { - TokenCache.obtainTokensForNamenodes(job.getCredentials, dirs, job) - - // find delta root path - val rootPath = DeltaTableUtils.findDeltaTableRoot(spark, dirs.head).get - val deltaLog = DeltaLog.forTable(spark, rootPath) - // get the snapshot of the version - val snapshotToUse = deltaLog.snapshot - - val fs = rootPath.getFileSystem(job) - - // get partition filters - val partitionFilters = if (rootPath != dirs.head) { - val partitionFragments = dirs.map { dir => - val relativePath = DeltaFileOperations.tryRelativizePath(fs, rootPath, dir) - assert( - !relativePath.isAbsolute, - s"Fail to relativize path $dir against base path $rootPath.") - relativePath.toUri.toString - } - DeltaHelper.resolvePathFilters(snapshotToUse, partitionFragments) - } else { - assert(dirs.length == 1, "Not-partitioned table should only have one input dir.") - Nil - } - - // selected files to Hive to be processed - DeltaLog.filterFileList( - snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), partitionFilters) - .as[AddFile](SingleAction.addFileEncoder) - .collect() - .map { file => - fs.getFileStatus(new Path(rootPath, file.path)) - } - } - } catch { - case e: Throwable => - e.printStackTrace() - throw e - } -} - -object DeltaInputFormat { - def spark: SparkSession = SparkSession.builder() - .master("local[*]") - .appName("HiveOnDelta Get Files") - .getOrCreate() -} diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 06c2524446c..144af6de137 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -1,38 +1,123 @@ package org.apache.spark.sql.delta -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{And, EqualTo, Expression, Literal} -import org.apache.spark.sql.delta.actions.AddFile -import org.apache.spark.sql.delta.util.PartitionUtils - -object DeltaHelper { - - def resolvePathFilters(snapshot: Snapshot, partitionFragments: Seq[String]): Seq[Expression] = { - val metadata = snapshot.metadata - val partitionFilters = partitionFragments.map { fragment => - val partitions = try { - PartitionUtils.parsePathFragmentAsSeq(fragment) - } catch { - case _: ArrayIndexOutOfBoundsException => - throw DeltaErrors.partitionPathParseException(fragment) - } +import io.delta.hive.DeltaStorageHandler +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} +import org.apache.hadoop.hive.ql.plan.TableScanDesc +import org.apache.hadoop.mapred.JobConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.actions.{AddFile, SingleAction} + +object DeltaHelper extends Logging { + + def parsePathPartition(path: Path, partitionCols: Seq[String]): Map[String, String] = { + val columns = ArrayBuffer.empty[(String, String)] + // Old Hadoop versions don't have `Path.isRoot` + var finished = path.getParent == null + // currentPath is the current path that we will use to parse partition column value. + var currentPath: Path = path + + while (!finished) { + // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. + // Once we get the string, we try to parse it and find the partition column and value. + val fragment = currentPath.getName + val maybeColumn = + parsePartitionColumn(currentPath.getName) + + maybeColumn.foreach(columns += _) + + finished = + (maybeColumn.isEmpty && columns.nonEmpty) || currentPath.getParent == null - val badColumns = partitions.map(_._1).filterNot(metadata.partitionColumns.contains) - if (badColumns.nonEmpty) { - throw DeltaErrors.partitionPathInvolvesNonPartitionColumnException(badColumns, fragment) + if (!finished) { + // For the above example, currentPath will be "/table/". + currentPath = currentPath.getParent } + } + + assert(columns.map(_._1).zip(partitionCols).forall(c => c._1 == c._2), + s""" + |partitionCols(${columns.map(_._1).mkString(",")}) parsed from $path + |does not match the created table partition(${partitionCols.mkString(",")}) + """.stripMargin) + + columns.toMap + } + + private def parsePartitionColumn(columnSpec: String): Option[(String, String)] = { + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName + + val equalSignIndex = columnSpec.indexOf('=') + if (equalSignIndex == -1) { + None + } else { + val columnName = unescapePathName(columnSpec.take(equalSignIndex)) + assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") + + val rawColumnValue = columnSpec.drop(equalSignIndex + 1) + assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - partitions.map { case (key, value) => - EqualTo(UnresolvedAttribute(key), Literal(value)) - }.reduce(And) + Some(columnName, rawColumnValue) } + } + + def listDeltaFiles(rootPath: Path, job: JobConf): java.util.List[FileStatus] = { + val deltaLog = DeltaLog.forTable(spark, rootPath) + // get the snapshot of the version + val snapshotToUse = deltaLog.snapshot + + val fs = rootPath.getFileSystem(job) + + // get the partition prune exprs + val filterExprSerialized = job.get(TableScanDesc.FILTER_EXPR_CONF_STR) + + val convertedFilterExpr = DeltaPushFilter.partitionFilterConverter(filterExprSerialized) + + // selected files to Hive to be processed + DeltaLog.filterFileList( + snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), convertedFilterExpr) + .as[AddFile](SingleAction.addFileEncoder) + .collect().par.map { f => + logInfo(s"selected delta file ${f.path} under $rootPath") + fs.getFileStatus(new Path(rootPath, f.path)) + }.toList.asJava + } - import org.apache.spark.sql.delta.actions.SingleAction._ - val files = DeltaLog.filterFileList( - metadata.partitionColumns, snapshot.allFiles.toDF(), partitionFilters).as[AddFile].collect() - if (files.length == 0) { - throw DeltaErrors.pathNotExistsException(partitionFragments.mkString(",")) + def checkHiveColsInDelta( + rootPath: Path, + hiveSchema: java.util.List[FieldSchema]): Map[String, String] = { + val deltaMeta = DeltaLog.forTable(spark, rootPath).snapshot.metadata + assert(hiveSchema.size() == deltaMeta.schema.size, + s"Hive cols(${hiveSchema.asScala.map(_.getName).mkString(",")}) number does not match " + + s"Delta cols(${deltaMeta.schema.map(_.name).mkString(",")})") + + assert(hiveSchema.asScala.forall(h => deltaMeta.schema.exists(_.name == h.getName)), + s"Hive cols(${hiveSchema.asScala.map(_.getName).mkString(",")}) name does not match " + + s"Delta cols(${deltaMeta.schema.map(_.name).mkString(",")})") + + val (ds, ps) = hiveSchema.asScala.splitAt(hiveSchema.size() - deltaMeta.partitionColumns.size) + + if (ds.forall(s => deltaMeta.dataSchema.exists(_.name == s.getName)) + && ps.forall(s => deltaMeta.partitionColumns.contains(s.getName))) { + Map(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES -> ps.map(_.getName).mkString(","), + DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES -> ps.map(_.getType).mkString(":")) + } else { + throw new MetaException(s"The partition cols of Delta should be after data cols " + + s"when creating hive table. Delta dataschema is " + + s"${deltaMeta.dataSchema.json} and partitionschema is ${deltaMeta.partitionSchema.json}") } - partitionFilters } -} + + def getPartitionCols(rootPath: Path): Seq[String] = { + DeltaLog.forTable(spark, rootPath).snapshot.metadata.partitionColumns + } + + def spark: SparkSession = SparkSession.builder() + .master("local[*]") + .appName("HiveOnDelta Get Files") + .getOrCreate() +} \ No newline at end of file diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala new file mode 100644 index 00000000000..f35945f8775 --- /dev/null +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala @@ -0,0 +1,98 @@ +package org.apache.spark.sql.delta + +import scala.collection.immutable.HashSet +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, SerializationUtilities} +import org.apache.hadoop.hive.ql.lib._ +import org.apache.hadoop.hive.ql.parse.SemanticException +import org.apache.hadoop.hive.ql.plan.{ExprNodeColumnDesc, ExprNodeConstantDesc, ExprNodeGenericFuncDesc} +import org.apache.hadoop.hive.ql.udf.generic._ +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{And, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, InSet, LessThan, LessThanOrEqual, Like, Literal, Not} + +object DeltaPushFilter extends Logging { + lazy val supportedPushDownUDFs = Array( + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS", + "org.apache.hadoop.hive.ql.udf.UDFLike", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn" + ) + + def partitionFilterConverter(hiveFilterExprSeriablized: String): Seq[Expression] = { + if (hiveFilterExprSeriablized != null) { + val filterExpr = SerializationUtilities.deserializeExpression(hiveFilterExprSeriablized) + val opRules = new java.util.LinkedHashMap[Rule, NodeProcessor]() + val nodeProcessor = new NodeProcessor() { + @throws[SemanticException] + def process(nd: Node, stack: java.util.Stack[Node], + procCtx: NodeProcessorCtx, nodeOutputs: Object*): Object = { + nd match { + case e: ExprNodeGenericFuncDesc if FunctionRegistry.isOpAnd(e) => + nodeOutputs.map(_.asInstanceOf[Expression]).reduce(And) + case e: ExprNodeGenericFuncDesc => + val (columnDesc, constantDesc) = + if (nd.getChildren.get(0).isInstanceOf[ExprNodeColumnDesc]) { + (nd.getChildren.get(0), nd.getChildren.get(1)) + } else { (nd.getChildren.get(1), nd.getChildren.get(0)) } + + val columnAttr = UnresolvedAttribute( + columnDesc.asInstanceOf[ExprNodeColumnDesc].getColumn) + val constantVal = Literal(constantDesc.asInstanceOf[ExprNodeConstantDesc].getValue) + nd.asInstanceOf[ExprNodeGenericFuncDesc].getGenericUDF match { + case f: GenericUDFOPNotEqualNS => + Not(EqualNullSafe(columnAttr, constantVal)) + case f: GenericUDFOPNotEqual => + Not(EqualTo(columnAttr, constantVal)) + case f: GenericUDFOPEqualNS => + EqualNullSafe(columnAttr, constantVal) + case f: GenericUDFOPEqual => + EqualTo(columnAttr, constantVal) + case f: GenericUDFOPGreaterThan => + GreaterThan(columnAttr, constantVal) + case f: GenericUDFOPEqualOrGreaterThan => + GreaterThanOrEqual(columnAttr, constantVal) + case f: GenericUDFOPLessThan => + LessThan(columnAttr, constantVal) + case f: GenericUDFOPEqualOrLessThan => + LessThanOrEqual(columnAttr, constantVal) + case f: GenericUDFBridge if f.getUdfName.equals("like") => + Like(columnAttr, constantVal) + case f: GenericUDFIn => + val inConstantVals = nd.getChildren.asScala + .filter(_.isInstanceOf[ExprNodeConstantDesc]) + .map(_.asInstanceOf[ExprNodeConstantDesc].getValue) + .map(Literal(_)).toSet + InSet(columnAttr, HashSet() ++ inConstantVals) + case _ => + throw new RuntimeException(s"Unsupported func(${nd.getName}) " + + s"which can not be pushed down to delta") + } + case _ => null + } + } + } + + val disp = new DefaultRuleDispatcher(nodeProcessor, opRules, null) + val ogw = new DefaultGraphWalker(disp) + val topNodes = new java.util.ArrayList[Node]() + topNodes.add(filterExpr) + val nodeOutput = new java.util.HashMap[Node, Object]() + try { + ogw.startWalking(topNodes, nodeOutput) + } catch { + case ex: Exception => + throw new RuntimeException(ex) + } + logInfo(s"converted partition filter expr:" + + s"${nodeOutput.get(filterExpr).asInstanceOf[Expression].toJSON}") + Seq(nodeOutput.get(filterExpr).asInstanceOf[Expression]) + } else Seq.empty[org.apache.spark.sql.catalyst.expressions.Expression] + } +} diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index 1cdfce6b635..a8bea7a458d 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -1,12 +1,11 @@ package io.delta.hive import java.io.File -import java.nio.file.Files import io.delta.hive.test.HiveTest -import org.apache.spark.SparkConf +import io.delta.tables.DeltaTable + import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog import org.scalatest.BeforeAndAfterEach @@ -20,108 +19,377 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { DeltaLog.clearCache() } + test("DDL: HiveOnDelta should be a external table ") { + withTable("deltaTbl") { + withTempDir { dir => + val e = intercept[Exception] { + runQuery( + s""" + |create table deltaTbl(a string, b int) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e.contains("HiveOnDelta should be an external table")) + } + } + } + + test("DDL: location should be set when creating table") { + withTable("deltaTbl") { + val e = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(a string, b int) + |stored by 'io.delta.hive.DeltaStorageHandler' + """.stripMargin + ) + }.getMessage + assert(e.contains("table location should be set when creating table")) + } + } + + test("DDL: HiveOnDelta should not be a partitioned hive table") { + withTable("deltaTbl") { + withTempDir { dir => + val e = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(a string, b int) + |partitioned by(c string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e.contains("HiveOnDelta does not support to create a partition hive table")) + } + } + } + + test("DDL: the delta root path should be existed when create hive table") { + withTable("deltaTbl") { + withTempDir { dir => + JavaUtils.deleteRecursively(dir) + + val e = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(a string, b int) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e.contains(s"delta.table.path(${dir.getCanonicalPath}) does not exist...")) + } + } + } + + test("DDL: when creating hive table on a partitioned delta, " + + "the partition columns should be after data columns") { + withTable("deltaTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession { spark => + import spark.implicits._ + testData.toDS.toDF("a", "b").write.format("delta") + .partitionBy("b").save(dir.getCanonicalPath) + } + + val e = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(b string, a string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e.contains(s"The partition cols of Delta should be after data cols")) + } + } + } + + // check column number & column name + test("DDL: Hive schema should match delta's schema") { + withTable("deltaTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}", s"test${x % 3}")) + + withSparkSession { spark => + import spark.implicits._ + val x = testData.toDS.toDF("a", "b", "c") + testData.toDS.toDF("a", "b", "c").write.format("delta") + .partitionBy("b").save(dir.getCanonicalPath) + } + + // column number mismatch + val e1 = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(a string, b string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e1.contains(s"number does not match")) + + // column name mismatch + val e2 = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(e string, c string, b string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + }.getMessage + assert(e2.contains(s"name does not match")) + } + } + } + test("read a non-partitioned table") { // Create a Delta table - val tempPath = Files.createTempDirectory("testdata").toFile - try { - val conf = new SparkConf() - val spark = SparkSession.builder() - .appName("HiveConnectorSuite") - .master("local[2]") - .getOrCreate() - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta").save(tempPath.getCanonicalPath) - // Clean up resources so that we can use new DeltaLog and SparkSession - spark.stop() - DeltaLog.clearCache() - runQuery( - s""" - |create external table deltatesttable(c1 INT, c2 STRING) - |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - |stored as inputformat 'io.delta.hive.DeltaInputFormat' - |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - |location '${tempPath.getCanonicalPath}' - """.stripMargin) - assert(runQuery( - "select * from deltatesttable").sorted === testData.map(r => s"${r._1}\t${r._2}").sorted) - runQuery("drop table deltatesttable") - } finally { - JavaUtils.deleteRecursively(tempPath) + withTable("deltaNonPartitionTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession{ spark => + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaNonPartitionTbl(c1 int, c2 string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + assert(runQuery( + "select * from deltaNonPartitionTbl").sorted === + testData.map(r => s"${r._1}\t${r._2}").sorted) + } } } test("read a partitioned table") { // Create a Delta table - val tempPath = Files.createTempDirectory("testdata").toFile - try { - val conf = new SparkConf() - val spark = SparkSession.builder() - .appName("HiveConnectorSuite") - .master("local[2]") - .getOrCreate() - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta") - .partitionBy("c2").save(tempPath.getCanonicalPath) - // Clean up resources so that we can use new DeltaLog and SparkSession - spark.stop() - DeltaLog.clearCache() - runQuery( - s""" - |create external table deltatesttable(c1 INT) - |partitioned by(c2 STRING) -- TODO Remove this. This should be read from Delta's metadata - |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - |stored as inputformat 'io.delta.hive.DeltaInputFormat' - |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - |location '${tempPath.getCanonicalPath}' - """.stripMargin) - // TODO Remove this. We should discover partitions automatically - runQuery("msck repair table deltatesttable") - assert(runQuery( - "select * from deltatesttable").sorted === testData.map(r => s"${r._1}\t${r._2}").sorted) - runQuery("drop table deltatesttable") - } finally { - JavaUtils.deleteRecursively(tempPath) + withTable("deltaPartitionTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession { spark => + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta") + .partitionBy("c2").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaPartitionTbl(c1 int, c2 string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + assert(runQuery( + "select * from deltaPartitionTbl").sorted === + testData.map(r => s"${r._1}\t${r._2}").sorted) + + // select partition column order change + assert(runQuery( + "select c2, c1 from deltaPartitionTbl").sorted === + testData.map(r => s"${r._2}\t${r._1}").sorted) + + assert(runQuery( + "select c2, c1, c2 as c3 from deltaPartitionTbl").sorted === + testData.map(r => s"${r._2}\t${r._1}\t${r._2}").sorted) + } } } test("read a partitioned table with a partition filter") { // Create a Delta table - val tempPath = Files.createTempDirectory("testdata").toFile - try { - val conf = new SparkConf() - val spark = SparkSession.builder() - .appName("HiveConnectorSuite") - .master("local[2]") - .getOrCreate() - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")).toSeq - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta") - .partitionBy("c2").save(tempPath.getCanonicalPath) - // Clean up resources so that we can use new DeltaLog and SparkSession - spark.stop() - DeltaLog.clearCache() - runQuery( - s""" - |create external table deltatesttable(c1 INT) - |partitioned by(c2 STRING) -- TODO Remove this. This should be read from Delta's metadata - |row format serde 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - |stored as inputformat 'io.delta.hive.DeltaInputFormat' - |outputformat 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - |location '${tempPath.getCanonicalPath}' - """.stripMargin) - // TODO Remove this. We should discover partitions automatically - runQuery("msck repair table deltatesttable") - // Delete the partition not needed in the below query to verify the partition pruning works - JavaUtils.deleteRecursively(new File(tempPath, "c2=foo1")) - assert(tempPath.listFiles.map(_.getName).sorted === Seq("_delta_log", "c2=foo0").sorted) - assert(runQuery( - "select * from deltatesttable where c2 = 'foo0'").sorted === - testData.filter(_._2 == "foo0").map(r => s"${r._1}\t${r._2}").sorted) - runQuery("drop table deltatesttable") - } finally { - JavaUtils.deleteRecursively(tempPath) + withTable("deltaPartitionTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession { spark => + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta") + .partitionBy("c2").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaPartitionTbl(c1 int, c2 string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + // Delete the partition not needed in the below query to verify the partition pruning works + JavaUtils.deleteRecursively(new File(dir, "c2=foo1")) + assert(dir.listFiles.map(_.getName).sorted === Seq("_delta_log", "c2=foo0").sorted) + assert(runQuery( + "select * from deltaPartitionTbl where c2 = 'foo0'").sorted === + testData.filter(_._2 == "foo0").map(r => s"${r._1}\t${r._2}").sorted) + } + } + } + + test("partition prune") { + withTable("deltaPartitionTbl") { + withTempDir { dir => + val testData = Seq( + ("hz", "20180520", "Jim", 3), + ("hz", "20180718", "Jone", 7), + ("bj", "20180520", "Trump", 1), + ("sh", "20180512", "Jay", 4), + ("sz", "20181212", "Linda", 8) + ) + + withSparkSession { spark => + import spark.implicits._ + testData.toDS.toDF("city", "date", "name", "cnt").write.format("delta") + .partitionBy("date", "city").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaPartitionTbl(name string, cnt int, city string, `date` string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + // equal pushed down + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'") + .mkString(" ").contains("filterExpr: (date = '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'") + .toList.sorted === testData.filter(_._2 == "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'") + .mkString(" ").contains("filterExpr: (date <> '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'") + .toList.sorted === testData.filter(_._2 != "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'") + .mkString(" ").contains("filterExpr: (date > '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'") + .toList.sorted === testData.filter(_._2 > "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'") + .mkString(" ").contains("filterExpr: (date >= '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'") + .toList.sorted === testData.filter(_._2 >= "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'") + .mkString(" ").contains("filterExpr: (date < '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'") + .toList.sorted === testData.filter(_._2 < "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + assert(runQuery( + "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'") + .mkString(" ").contains("filterExpr: (date <= '20180520')")) + assert(runQuery( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'") + .toList.sorted === testData.filter(_._2 <= "20180520") + .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + + // expr(like) pushed down + assert(runQuery( + "explain select * from deltaPartitionTbl where `date` like '201805%'") + .mkString(" ").contains("filterExpr: (date like '201805%')")) + assert(runQuery( + "select * from deltaPartitionTbl where `date` like '201805%'").toList.sorted === testData + .filter(_._2.contains("201805")).map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + + // expr(in) pushed down + assert(runQuery( + "explain select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')") + .mkString(" ").contains("filterExpr: (city) IN ('hz', 'sz')")) + assert(runQuery( + "select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')") + .toList.sorted === testData.filter(c => Seq("hz", "sz").contains(c._1)) + .map(r => s"${r._3}\t${r._2}\t${r._4}").sorted) + + // two partition column pushed down + assert(runQuery( + "explain select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')") + .mkString(" ").contains("filterExpr: ((city) IN ('hz', 'sz') and (date = '20181212'))")) + assert(runQuery( + "select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')") + .toList.sorted === testData + .filter(c => Seq("hz", "sz").contains(c._1) && c._2 == "20181212") + .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + + // data column not be pushed down + assert(runQuery( + "explain select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'") + .mkString(" ").contains("filterExpr: (city = 'hz'")) + assert(runQuery( + "select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'") + .toList.sorted === testData + .filter(c => c._1 == "hz" && c._3 == "Jim") + .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + } + } + } + + test("auto-detected delta partition change") { + withTable("deltaPartitionTbl") { + withTempDir { dir => + val testData1 = Seq( + ("hz", "20180520", "Jim", 3), + ("hz", "20180718", "Jone", 7) + ) + + withSparkSession { spark => + import spark.implicits._ + testData1.toDS.toDF("city", "date", "name", "cnt").write.format("delta") + .partitionBy("date", "city").save(dir.getCanonicalPath) + + runQuery( + s""" + |create external table deltaPartitionTbl(name string, cnt int, city string, `date` string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + assert(runQuery( + "select * from deltaPartitionTbl").toList.sorted === testData1 + .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + + // insert another partition data + val testData2 = Seq(("bj", "20180520", "Trump", 1)) + testData2.toDS.toDF("city", "date", "name", "cnt").write.mode("append").format("delta") + .partitionBy("date", "city").save(dir.getCanonicalPath) + val testData = testData1 ++ testData2 + assert(runQuery( + "select * from deltaPartitionTbl").toList.sorted === testData + .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + + // delete one partition + val deltaTable = DeltaTable.forPath(spark, dir.getCanonicalPath) + deltaTable.delete("city='hz'") + assert(runQuery( + "select * from deltaPartitionTbl").toList.sorted === testData + .filterNot(_._1 == "hz").map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + } + } } } } \ No newline at end of file diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 1520228b7da..b96e8a8096f 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -14,9 +14,11 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.mapred.{JobConf, MiniMRCluster} import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.{BeforeAndAfterAll, FunSuite} - +import org.apache.spark.SparkConf import org.apache.spark.network.util.JavaUtils +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.DeltaLog +import org.scalatest.{BeforeAndAfterAll, FunSuite} // TODO Yarn is using log4j2. Disable its verbose logs. trait HiveTest extends FunSuite with BeforeAndAfterAll { @@ -37,6 +39,10 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { // MetaException(message:Version information not found in metastore.)t conf.set("hive.metastore.schema.verification", "false") conf.set("datanucleus.schema.autoCreateAll", "true") + // if hive.fetch.task.conversion set to none, the hive.input.format should be + // io.delta.hive.HiveInputFormat + conf.set("hive.fetch.task.conversion", "none") + conf.set("hive.input.format", "io.delta.hive.HiveInputFormat") conf.set( "javax.jdo.option.ConnectionURL", s"jdbc:derby:memory:;databaseName=${metastoreDir.getCanonicalPath};create=true") @@ -81,4 +87,43 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { Nil } } + + /** + * Drops table `tableName` after calling `f`. + */ + protected def withTable(tableNames: String*)(f: => Unit): Unit = { + try f finally { + tableNames.foreach { name => + runQuery(s"DROP TABLE IF EXISTS $name") + } + } + } + + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("hiveondelta").toFile + + try f(dir) finally { + JavaUtils.deleteRecursively(dir) + } + } + + protected def withSparkSession(f: SparkSession => Unit): Unit = { + val conf = new SparkConf() + val spark = SparkSession.builder() + .appName("HiveConnectorSuite") + .master("local[2]") + .getOrCreate() + + try f(spark) finally { + // Clean up resources so that we can use new DeltaLog and SparkSession + spark.stop() + DeltaLog.clearCache() + } + } } From 4aeac03de0b613622404e8be4b7098693e69a9d1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 26 Nov 2019 22:40:00 -0800 Subject: [PATCH 011/291] Rewrite Java files using Scala (#7) This PR rewrites Java files using Scala to make future development easier since Delta itself is written using Scala. `IndexPredicateAnalyzer` is not rewritten because it's a fork from Hive's `IndexPredicateAnalyzer` and it's better to not change it so that we can compare them when `IndexPredicateAnalyzer` is changed in Hive. This PR doesn't change any logic in codes. --- .../java/io/delta/hive/DeltaInputFormat.java | 60 ------ .../delta/hive/DeltaRecordReaderWrapper.java | 106 --------- .../io/delta/hive/DeltaStorageHandler.java | 202 ------------------ .../java/io/delta/hive/HiveInputFormat.java | 15 -- .../io/delta/hive/DeltaInputFormat.scala | 46 ++++ .../delta/hive/DeltaRecordReaderWrapper.scala | 81 +++++++ .../io/delta/hive/DeltaStorageHandler.scala | 180 ++++++++++++++++ .../scala/io/delta/hive/HiveInputFormat.scala | 15 ++ 8 files changed, 322 insertions(+), 383 deletions(-) delete mode 100644 hive/src/main/java/io/delta/hive/DeltaInputFormat.java delete mode 100644 hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java delete mode 100644 hive/src/main/java/io/delta/hive/DeltaStorageHandler.java delete mode 100644 hive/src/main/java/io/delta/hive/HiveInputFormat.java create mode 100644 hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala create mode 100644 hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala create mode 100644 hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala create mode 100644 hive/src/main/scala/io/delta/hive/HiveInputFormat.scala diff --git a/hive/src/main/java/io/delta/hive/DeltaInputFormat.java b/hive/src/main/java/io/delta/hive/DeltaInputFormat.java deleted file mode 100644 index 406e91c96cd..00000000000 --- a/hive/src/main/java/io/delta/hive/DeltaInputFormat.java +++ /dev/null @@ -1,60 +0,0 @@ -package io.delta.hive; - -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport; -import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.*; -import org.apache.hadoop.mapreduce.security.TokenCache; -import org.apache.parquet.hadoop.ParquetInputFormat; -import org.apache.spark.sql.delta.DeltaHelper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DeltaInputFormat extends FileInputFormat { - - private static final Logger LOG = LoggerFactory.getLogger(DeltaInputFormat.class); - private final ParquetInputFormat realInput; - - public DeltaInputFormat() { - this(new ParquetInputFormat(DataWritableReadSupport.class)); - } - - protected DeltaInputFormat(ParquetInputFormat inputFormat) { - this.realInput = inputFormat; - } - - @Override - public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException { - try { - if (Utilities.getUseVectorizedInputFileFormat(job)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Using vectorized record reader"); - } - throw new IOException("Currently not support Delta VectorizedReader"); - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Using row-mode record reader"); - } - return new DeltaRecordReaderWrapper(this.realInput, split, job, reporter); - } - - } catch (InterruptedException var5) { - throw new RuntimeException("Cannot create a RecordReaderWrapper", var5); - } - } - - @Override - protected FileStatus[] listStatus(JobConf job) throws IOException { - Path deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)); - TokenCache.obtainTokensForNamenodes(job.getCredentials(), new Path[]{deltaRootPath}, job); - - List filteredDeltaFiles = DeltaHelper.listDeltaFiles(deltaRootPath, job); - return filteredDeltaFiles.toArray(new FileStatus[filteredDeltaFiles.size()]); - } -} diff --git a/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java b/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java deleted file mode 100644 index 9392dd5fee3..00000000000 --- a/hive/src/main/java/io/delta/hive/DeltaRecordReaderWrapper.java +++ /dev/null @@ -1,106 +0,0 @@ -package io.delta.hive; - -import com.google.common.base.Joiner; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import scala.collection.JavaConverters; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; -import org.apache.parquet.hadoop.ParquetInputFormat; -import org.apache.spark.sql.delta.DeltaHelper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DeltaRecordReaderWrapper extends ParquetRecordReaderWrapper { - - private static final Logger LOG = LoggerFactory.getLogger(DeltaRecordReaderWrapper.class); - - private Writable[] partitionWritable = null; - - public DeltaRecordReaderWrapper(ParquetInputFormat newInputFormat, InputSplit oldSplit, JobConf oldJobConf, Reporter reporter) throws IOException, InterruptedException { - super(newInputFormat, oldSplit, oldJobConf, reporter); - - if (!(oldSplit instanceof FileSplit)) { - throw new IllegalArgumentException("Unknown split type: " + oldSplit); - } else { - final String columnNameProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES); - final String columnTypeProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES); - LOG.info("Delta partition cols: " + columnNameProperty + " with types: " + columnTypeProperty); - - if (columnNameProperty == null || columnNameProperty.trim().length() == 0 - || columnTypeProperty == null || columnTypeProperty.trim().length() == 0) { - LOG.info("No partition info is provided..."); - } else { - // generate partition writale values which will be appended after data values from parquet - final List columnNames; - final List columnTypes; - if (columnNameProperty.length() == 0) { - columnNames = new ArrayList(); - } else { - columnNames = Arrays.asList(columnNameProperty.split(",")); - } - if (columnTypeProperty.length() == 0) { - columnTypes = new ArrayList(); - } else { - columnTypes = Arrays.asList(columnTypeProperty.split(":")); - } - - Path filePath = ((FileSplit)oldSplit).getPath(); - - Map parsedPartitions = JavaConverters.mapAsJavaMap( - DeltaHelper.parsePathPartition(filePath, JavaConverters.asScalaBufferConverter(columnNames).asScala().toSeq())); - - partitionWritable = new Writable[columnNames.size()]; - // inspect partition values - for(int i=0; i < columnNames.size(); i++) { - ObjectInspector oi = PrimitiveObjectInspectorFactory - .getPrimitiveWritableObjectInspector(TypeInfoFactory - .getPrimitiveTypeInfo(columnTypes.get(i))); - - partitionWritable[i] = (Writable)ObjectInspectorConverters.getConverter( - PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi).convert(parsedPartitions.get(columnNames.get(i))); - } - LOG.info("Parsed partition values from " + filePath.toString() + " list: " + Joiner.on(",").withKeyValueSeparator("=").join(parsedPartitions) - + ", partitionWritable length:" + partitionWritable.length); - } - } - } - - @Override - public boolean next(NullWritable key, ArrayWritable value) throws IOException { - boolean hasNext = super.next(key, value); - if (partitionWritable != null && partitionWritable.length != 0) { - // append partition values to data values - for (int i=0; i < partitionWritable.length; i++) { - value.get()[value.get().length - partitionWritable.length + i] = partitionWritable[i]; - } - } - return hasNext; - } - - @Override - public ArrayWritable createValue() { - ArrayWritable value = super.createValue(); - if (partitionWritable != null && partitionWritable.length != 0) { - for (int i=0; i < partitionWritable.length; i++) { - value.get()[value.get().length - partitionWritable.length + i] = partitionWritable[i]; - } - } - return value; - } -} diff --git a/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java b/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java deleted file mode 100644 index 61d9826c926..00000000000 --- a/hive/src/main/java/io/delta/hive/DeltaStorageHandler.java +++ /dev/null @@ -1,202 +0,0 @@ -package io.delta.hive; - -import com.google.common.base.Joiner; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import scala.collection.JavaConverters; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.HiveMetaHook; -import org.apache.hadoop.hive.metastore.MetaStoreUtils; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.index.IndexSearchCondition; -import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; -import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler; -import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.StringUtils; -import org.apache.spark.sql.delta.DeltaHelper; -import org.apache.spark.sql.delta.DeltaPushFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DeltaStorageHandler extends DefaultStorageHandler - implements HiveMetaHook, HiveStoragePredicateHandler { - - private static final Logger LOG = LoggerFactory.getLogger(DeltaStorageHandler.class); - - public static final String DELTA_TABLE_PATH = "delta.table.path"; - public static final String DELTA_PARTITION_COLS_NAMES = "delta.partition.columns"; - public static final String DELTA_PARTITION_COLS_TYPES = "delta.partition.columns.types"; - - @Override - public Class getInputFormatClass() { - return DeltaInputFormat.class; - } - - @Override - public Class getSerDeClass() { - return ParquetHiveSerDe.class; - } - - @Override - public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { - super.configureInputJobProperties(tableDesc, jobProperties); - jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)); - jobProperties.put(DELTA_PARTITION_COLS_NAMES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_NAMES)); - jobProperties.put(DELTA_PARTITION_COLS_TYPES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_TYPES)); - } - - @Override - public DecomposedPredicate decomposePredicate( - JobConf jobConf, Deserializer deserializer, - ExprNodeDesc predicate) { - // Get the delta root path - String deltaRootPath = jobConf.get(DELTA_TABLE_PATH); - // Get the partitionColumns of Delta - List partitionColumns = JavaConverters.seqAsJavaList( - DeltaHelper.getPartitionCols(new Path(deltaRootPath))); - LOG.info("delta partitionColumns is " + Joiner.on(",").join(partitionColumns)); - - IndexPredicateAnalyzer analyzer = newIndexPredicateAnalyzer(partitionColumns); - - List conditions = new ArrayList(); - ExprNodeGenericFuncDesc pushedPredicate = null; - ExprNodeGenericFuncDesc residualPredicate = - (ExprNodeGenericFuncDesc) analyzer.analyzePredicate(predicate, conditions); - - for (List searchConditions : decompose(conditions).values()) { - // still push back the pushedPredicate to residualPredicate - residualPredicate = - extractResidualCondition(analyzer, searchConditions, residualPredicate); - pushedPredicate = - extractStorageHandlerCondition(analyzer, searchConditions, pushedPredicate); - } - - LOG.info("pushedPredicate:" + (pushedPredicate == null? "null":pushedPredicate.getExprString()) - + ",residualPredicate" + residualPredicate); - - DecomposedPredicate decomposedPredicate = new DecomposedPredicate(); - decomposedPredicate.pushedPredicate = pushedPredicate; - decomposedPredicate.residualPredicate = residualPredicate; - return decomposedPredicate; - } - - private IndexPredicateAnalyzer newIndexPredicateAnalyzer( - List partitionColumns) { - IndexPredicateAnalyzer analyzer = new IndexPredicateAnalyzer(); - for (String col : partitionColumns) { - // Supported filter exprs on partition column to be pushed down to delta - analyzer.addComparisonOp(col, DeltaPushFilter.supportedPushDownUDFs()); - } - return analyzer; - } - - private static Map> decompose( - List searchConditions) { - Map> result = - new HashMap>(); - for (IndexSearchCondition condition : searchConditions) { - List conditions = result.get(condition.getColumnDesc().getColumn()); - if (conditions == null) { - conditions = new ArrayList(); - result.put(condition.getColumnDesc().getColumn(), conditions); - } - conditions.add(condition); - } - return result; - } - - private static ExprNodeGenericFuncDesc extractResidualCondition( - IndexPredicateAnalyzer analyzer, - List searchConditions, ExprNodeGenericFuncDesc inputExpr) { - if (inputExpr == null) { - return analyzer.translateOriginalConditions(searchConditions); - } - List children = new ArrayList(); - children.add(analyzer.translateOriginalConditions(searchConditions)); - children.add(inputExpr); - return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, - FunctionRegistry.getGenericUDFForAnd(), children); - } - - private static ExprNodeGenericFuncDesc extractStorageHandlerCondition( - IndexPredicateAnalyzer analyzer, - List searchConditions, ExprNodeGenericFuncDesc inputExpr) { - if (inputExpr == null) { - return analyzer.translateSearchConditions(searchConditions); - } - List children = new ArrayList(); - children.add(analyzer.translateSearchConditions(searchConditions)); - children.add(inputExpr); - return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, - FunctionRegistry.getGenericUDFForAnd(), children); - } - - public HiveMetaHook getMetaHook() { - return this; - } - - public void preCreateTable(Table tbl) throws MetaException { - boolean isExternal = MetaStoreUtils.isExternalTable(tbl); - if (!isExternal) { - throw new MetaException("HiveOnDelta should be an external table."); - } else if (tbl.getPartitionKeysSize() > 0) { - throw new MetaException("HiveOnDelta does not support to create a partition hive table"); - } - - String deltaRootString = tbl.getSd().getLocation(); - try { - if (deltaRootString == null || deltaRootString.trim().length() == 0) { - throw new MetaException("table location should be set when creating table"); - } else { - Path deltaPath = new Path(deltaRootString); - FileSystem fs = deltaPath.getFileSystem(getConf()); - if (!fs.exists(deltaPath)) { - throw new MetaException("delta.table.path(" + deltaRootString + ") does not exist..."); - } else { - Map partitionProps = JavaConverters.mapAsJavaMap( - DeltaHelper.checkHiveColsInDelta(deltaPath, tbl.getSd().getCols())); - tbl.getSd().getSerdeInfo().getParameters().putAll(partitionProps); - tbl.getSd().getSerdeInfo().getParameters().put(DELTA_TABLE_PATH, deltaRootString); - LOG.info("write partition cols/types to table properties " + - Joiner.on(",").withKeyValueSeparator("=").join(partitionProps)); - } - } - } catch (Exception e) { - throw new MetaException(StringUtils.stringifyException(e)); - } - } - - public void rollbackCreateTable(Table table) throws MetaException { - // nothing to do - } - - public void commitCreateTable(Table table) throws MetaException { - // nothing to do - } - - public void preDropTable(Table table) throws MetaException { - // nothing to do - } - - public void rollbackDropTable(Table table) throws MetaException { - // nothing to do - } - - public void commitDropTable(Table tbl, boolean deleteData) throws MetaException { - // nothing to do - } -} diff --git a/hive/src/main/java/io/delta/hive/HiveInputFormat.java b/hive/src/main/java/io/delta/hive/HiveInputFormat.java deleted file mode 100644 index fb27cbe7205..00000000000 --- a/hive/src/main/java/io/delta/hive/HiveInputFormat.java +++ /dev/null @@ -1,15 +0,0 @@ -package io.delta.hive; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.JobConf; - -public class HiveInputFormat extends org.apache.hadoop.hive.ql.io.HiveInputFormat { - @Override - protected void pushProjectionsAndFilters(JobConf jobConf, Class inputFormatClass, Path splitPath, boolean nonNative) { - if (inputFormatClass == DeltaInputFormat.class ) { - super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, false); - } else { - super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, nonNative); - } - } -} diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala new file mode 100644 index 00000000000..80e70c172be --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -0,0 +1,46 @@ +package io.delta.hive + +import java.io.IOException + +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport +import org.apache.hadoop.io.ArrayWritable +import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.security.TokenCache +import org.apache.parquet.hadoop.ParquetInputFormat +import org.apache.spark.sql.delta.DeltaHelper +import org.slf4j.LoggerFactory + +class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends FileInputFormat[NullWritable, ArrayWritable] { + + private val LOG = LoggerFactory.getLogger(classOf[DeltaInputFormat]) + + def this() { + this(new ParquetInputFormat[ArrayWritable](classOf[DataWritableReadSupport])) + } + + override def getRecordReader(split: InputSplit, job: JobConf, reporter: Reporter): RecordReader[NullWritable, ArrayWritable] = { + if (Utilities.getUseVectorizedInputFileFormat(job)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Using vectorized record reader") + } + throw new IOException("Currently not support Delta VectorizedReader") + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Using row-mode record reader") + } + new DeltaRecordReaderWrapper(this.realInput, split, job, reporter) + } + } + + override def listStatus(job: JobConf): Array[FileStatus] = { + val deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)) + TokenCache.obtainTokensForNamenodes(job.getCredentials(), Array(deltaRootPath), job) + + val filteredDeltaFiles = DeltaHelper.listDeltaFiles(deltaRootPath, job) + filteredDeltaFiles.toArray(new Array[FileStatus](filteredDeltaFiles.size)) + } +} diff --git a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala new file mode 100644 index 00000000000..4d946cbbd35 --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala @@ -0,0 +1,81 @@ +package io.delta.hive + +import com.google.common.base.Joiner +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory +import org.apache.hadoop.io.ArrayWritable +import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.FileSplit +import org.apache.hadoop.mapred.InputSplit +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.Reporter +import org.apache.parquet.hadoop.ParquetInputFormat +import org.apache.spark.sql.delta.DeltaHelper +import org.slf4j.LoggerFactory + +class DeltaRecordReaderWrapper(newInputFormat: ParquetInputFormat[ArrayWritable], oldSplit: InputSplit, oldJobConf: JobConf, reporter: Reporter) extends ParquetRecordReaderWrapper(newInputFormat, oldSplit, oldJobConf, reporter) { + + private val LOG = LoggerFactory.getLogger(classOf[DeltaRecordReaderWrapper]) + + private val partitionWritable: Array[Writable] = + if (!oldSplit.isInstanceOf[FileSplit]) { + throw new IllegalArgumentException("Unknown split type: " + oldSplit) + } else { + val columnNameProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES) + val columnTypeProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES) + LOG.info("Delta partition cols: " + columnNameProperty + " with types: " + columnTypeProperty) + + if (columnNameProperty == null || columnNameProperty.trim().length() == 0 + || columnTypeProperty == null || columnTypeProperty.trim().length() == 0) { + LOG.info("No partition info is provided...") + null + } else { + // generate partition writale values which will be appended after data values from parquet + val columnNames = columnNameProperty.split(",") + val columnTypes = columnTypeProperty.split(":") + + val filePath = oldSplit.asInstanceOf[FileSplit].getPath() + val parsedPartitions = DeltaHelper.parsePathPartition(filePath, columnNames).asJava + + val partitionWritable = new Array[Writable](columnNames.length) + // inspect partition values + for (i <- 0 until columnNames.length) { + val oi = PrimitiveObjectInspectorFactory + .getPrimitiveWritableObjectInspector(TypeInfoFactory + .getPrimitiveTypeInfo(columnTypes(i))) + + partitionWritable(i) = ObjectInspectorConverters.getConverter( + PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi).convert(parsedPartitions.get(columnNames(i))).asInstanceOf[Writable] + } + LOG.info("Parsed partition values from " + filePath.toString() + " list: " + Joiner.on(",").withKeyValueSeparator("=").join(parsedPartitions) + + ", partitionWritable length:" + partitionWritable.length) + partitionWritable + } + } + + override def next(key: NullWritable, value: ArrayWritable): Boolean = { + val hasNext = super.next(key, value) + if (partitionWritable != null && partitionWritable.length != 0) { + // append partition values to data values + for (i <- 0 until partitionWritable.length) { + value.get()(value.get().length - partitionWritable.length + i) = partitionWritable(i) + } + } + hasNext + } + + override def createValue(): ArrayWritable = { + val value = super.createValue() + if (partitionWritable != null && partitionWritable.length != 0) { + for (i <- 0 until partitionWritable.length) { + value.get()(value.get().length - partitionWritable.length + i) = partitionWritable(i) + } + } + value + } +} diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala new file mode 100644 index 00000000000..1223eadcfb6 --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -0,0 +1,180 @@ +package io.delta.hive + +import com.google.common.base.Joiner +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.HiveMetaHook +import org.apache.hadoop.hive.metastore.MetaStoreUtils +import org.apache.hadoop.hive.metastore.api.MetaException +import org.apache.hadoop.hive.metastore.api.Table +import org.apache.hadoop.hive.ql.exec.FunctionRegistry +import org.apache.hadoop.hive.ql.index.IndexSearchCondition +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe +import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler +import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler +import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler.DecomposedPredicate +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory +import org.apache.hadoop.mapred.InputFormat +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.util.StringUtils +import org.apache.spark.sql.delta.DeltaHelper +import org.apache.spark.sql.delta.DeltaPushFilter +import org.slf4j.LoggerFactory + +class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with HiveStoragePredicateHandler { + + import DeltaStorageHandler._ + + private val LOG = LoggerFactory.getLogger(classOf[DeltaStorageHandler]) + + + override def getInputFormatClass: Class[_ <: InputFormat[_, _]] = classOf[DeltaInputFormat] + + override def getSerDeClass(): Class[_ <: AbstractSerDe] = classOf[ParquetHiveSerDe] + + override def configureInputJobProperties(tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { + super.configureInputJobProperties(tableDesc, jobProperties) + jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)) + jobProperties.put(DELTA_PARTITION_COLS_NAMES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_NAMES)) + jobProperties.put(DELTA_PARTITION_COLS_TYPES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_TYPES)) + } + + override def decomposePredicate(jobConf: JobConf, deserializer: Deserializer, predicate: ExprNodeDesc): DecomposedPredicate = { + // Get the delta root path + val deltaRootPath = jobConf.get(DELTA_TABLE_PATH) + // Get the partitionColumns of Delta + val partitionColumns = DeltaHelper.getPartitionCols(new Path(deltaRootPath)) + LOG.info("delta partitionColumns is " + partitionColumns.mkString(", ")) + + val analyzer = newIndexPredicateAnalyzer(partitionColumns) + + val conditions = new java.util.ArrayList[IndexSearchCondition]() + var pushedPredicate: ExprNodeGenericFuncDesc = null + var residualPredicate = analyzer.analyzePredicate(predicate, conditions).asInstanceOf[ExprNodeGenericFuncDesc] + for (searchConditions <- decompose(conditions).values) { + // still push back the pushedPredicate to residualPredicate + residualPredicate = + extractResidualCondition(analyzer, searchConditions, residualPredicate) + pushedPredicate = + extractStorageHandlerCondition(analyzer, searchConditions, pushedPredicate) + } + + LOG.info("pushedPredicate:" + (if (pushedPredicate == null) "null" else pushedPredicate.getExprString()) + + ",residualPredicate" + residualPredicate) + + val decomposedPredicate = new DecomposedPredicate() + decomposedPredicate.pushedPredicate = pushedPredicate + decomposedPredicate.residualPredicate = residualPredicate + decomposedPredicate + } + + private def newIndexPredicateAnalyzer(partitionColumns: Seq[String]): IndexPredicateAnalyzer = { + val analyzer = new IndexPredicateAnalyzer() + for (col <- partitionColumns) { + // Supported filter exprs on partition column to be pushed down to delta + analyzer.addComparisonOp(col, DeltaPushFilter.supportedPushDownUDFs: _*) + } + analyzer + } + + private def decompose(searchConditions: java.util.ArrayList[IndexSearchCondition]): Map[String, java.util.ArrayList[IndexSearchCondition]] = { + val result = mutable.Map[String, java.util.ArrayList[IndexSearchCondition]]() + for (condition <- searchConditions.asScala) { + val conditions = result.getOrElseUpdate(condition.getColumnDesc().getColumn(), new java.util.ArrayList[IndexSearchCondition]()) + conditions.add(condition) + } + result.toMap + } + + private def extractResidualCondition( + analyzer: IndexPredicateAnalyzer, + searchConditions: java.util.ArrayList[IndexSearchCondition], + inputExpr: ExprNodeGenericFuncDesc): ExprNodeGenericFuncDesc = { + if (inputExpr == null) { + analyzer.translateOriginalConditions(searchConditions) + } else { + val children = new java.util.ArrayList[ExprNodeDesc] + children.add(analyzer.translateOriginalConditions(searchConditions)) + children.add(inputExpr) + new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), children) + } + } + + private def extractStorageHandlerCondition( + analyzer: IndexPredicateAnalyzer, + searchConditions: java.util.ArrayList[IndexSearchCondition], + inputExpr: ExprNodeGenericFuncDesc): ExprNodeGenericFuncDesc = { + if (inputExpr == null) { + analyzer.translateSearchConditions(searchConditions) + } else { + val children = new java.util.ArrayList[ExprNodeDesc] + children.add(analyzer.translateSearchConditions(searchConditions)) + children.add(inputExpr) + new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, + FunctionRegistry.getGenericUDFForAnd(), children) + } + } + + override def getMetaHook: HiveMetaHook = this + + override def preCreateTable(tbl: Table): Unit = { + val isExternal = MetaStoreUtils.isExternalTable(tbl) + if (!isExternal) { + throw new MetaException("HiveOnDelta should be an external table.") + } else if (tbl.getPartitionKeysSize() > 0) { + throw new MetaException("HiveOnDelta does not support to create a partition hive table") + } + + val deltaRootString = tbl.getSd().getLocation() + if (deltaRootString == null || deltaRootString.trim().length() == 0) { + throw new MetaException("table location should be set when creating table") + } else { + val deltaPath = new Path(deltaRootString) + val fs = deltaPath.getFileSystem(getConf()) + if (!fs.exists(deltaPath)) { + throw new MetaException("delta.table.path(" + deltaRootString + ") does not exist...") + } else { + val partitionProps = DeltaHelper.checkHiveColsInDelta(deltaPath, tbl.getSd().getCols()) + tbl.getSd().getSerdeInfo().getParameters().putAll(partitionProps.asJava) + tbl.getSd().getSerdeInfo().getParameters().put(DELTA_TABLE_PATH, deltaRootString) + LOG.info("write partition cols/types to table properties " + + partitionProps.map(kv => s"${kv._1}=${kv._2}").mkString(", ")) + } + } + } + + override def rollbackCreateTable(table: Table): Unit = { + // TODO What should we do? + } + + override def commitCreateTable(table: Table): Unit = { + // TODO What should we do? + } + + override def preDropTable(table: Table): Unit = { + // TODO What should we do? + } + + override def rollbackDropTable(table: Table): Unit = { + // TODO What should we do? + } + + override def commitDropTable(table: Table, b: Boolean): Unit = { + // TODO What should we do? + } +} + +object DeltaStorageHandler { + val DELTA_TABLE_PATH = "delta.table.path" + val DELTA_PARTITION_COLS_NAMES = "delta.partition.columns" + val DELTA_PARTITION_COLS_TYPES = "delta.partition.columns.types" +} diff --git a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala new file mode 100644 index 00000000000..12c728c1569 --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala @@ -0,0 +1,15 @@ +package io.delta.hive + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.JobConf + +class HiveInputFormat extends org.apache.hadoop.hive.ql.io.HiveInputFormat { + + override def pushProjectionsAndFilters(jobConf: JobConf, inputFormatClass: Class[_], splitPath: Path, nonNative: Boolean) = { + if (inputFormatClass == classOf[DeltaInputFormat]) { + super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, false) + } else { + super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, nonNative) + } + } +} From e932a923e017af9f068731eda5cb934148682a33 Mon Sep 17 00:00:00 2001 From: windpiger Date: Wed, 27 Nov 2019 14:42:54 +0800 Subject: [PATCH 012/291] minor modify README.md (#8) --- README.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 016f683d01a..c2f4d6c727e 100644 --- a/README.md +++ b/README.md @@ -20,11 +20,13 @@ This project generates a single uber jar containing Delta Lake and all it transi - To generate the uber jar, run `build/sbt core/compile` - To test the uber jar, run `build/sbt coreTest/test` -## 2. Hive connector (WIP) +## 2. Hive connector This project contains all the code needed to make Hive read Delta Lake tables. - To compile the project, run `build/sbt hive/compile` - To test the project, run `build/sbt hive/test` -- To generate the connector jar run `bulld/sbt hive/package` +- To generate the connector jar run `build/sbt hive/package` + +Config `HIVE_AUX_JARS_PATH` in hive-env.sh with above two jars(uber jar and Hive connector jar) Refer to [SBT docs](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html) for more commands. From 5a2198b4ca3873004fbc68965cfe7d1993c2bb4f Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 28 Nov 2019 09:20:17 -0800 Subject: [PATCH 013/291] Improve error messages for unsupported features in Hive (#9) This PR makes some minor improvements in the error messages for unsupported features. It also adds the table property `spark.sql.sources.provider` so that a Delta table created by Hive can be read by Spark 3.0.0+ when they share the same metastore. --- .../io/delta/hive/DeltaOutputFormat.scala | 26 +++++++ .../io/delta/hive/DeltaStorageHandler.scala | 32 +++++---- .../io/delta/hive/HiveConnectorSuite.scala | 70 ++++++++++++------- 3 files changed, 89 insertions(+), 39 deletions(-) create mode 100644 hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala diff --git a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala new file mode 100644 index 00000000000..30f01814a2a --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala @@ -0,0 +1,26 @@ +package io.delta.hive + +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.io.{ArrayWritable, NullWritable} +import org.apache.hadoop.mapred.{JobConf, OutputFormat, RecordWriter} +import org.apache.hadoop.util.Progressable + +/** + * This class is not a real implementation. We use it to prevent from writing to a Delta table in + * Hive before we support it. + */ +class DeltaOutputFormat extends OutputFormat[NullWritable, ArrayWritable] { + + private def writingNotSupported[T](): T = { + throw new UnsupportedOperationException( + "Writing to a Delta table in Hive is not supported. Please use Spark to write.") + } + + override def getRecordWriter( + ignored: FileSystem, + job: JobConf, + name: String, + progress: Progressable): RecordWriter[NullWritable, ArrayWritable] = writingNotSupported() + + override def checkOutputSpecs(ignored: FileSystem, job: JobConf): Unit = writingNotSupported() +} diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 1223eadcfb6..754307330df 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -1,10 +1,7 @@ package io.delta.hive -import com.google.common.base.Joiner import scala.collection.JavaConverters._ import scala.collection.mutable - -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.HiveMetaHook import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -22,9 +19,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.util.StringUtils +import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} import org.apache.spark.sql.delta.DeltaHelper import org.apache.spark.sql.delta.DeltaPushFilter import org.slf4j.LoggerFactory @@ -35,9 +30,15 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H private val LOG = LoggerFactory.getLogger(classOf[DeltaStorageHandler]) - override def getInputFormatClass: Class[_ <: InputFormat[_, _]] = classOf[DeltaInputFormat] + /** + * Returns a special [[OutputFormat]] to prevent from writing to a Delta table in Hive before we + * support it. We have to give Hive some class when creating a table, hence we have to implement + * an [[OutputFormat]] which throws an exception when Hive is using it. + */ + override def getOutputFormatClass: Class[_ <: OutputFormat[_, _]] = classOf[DeltaOutputFormat] + override def getSerDeClass(): Class[_ <: AbstractSerDe] = classOf[ParquetHiveSerDe] override def configureInputJobProperties(tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { @@ -127,13 +128,19 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H override def getMetaHook: HiveMetaHook = this override def preCreateTable(tbl: Table): Unit = { - val isExternal = MetaStoreUtils.isExternalTable(tbl) - if (!isExternal) { - throw new MetaException("HiveOnDelta should be an external table.") - } else if (tbl.getPartitionKeysSize() > 0) { - throw new MetaException("HiveOnDelta does not support to create a partition hive table") + if (!MetaStoreUtils.isExternalTable(tbl)) { + throw new UnsupportedOperationException( + s"The type of table ${tbl.getDbName}:${tbl.getTableName} is ${tbl.getTableType}." + + "Only external Delta tables can be read in Hive right now") } + if (tbl.getPartitionKeysSize > 0) { + throw new MetaException( + s"Found partition columns " + + s"(${tbl.getPartitionKeys.asScala.map(_.getName).mkString(",")}) in table " + + s"${tbl.getDbName}:${tbl.getTableName}. The partition columns in a Delta table " + + s"will be read from its own metadata and should not be set manually.") } + val deltaRootString = tbl.getSd().getLocation() if (deltaRootString == null || deltaRootString.trim().length() == 0) { throw new MetaException("table location should be set when creating table") @@ -146,6 +153,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H val partitionProps = DeltaHelper.checkHiveColsInDelta(deltaPath, tbl.getSd().getCols()) tbl.getSd().getSerdeInfo().getParameters().putAll(partitionProps.asJava) tbl.getSd().getSerdeInfo().getParameters().put(DELTA_TABLE_PATH, deltaRootString) + tbl.getParameters.put("spark.sql.sources.provider", "DELTA") LOG.info("write partition cols/types to table properties " + partitionProps.map(kv => s"${kv._1}=${kv._2}").mkString(", ")) } diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index a8bea7a458d..67b4e6be5ee 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -19,23 +19,18 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { DeltaLog.clearCache() } - test("DDL: HiveOnDelta should be a external table ") { - withTable("deltaTbl") { - withTempDir { dir => - val e = intercept[Exception] { - runQuery( - s""" - |create table deltaTbl(a string, b int) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' - """.stripMargin - ) - }.getMessage - assert(e.contains("HiveOnDelta should be an external table")) - } + test("should not allow to create a non external Delta table") { + val e = intercept[Exception] { + runQuery( + s""" + |create table deltaTbl(a string, b int) + |stored by 'io.delta.hive.DeltaStorageHandler'""".stripMargin + ) } + assert(e.getMessage != null && e.getMessage.contains("Only external Delta tables")) } - test("DDL: location should be set when creating table") { + test("location should be set when creating table") { withTable("deltaTbl") { val e = intercept[Exception] { runQuery( @@ -49,24 +44,45 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } } - test("DDL: HiveOnDelta should not be a partitioned hive table") { + test("should not allow to specify partition columns") { + withTempDir { dir => + val e = intercept[Exception] { + runQuery( + s""" + |CREATE EXTERNAL TABLE deltaTbl(a STRING, b INT) + |PARTITIONED BY(c STRING) + |STORED BY 'io.delta.hive.DeltaStorageHandler' + |LOCATION '${dir.getCanonicalPath}' """.stripMargin) + } + assert(e.getMessage != null && e.getMessage.matches( + "(?s).*partition columns.*should not be set manually.*")) + } + } + + test("should not allow to write to a Delta table") { withTable("deltaTbl") { withTempDir { dir => + withSparkSession { spark => + import spark.implicits._ + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + testData.toDS.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) + |STORED BY 'io.delta.hive.DeltaStorageHandler' + |LOCATION '${dir.getCanonicalPath}'""".stripMargin) val e = intercept[Exception] { - runQuery( - s""" - |create external table deltaTbl(a string, b int) - |partitioned by(c string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' - """.stripMargin - ) - }.getMessage - assert(e.contains("HiveOnDelta does not support to create a partition hive table")) + runQuery("INSERT INTO deltaTbl(a, b) VALUES(123, 'foo')") + } + assert(e.getMessage != null && e.getMessage.contains( + "Writing to a Delta table in Hive is not supported")) } } } - test("DDL: the delta root path should be existed when create hive table") { + test("the delta root path should be existed when create hive table") { withTable("deltaTbl") { withTempDir { dir => JavaUtils.deleteRecursively(dir) @@ -84,7 +100,7 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } } - test("DDL: when creating hive table on a partitioned delta, " + + test("when creating hive table on a partitioned delta, " + "the partition columns should be after data columns") { withTable("deltaTbl") { withTempDir { dir => @@ -110,7 +126,7 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } // check column number & column name - test("DDL: Hive schema should match delta's schema") { + test("Hive schema should match delta's schema") { withTable("deltaTbl") { withTempDir { dir => val testData = (0 until 10).map(x => (x, s"foo${x % 2}", s"test${x % 3}")) From 206c268f052324e8d9697180b93bd1f97049df4b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 5 Dec 2019 21:44:04 -0800 Subject: [PATCH 014/291] Convert AddFile to FileStatus directly to save FileSystem RPC calls (#10) This PR creates `FileStatus` directly from `AddFile` to save FileSystem RPC calls. As `AddFile` doesn't have the block locations, we lose the locality. But this is fine today since most of storage systems are on Cloud and the computation is running separately. I also fixes a bug that we return incorrect file paths to Hive when the partition values have some special values that are escaped in the file path. --- .../io/delta/hive/DeltaInputFormat.scala | 4 +- .../apache/spark/sql/delta/DeltaHelper.scala | 77 +++++++++++++++++-- .../io/delta/hive/HiveConnectorSuite.scala | 27 ++++++- 3 files changed, 97 insertions(+), 11 deletions(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index 80e70c172be..b16759b043f 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -39,8 +39,6 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends Fil override def listStatus(job: JobConf): Array[FileStatus] = { val deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)) TokenCache.obtainTokensForNamenodes(job.getCredentials(), Array(deltaRootPath), job) - - val filteredDeltaFiles = DeltaHelper.listDeltaFiles(deltaRootPath, job) - filteredDeltaFiles.toArray(new Array[FileStatus](filteredDeltaFiles.size)) + DeltaHelper.listDeltaFiles(deltaRootPath, job) } } diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 144af6de137..28063782ce8 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -1,10 +1,12 @@ package org.apache.spark.sql.delta -import io.delta.hive.DeltaStorageHandler +import java.net.URI + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.fs.{FileStatus, Path} +import io.delta.hive.DeltaStorageHandler +import org.apache.hadoop.fs._ import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} import org.apache.hadoop.hive.ql.plan.TableScanDesc import org.apache.hadoop.mapred.JobConf @@ -65,26 +67,87 @@ object DeltaHelper extends Logging { } } - def listDeltaFiles(rootPath: Path, job: JobConf): java.util.List[FileStatus] = { + def listDeltaFiles(nonNormalizedPath: Path, job: JobConf): Array[FileStatus] = { + val fs = nonNormalizedPath.getFileSystem(job) + // We need to normalize the table path so that all paths we return to Hive will be normalized + // This is necessary because `HiveInputFormat.pushProjectionsAndFilters` will try to figure out + // which table a split path belongs to by comparing the split path with the normalized (? I have + // not yet confirmed this) table paths. + // TODO The assumption about Path in Hive is too strong, we should try to see if we can fail if + // `pushProjectionsAndFilters` doesn't find a table for a Delta split path. + val rootPath = fs.makeQualified(nonNormalizedPath) val deltaLog = DeltaLog.forTable(spark, rootPath) // get the snapshot of the version val snapshotToUse = deltaLog.snapshot - val fs = rootPath.getFileSystem(job) + // TODO Verify the table schema is consistent with `snapshotToUse.metadata`. // get the partition prune exprs val filterExprSerialized = job.get(TableScanDesc.FILTER_EXPR_CONF_STR) val convertedFilterExpr = DeltaPushFilter.partitionFilterConverter(filterExprSerialized) + // The default value 128M is the same as the default value of + // "spark.sql.files.maxPartitionBytes" in Spark. It's also the default parquet row group size + // which is usually the best split size for parquet files. + val blockSize = job.getLong("parquet.block.size", 128L * 1024 * 1024) + // selected files to Hive to be processed DeltaLog.filterFileList( snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), convertedFilterExpr) .as[AddFile](SingleAction.addFileEncoder) - .collect().par.map { f => + .collect().map { f => logInfo(s"selected delta file ${f.path} under $rootPath") - fs.getFileStatus(new Path(rootPath, f.path)) - }.toList.asJava + toFileStatus(fs, rootPath, f, blockSize) + } + } + + /** + * Convert an [[AddFile]] to Hadoop's [[FileStatus]]. + * + * @param root the table path which will be used to create the real path from relative path. + */ + private def toFileStatus(fs: FileSystem, root: Path, f: AddFile, blockSize: Long): FileStatus = { + val status = new FileStatus( + f.size, // length + false, // isDir + 1, // blockReplication, FileInputFormat doesn't use this + blockSize, // blockSize + f.modificationTime, // modificationTime + absolutePath(fs, root, f.path) // path + ) + // We don't have `blockLocations` in `AddFile`. However, fetching them by calling + // `getFileStatus` for each file is unacceptable because that's pretty inefficient and it will + // make Delta look worse than a parquet table because of these FileSystem RPC calls. + // + // But if we don't set the block locations, [[FileInputFormat]] will try to fetch them. Hence, + // we create a `LocatedFileStatus` with dummy block locations to save FileSystem RPC calls. We + // lose the locality but this is fine today since most of storage systems are on Cloud and the + // computation is running separately. + // + // An alternative solution is using "listStatus" recursively to get all `FileStatus`s and keep + // those present in `AddFile`s. This is much cheaper and the performance should be the same as a + // parquet table. However, it's pretty complicated as we need to be careful to avoid listing + // unnecessary directories. So we decide to not do this right now. + val dummyBlockLocations = + Array(new BlockLocation(Array("localhost:50010"), Array("localhost"), 0, f.size)) + new LocatedFileStatus(status, dummyBlockLocations) + } + + /** + * Create an absolute [[Path]] from `child` using the `root` path if `child` is a relative path. + * Return a [[Path]] version of child` if it is an absolute path. + * + * @param child an escaped string read from Delta's [[AddFile]] directly which requires to + * unescape before creating the [[Path]] object. + */ + private def absolutePath(fs: FileSystem, root: Path, child: String): Path = { + val p = new Path(new URI(child)) + if (p.isAbsolute) { + fs.makeQualified(p) + } else { + new Path(root, p) + } } def checkHiveColsInDelta( diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index 67b4e6be5ee..3897346e712 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -408,4 +408,29 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } } } -} \ No newline at end of file + + test("read a partitioned table that contains special chars in a partition column") { + withTable("deltaPartitionTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"+ =%${x % 2}")) + + withSparkSession { spark => + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta") + .partitionBy("c2").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaPartitionTbl(c1 int, c2 string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + // TODO Read partition values from `AddFile.partitionValues` to fix incorrect escaped + // partition values. + runQuery("select * from deltaPartitionTbl") + } + } + } +} From ea203636e075f9d7fbd23c805ba91869e52daa8e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 13 Dec 2019 12:36:06 -0800 Subject: [PATCH 015/291] Consistent schema in Hive and Delta metadata (#11) Right now we require the partition columns should be after the data columns. This PR adds a new DeltaInputSplit to remove the above limitation and also adds validation to ensure Hive's schema is always consistent with Delta's metadata regarding column types and order. --- .../java/io/delta/hive/DeltaInputSplit.java | 56 ++++ .../io/delta/hive/DeltaInputFormat.scala | 91 ++++- .../delta/hive/DeltaRecordReaderWrapper.scala | 98 +++--- .../io/delta/hive/DeltaStorageHandler.scala | 74 ++-- .../io/delta/hive/PartitionColumnInfo.scala | 33 ++ .../apache/spark/sql/delta/DeltaHelper.scala | 174 ++++++++-- .../io/delta/hive/HiveConnectorSuite.scala | 317 ++++++++++++------ .../scala/io/delta/hive/test/HiveTest.scala | 29 ++ 8 files changed, 640 insertions(+), 232 deletions(-) create mode 100644 hive/src/main/java/io/delta/hive/DeltaInputSplit.java create mode 100644 hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala diff --git a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java new file mode 100644 index 00000000000..a3abdb89e2b --- /dev/null +++ b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java @@ -0,0 +1,56 @@ +package io.delta.hive; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; + +/** + * A special {@link FileSplit} that holds the corresponding partition information of the file. + * + * This file is written in Java because we need to call two different constructors of + * {@link FileSplit} but Scala doesn't support it. + */ +public class DeltaInputSplit extends FileSplit { + + private PartitionColumnInfo[] partitionColumns; + + protected DeltaInputSplit() { + super(); + partitionColumns = new PartitionColumnInfo[0]; + } + + public DeltaInputSplit(Path file, long start, long length, String[] hosts, PartitionColumnInfo[] partitionColumns) { + super(file, start, length, hosts); + this.partitionColumns = partitionColumns; + } + + public DeltaInputSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts, PartitionColumnInfo[] partitionColumns) { + super(file, start, length, hosts, inMemoryHosts); + this.partitionColumns = partitionColumns; + } + + public PartitionColumnInfo[] getPartitionColumns() { + return partitionColumns; + } + + public void write(DataOutput out) throws IOException { + super.write(out); + out.writeInt(partitionColumns.length); + for (PartitionColumnInfo partitionColumn : partitionColumns) { + partitionColumn.write(out); + } + } + + public void readFields(DataInput in) throws IOException { + super.readFields(in); + int size = in.readInt(); + partitionColumns = new PartitionColumnInfo[size]; + for (int i = 0; i < size; i++) { + PartitionColumnInfo partitionColumn = new PartitionColumnInfo(); + partitionColumn.readFields(in); + partitionColumns[i] = partitionColumn; + } + } +} diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index b16759b043f..ea0ba4a36c5 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -1,44 +1,109 @@ package io.delta.hive import java.io.IOException +import java.net.URI import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport -import org.apache.hadoop.io.ArrayWritable -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{ArrayWritable, NullWritable, Writable} import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.security.TokenCache import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.delta.DeltaHelper import org.slf4j.LoggerFactory +/** + * A special [[InputFormat]] to wrap [[ParquetInputFormat]] to read a Delta table. + * + * The underlying files in a Delta table are in Parquet format. However, we cannot use the existing + * [[ParquetInputFormat]] to read them directly because they only store data for data columns. + * The values of partition columns are in Delta's metadata. Hence, we need to read them from Delta's + * metadata and re-assemble rows to include partition values and data values from the raw Parquet + * files. + * + * Note: We cannot use the file name to infer partition values because Delta Transaction Log + * Protocol requires "Actual partition values for a file must be read from the transaction log". + * + * In the current implementation, when listing files, we also read the partition values and put them + * into an `Array[PartitionColumnInfo]`. Then create a temp `Map` to store the mapping from the file + * path to `PartitionColumnInfo`s. When creating an [[InputSplit]], we will create a special + * [[FileSplit]] called [[DeltaInputSplit]] to carry over `PartitionColumnInfo`s. + * + * For each reader created from a [[DeltaInputSplit]], we can get all partition column types, the + * locations of a partition column in the schema, and their string values. The reader can build + * [[Writable]] for all partition values, and insert them to the raw row returned by + * [[org.apache.parquet.hadoop.ParquetRecordReader]]. + */ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends FileInputFormat[NullWritable, ArrayWritable] { private val LOG = LoggerFactory.getLogger(classOf[DeltaInputFormat]) + /** + * A temp [[Map]] to store the path uri and its partition information. We build this map in + * `listStatus` and `makeSplit` will use it to retrieve the partition information for each split. + * */ + private var fileToPartition: Map[URI, Array[PartitionColumnInfo]] = Map.empty + def this() { this(new ParquetInputFormat[ArrayWritable](classOf[DataWritableReadSupport])) } override def getRecordReader(split: InputSplit, job: JobConf, reporter: Reporter): RecordReader[NullWritable, ArrayWritable] = { - if (Utilities.getUseVectorizedInputFileFormat(job)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Using vectorized record reader") - } - throw new IOException("Currently not support Delta VectorizedReader") - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Using row-mode record reader") - } - new DeltaRecordReaderWrapper(this.realInput, split, job, reporter) + split match { + case deltaSplit: DeltaInputSplit => + if (Utilities.getUseVectorizedInputFileFormat(job)) { + throw new UnsupportedOperationException( + "Reading Delta tables using Parquet's VectorizedReader is not supported") + } else { + new DeltaRecordReaderWrapper(this.realInput, deltaSplit, job, reporter) + } + case _ => + throw new IllegalArgumentException("Expected DeltaInputSplit but it was: " + split) } } + @throws(classOf[IOException]) override def listStatus(job: JobConf): Array[FileStatus] = { val deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)) TokenCache.obtainTokensForNamenodes(job.getCredentials(), Array(deltaRootPath), job) - DeltaHelper.listDeltaFiles(deltaRootPath, job) + val (files, partitions) = + try { + DeltaHelper.listDeltaFiles(deltaRootPath, job) + } catch { + // Hive is using Java Reflection to call `listStatus`. Because `listStatus` doesn't declare + // `MetaException`, the Reflection API would throw `UndeclaredThrowableException` without an + // error message if `MetaException` was thrown directly. To improve the user experience, we + // wrap `MetaException` with `IOException` which will provide a better error message. + case e: MetaException => throw new IOException(e) + } + fileToPartition = partitions.filter(_._2.nonEmpty) + files + } + + override def makeSplit( + file: Path, + start: Long, + length: Long, + hosts: Array[String]): FileSplit = { + new DeltaInputSplit(file, start, length, hosts, fileToPartition.getOrElse(file.toUri, Array.empty)) + } + + override def makeSplit( + file: Path, + start: Long, + length: Long, + hosts: Array[String], + inMemoryHosts: Array[String]): FileSplit = { + new DeltaInputSplit(file, start, length, hosts, inMemoryHosts, fileToPartition.getOrElse(file.toUri, Array.empty)) + } + + override def getSplits(job: JobConf, numSplits: Int): Array[InputSplit] = { + val splits = super.getSplits(job, numSplits) + // Reset the temp [[Map]] to release the memory + fileToPartition = Map.empty + splits } } diff --git a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala index 4d946cbbd35..90783d80702 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala @@ -1,8 +1,5 @@ package io.delta.hive -import com.google.common.base.Joiner -import scala.collection.JavaConverters._ - import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory @@ -10,72 +7,67 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.io.ArrayWritable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.FileSplit -import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.Reporter import org.apache.parquet.hadoop.ParquetInputFormat -import org.apache.spark.sql.delta.DeltaHelper import org.slf4j.LoggerFactory -class DeltaRecordReaderWrapper(newInputFormat: ParquetInputFormat[ArrayWritable], oldSplit: InputSplit, oldJobConf: JobConf, reporter: Reporter) extends ParquetRecordReaderWrapper(newInputFormat, oldSplit, oldJobConf, reporter) { +/** + * A record reader that reads data from the underlying Parquet reader and inserts partition values + * which don't exist in the Parquet files. + * + * As we have verified the Hive schema in metastore is consistent with the Delta schema, the row + * returned by the underlying Parquet reader will match the Delta schema except that it leaves all + * partition columns as `null` since they are not in the raw parquet files. Hence, for the missing + * partition values, we need to use the partition information in [[DeltaInputSplit]] to create the + * corresponding [[Writable]]s, and insert them into the corresponding positions when reading a row. + */ +class DeltaRecordReaderWrapper( + inputFormat: ParquetInputFormat[ArrayWritable], + split: DeltaInputSplit, + jobConf: JobConf, + reporter: Reporter) extends ParquetRecordReaderWrapper(inputFormat, split, jobConf, reporter) { private val LOG = LoggerFactory.getLogger(classOf[DeltaRecordReaderWrapper]) - private val partitionWritable: Array[Writable] = - if (!oldSplit.isInstanceOf[FileSplit]) { - throw new IllegalArgumentException("Unknown split type: " + oldSplit) - } else { - val columnNameProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES) - val columnTypeProperty = oldJobConf.get(DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES) - LOG.info("Delta partition cols: " + columnNameProperty + " with types: " + columnTypeProperty) - - if (columnNameProperty == null || columnNameProperty.trim().length() == 0 - || columnTypeProperty == null || columnTypeProperty.trim().length() == 0) { - LOG.info("No partition info is provided...") - null - } else { - // generate partition writale values which will be appended after data values from parquet - val columnNames = columnNameProperty.split(",") - val columnTypes = columnTypeProperty.split(":") - - val filePath = oldSplit.asInstanceOf[FileSplit].getPath() - val parsedPartitions = DeltaHelper.parsePathPartition(filePath, columnNames).asJava - - val partitionWritable = new Array[Writable](columnNames.length) - // inspect partition values - for (i <- 0 until columnNames.length) { - val oi = PrimitiveObjectInspectorFactory - .getPrimitiveWritableObjectInspector(TypeInfoFactory - .getPrimitiveTypeInfo(columnTypes(i))) - - partitionWritable(i) = ObjectInspectorConverters.getConverter( - PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi).convert(parsedPartitions.get(columnNames(i))).asInstanceOf[Writable] - } - LOG.info("Parsed partition values from " + filePath.toString() + " list: " + Joiner.on(",").withKeyValueSeparator("=").join(parsedPartitions) - + ", partitionWritable length:" + partitionWritable.length) - partitionWritable - } + /** The positions of partition columns in Delta schema and their corresponding values. */ + private val partitionValues: Array[(Int, Writable)] = + split.getPartitionColumns.map { partition => + val oi = PrimitiveObjectInspectorFactory + .getPrimitiveWritableObjectInspector(TypeInfoFactory + .getPrimitiveTypeInfo(partition.tpe)) + val partitionValue = ObjectInspectorConverters.getConverter( + PrimitiveObjectInspectorFactory.javaStringObjectInspector, + oi).convert(partition.value).asInstanceOf[Writable] + (partition.index, partitionValue) } override def next(key: NullWritable, value: ArrayWritable): Boolean = { val hasNext = super.next(key, value) - if (partitionWritable != null && partitionWritable.length != 0) { - // append partition values to data values - for (i <- 0 until partitionWritable.length) { - value.get()(value.get().length - partitionWritable.length + i) = partitionWritable(i) - } + // TODO Figure out when the parent reader resets partition columns to null so that we may come + // out a better solution to not insert partition values for each row. + if (hasNext) { + insertPartitionValues(value) } hasNext } - override def createValue(): ArrayWritable = { - val value = super.createValue() - if (partitionWritable != null && partitionWritable.length != 0) { - for (i <- 0 until partitionWritable.length) { - value.get()(value.get().length - partitionWritable.length + i) = partitionWritable(i) - } + /** + * As partition columns are not in the parquet files, they will be set to `null`s every time + * `next` is called. We should insert partition values manually for each row. + */ + private def insertPartitionValues(value: ArrayWritable): Unit = { + val valueArray = value.get() + var i = 0 + val n = partitionValues.length + // Using while loop for better performance since this method is called for each row. + while (i < n) { + val partition = partitionValues(i) + // The schema of `valueArray` is the Hive schema, and it's the same as the Delta + // schema since we have verified it in `DeltaInputFormat`. Hence, the position of a partition + // column in `valueArray` is the same as its position in Delta schema. + valueArray(partition._1) = partition._2 + i += 1 } - value } } diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 754307330df..d991dc5af68 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -18,10 +18,9 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory +import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfo, TypeInfoFactory, TypeInfoUtils} import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} -import org.apache.spark.sql.delta.DeltaHelper -import org.apache.spark.sql.delta.DeltaPushFilter +import org.apache.spark.sql.delta.{DeltaHelper, DeltaPushFilter} import org.slf4j.LoggerFactory class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with HiveStoragePredicateHandler { @@ -44,8 +43,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H override def configureInputJobProperties(tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { super.configureInputJobProperties(tableDesc, jobProperties) jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)) - jobProperties.put(DELTA_PARTITION_COLS_NAMES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_NAMES)) - jobProperties.put(DELTA_PARTITION_COLS_TYPES, tableDesc.getProperties().getProperty(DELTA_PARTITION_COLS_TYPES)) + jobProperties.put(DELTA_TABLE_SCHEMA, tableDesc.getProperties().getProperty(DELTA_TABLE_SCHEMA)) } override def decomposePredicate(jobConf: JobConf, deserializer: Deserializer, predicate: ExprNodeDesc): DecomposedPredicate = { @@ -141,48 +139,68 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H s"${tbl.getDbName}:${tbl.getTableName}. The partition columns in a Delta table " + s"will be read from its own metadata and should not be set manually.") } - val deltaRootString = tbl.getSd().getLocation() - if (deltaRootString == null || deltaRootString.trim().length() == 0) { - throw new MetaException("table location should be set when creating table") - } else { - val deltaPath = new Path(deltaRootString) - val fs = deltaPath.getFileSystem(getConf()) - if (!fs.exists(deltaPath)) { - throw new MetaException("delta.table.path(" + deltaRootString + ") does not exist...") - } else { - val partitionProps = DeltaHelper.checkHiveColsInDelta(deltaPath, tbl.getSd().getCols()) - tbl.getSd().getSerdeInfo().getParameters().putAll(partitionProps.asJava) - tbl.getSd().getSerdeInfo().getParameters().put(DELTA_TABLE_PATH, deltaRootString) - tbl.getParameters.put("spark.sql.sources.provider", "DELTA") - LOG.info("write partition cols/types to table properties " + - partitionProps.map(kv => s"${kv._1}=${kv._2}").mkString(", ")) - } + val deltaRootString = tbl.getSd.getLocation + if (deltaRootString == null || deltaRootString.trim.isEmpty) { + throw new MetaException("table location should be set when creating a Delta table") + } + + val snapshot = DeltaHelper.loadDeltaLatestSnapshot(new Path(deltaRootString)) + if (snapshot.version < 0) { + throw new MetaException(s"$deltaRootString does not exist or it's not a Delta table") + } + + // Extract the table schema in Hive and put it into the table property. Then we can compare it + // with the latest table schema in Delta logs and fail the query if it was changed. + // TODO Investigate if we can get the table schema without manually storing it in the table + // property. + val cols = tbl.getSd.getCols + val columnNames = new java.util.ArrayList[String](cols.size) + val columnTypes = new java.util.ArrayList[TypeInfo](cols.size) + cols.asScala.foreach { col => + columnNames.add(col.getName) + columnTypes.add(TypeInfoUtils.getTypeInfoFromTypeString(col.getType)) } + val hiveSchema = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes) + .asInstanceOf[StructTypeInfo] + DeltaHelper.checkTableSchema(snapshot.metadata.schema, hiveSchema) + tbl.getParameters.put(DELTA_TABLE_PATH, deltaRootString) + tbl.getParameters.put(DELTA_TABLE_SCHEMA, hiveSchema.toString) + tbl.getParameters.put("spark.sql.sources.provider", "DELTA") } override def rollbackCreateTable(table: Table): Unit = { - // TODO What should we do? + // We don't change the Delta table on the file system. Nothing to do } override def commitCreateTable(table: Table): Unit = { - // TODO What should we do? + // Nothing to do } override def preDropTable(table: Table): Unit = { - // TODO What should we do? + // Nothing to do } override def rollbackDropTable(table: Table): Unit = { - // TODO What should we do? + // Nothing to do } override def commitDropTable(table: Table, b: Boolean): Unit = { - // TODO What should we do? + // Nothing to do } } object DeltaStorageHandler { + /** + * The Delta table path we store in the table properties and it's also passed into `JobConf` so + * that `DeltaLog` can be accessed everywhere. + */ val DELTA_TABLE_PATH = "delta.table.path" - val DELTA_PARTITION_COLS_NAMES = "delta.partition.columns" - val DELTA_PARTITION_COLS_TYPES = "delta.partition.columns.types" + + /** + * A config we use to remember the table schema in the table properties. + * + * TODO Maybe Hive can tell us this in the `configureInputJobProperties` method. Then we don't + * need to store this extra information. + */ + val DELTA_TABLE_SCHEMA = "delta.table.schema" } diff --git a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala new file mode 100644 index 00000000000..325421f8ea9 --- /dev/null +++ b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala @@ -0,0 +1,33 @@ +package io.delta.hive + +import java.io.{DataInput, DataOutput} + +import org.apache.hadoop.io.Writable + +/** + * @param index the index of a partition column in the schema. + * @param tpe the Hive type of a partition column. + * @param value the string value of a partition column. The actual partition value should be + * parsed according to its type. + */ +case class PartitionColumnInfo( + var index: Int, + var tpe: String, + var value: String) extends Writable { + + def this() { + this(0, null, null) + } + + override def write(out: DataOutput): Unit = { + out.writeInt(index) + out.writeUTF(tpe) + out.writeUTF(value) + } + + override def readFields(in: DataInput): Unit = { + index = in.readInt() + tpe = in.readUTF() + value = in.readUTF() + } +} diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 28063782ce8..cf6ac51e971 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -2,22 +2,22 @@ package org.apache.spark.sql.delta import java.net.URI -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import io.delta.hive.DeltaStorageHandler +import scala.collection.mutable +import io.delta.hive.{DeltaStorageHandler, PartitionColumnInfo} import org.apache.hadoop.fs._ -import org.apache.hadoop.hive.metastore.api.{FieldSchema, MetaException} +import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.ql.plan.TableScanDesc +import org.apache.hadoop.hive.serde2.typeinfo._ import org.apache.hadoop.mapred.JobConf import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.actions.{AddFile, SingleAction} +import org.apache.spark.sql.types._ object DeltaHelper extends Logging { def parsePathPartition(path: Path, partitionCols: Seq[String]): Map[String, String] = { - val columns = ArrayBuffer.empty[(String, String)] + val columns = mutable.ArrayBuffer.empty[(String, String)] // Old Hadoop versions don't have `Path.isRoot` var finished = path.getParent == null // currentPath is the current path that we will use to parse partition column value. @@ -67,7 +67,16 @@ object DeltaHelper extends Logging { } } - def listDeltaFiles(nonNormalizedPath: Path, job: JobConf): Array[FileStatus] = { + /** + * List the file paths in the Delta table. The provided `JobConf` may consider pushed partition + * filters to do the partition pruning. + * + * The return value has two parts: all of the files matching the pushed partition filter and the + * mapping from file path to its partition information. + */ + def listDeltaFiles( + nonNormalizedPath: Path, + job: JobConf): (Array[FileStatus], Map[URI, Array[PartitionColumnInfo]]) = { val fs = nonNormalizedPath.getFileSystem(job) // We need to normalize the table path so that all paths we return to Hive will be normalized // This is necessary because `HiveInputFormat.pushProjectionsAndFilters` will try to figure out @@ -76,30 +85,62 @@ object DeltaHelper extends Logging { // TODO The assumption about Path in Hive is too strong, we should try to see if we can fail if // `pushProjectionsAndFilters` doesn't find a table for a Delta split path. val rootPath = fs.makeQualified(nonNormalizedPath) - val deltaLog = DeltaLog.forTable(spark, rootPath) - // get the snapshot of the version - val snapshotToUse = deltaLog.snapshot + val snapshotToUse = loadDeltaLatestSnapshot(rootPath) - // TODO Verify the table schema is consistent with `snapshotToUse.metadata`. + val hiveSchema = TypeInfoUtils.getTypeInfoFromTypeString( + job.get(DeltaStorageHandler.DELTA_TABLE_SCHEMA)).asInstanceOf[StructTypeInfo] + DeltaHelper.checkTableSchema(snapshotToUse.metadata.schema, hiveSchema) // get the partition prune exprs val filterExprSerialized = job.get(TableScanDesc.FILTER_EXPR_CONF_STR) val convertedFilterExpr = DeltaPushFilter.partitionFilterConverter(filterExprSerialized) + if (convertedFilterExpr.forall { filter => + DeltaTableUtils.isPredicatePartitionColumnsOnly( + filter, + snapshotToUse.metadata.partitionColumns, + spark) + }) { + // All of filters are based on partition columns. The partition columns may be changed because + // we cannot guarantee that we were using the same Delta Snapshot to generate the filters. But + // as long as the pushed filters are based on a subset of latest partition columns, it should + // be *correct*, even if we may not push all of partition filters and the query may be a bit + // slower. + } else { + throw new MetaException(s"The pushed filters $filterExprSerialized are not all based on" + + "partition columns. This may happen when the partition columns of a Delta table have " + + "been changed when running this query. Please try to re-run the query to pick up the " + + "latest partition columns.") + } // The default value 128M is the same as the default value of // "spark.sql.files.maxPartitionBytes" in Spark. It's also the default parquet row group size // which is usually the best split size for parquet files. val blockSize = job.getLong("parquet.block.size", 128L * 1024 * 1024) + val localFileToPartition = mutable.Map[URI, Array[PartitionColumnInfo]]() + + val partitionColumns = snapshotToUse.metadata.partitionColumns.toSet + val partitionColumnWithIndex = snapshotToUse.metadata.schema.zipWithIndex + .filter { case (t, _) => + partitionColumns.contains(t.name) + }.sortBy(_._2).toArray + // selected files to Hive to be processed - DeltaLog.filterFileList( + val files = DeltaLog.filterFileList( snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), convertedFilterExpr) .as[AddFile](SingleAction.addFileEncoder) .collect().map { f => logInfo(s"selected delta file ${f.path} under $rootPath") - toFileStatus(fs, rootPath, f, blockSize) + val status = toFileStatus(fs, rootPath, f, blockSize) + localFileToPartition += status.getPath.toUri -> partitionColumnWithIndex.map { case (t, index) => + // TODO Is `catalogString` always correct? We may need to add our own conversion rather + // than relying on Spark. + new PartitionColumnInfo(index, t.dataType.catalogString, f.partitionValues(t.name)) + } + status } + (files, localFileToPartition.toMap) } /** @@ -150,35 +191,102 @@ object DeltaHelper extends Logging { } } - def checkHiveColsInDelta( - rootPath: Path, - hiveSchema: java.util.List[FieldSchema]): Map[String, String] = { - val deltaMeta = DeltaLog.forTable(spark, rootPath).snapshot.metadata - assert(hiveSchema.size() == deltaMeta.schema.size, - s"Hive cols(${hiveSchema.asScala.map(_.getName).mkString(",")}) number does not match " + - s"Delta cols(${deltaMeta.schema.map(_.name).mkString(",")})") + def getPartitionCols(rootPath: Path): Seq[String] = { + loadDeltaLatestSnapshot(rootPath).metadata.partitionColumns + } - assert(hiveSchema.asScala.forall(h => deltaMeta.schema.exists(_.name == h.getName)), - s"Hive cols(${hiveSchema.asScala.map(_.getName).mkString(",")}) name does not match " + - s"Delta cols(${deltaMeta.schema.map(_.name).mkString(",")})") + /** Load the latest Delta [[Snapshot]] from the path. */ + def loadDeltaLatestSnapshot(rootPath: Path): Snapshot = { + DeltaLog.forTable(spark, rootPath).update() + } - val (ds, ps) = hiveSchema.asScala.splitAt(hiveSchema.size() - deltaMeta.partitionColumns.size) + /** + * Verify the underlying Delta table schema is the same as the Hive schema defined in metastore. + */ + @throws(classOf[MetaException]) + def checkTableSchema(deltaSchema: StructType, hiveSchema: StructTypeInfo): Unit = { + // TODO How to check column nullables? + if (!isSameStructType(deltaSchema, hiveSchema)) { + throw metaInconsistencyException(deltaSchema, hiveSchema) + } + } - if (ds.forall(s => deltaMeta.dataSchema.exists(_.name == s.getName)) - && ps.forall(s => deltaMeta.partitionColumns.contains(s.getName))) { - Map(DeltaStorageHandler.DELTA_PARTITION_COLS_NAMES -> ps.map(_.getName).mkString(","), - DeltaStorageHandler.DELTA_PARTITION_COLS_TYPES -> ps.map(_.getType).mkString(":")) + private def isSameStructType(sparkStruct: StructType, hiveStruct: StructTypeInfo): Boolean = { + if (sparkStruct.size == hiveStruct.getAllStructFieldNames.size) { + (0 until sparkStruct.size).forall { i => + val sparkField = sparkStruct(i) + val hiveFieldName = hiveStruct.getAllStructFieldNames.get(i) + val hiveFieldType = hiveStruct.getAllStructFieldTypeInfos.get(i) + // TODO Do we need to respect case insensitive config? + sparkField.name == hiveFieldName && isSameType(sparkField.dataType, hiveFieldType) + } } else { - throw new MetaException(s"The partition cols of Delta should be after data cols " + - s"when creating hive table. Delta dataschema is " + - s"${deltaMeta.dataSchema.json} and partitionschema is ${deltaMeta.partitionSchema.json}") + false } } - def getPartitionCols(rootPath: Path): Seq[String] = { - DeltaLog.forTable(spark, rootPath).snapshot.metadata.partitionColumns + private def isSameType(sparkType: DataType, hiveType: TypeInfo): Boolean = { + sparkType match { + case ByteType => hiveType == TypeInfoFactory.byteTypeInfo + case BinaryType => hiveType == TypeInfoFactory.binaryTypeInfo + case BooleanType => hiveType == TypeInfoFactory.booleanTypeInfo + case IntegerType => hiveType == TypeInfoFactory.intTypeInfo + case LongType => hiveType == TypeInfoFactory.longTypeInfo + case StringType => hiveType == TypeInfoFactory.stringTypeInfo + case FloatType => hiveType == TypeInfoFactory.floatTypeInfo + case DoubleType => hiveType == TypeInfoFactory.doubleTypeInfo + case ShortType => hiveType == TypeInfoFactory.shortTypeInfo + case DateType => hiveType == TypeInfoFactory.dateTypeInfo + case TimestampType => hiveType == TypeInfoFactory.timestampTypeInfo + case decimalType: DecimalType => + hiveType match { + case hiveDecimalType: DecimalTypeInfo => + decimalType.precision == hiveDecimalType.precision() && + decimalType.scale == hiveDecimalType.scale() + case _ => false + } + case arrayType: ArrayType => + hiveType match { + case hiveListType: ListTypeInfo => + isSameType(arrayType.elementType, hiveListType.getListElementTypeInfo) + case _ => false + } + case mapType: MapType => + hiveType match { + case hiveMapType: MapTypeInfo => + isSameType(mapType.keyType, hiveMapType.getMapKeyTypeInfo) && + isSameType(mapType.valueType, hiveMapType.getMapValueTypeInfo) + case _ => false + } + case structType: StructType => + hiveType match { + case hiveStructType: StructTypeInfo => isSameStructType(structType, hiveStructType) + case _ => false + } + case _ => + // TODO More Hive types: + // - void + // - char + // - varchar + // - intervalYearMonthType + // - intervalDayTimeType + // - UnionType + // - Others? + throw new UnsupportedOperationException(s"Spark type $sparkType is not supported Hive") + } + } + + private def metaInconsistencyException( + deltaSchema: StructType, + hiveSchema: StructTypeInfo): MetaException = { + new MetaException( + s"""The Delta table schema is not the same as the Hive schema. Please update your Hive + |table's schema to match the Delta table schema. + |Delta table schema: $deltaSchema + |Hive schema: $hiveSchema""".stripMargin) } + // TODO Configure `spark` to pick up the right Hadoop configuration. def spark: SparkSession = SparkSession.builder() .master("local[*]") .appName("HiveOnDelta Get Files") diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index 3897346e712..99e0b0ef38a 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -39,8 +39,8 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { |stored by 'io.delta.hive.DeltaStorageHandler' """.stripMargin ) - }.getMessage - assert(e.contains("table location should be set when creating table")) + } + assert(e.getMessage.contains("table location should be set")) } } @@ -82,50 +82,37 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } } - test("the delta root path should be existed when create hive table") { + test("the table path should point to a Delta table") { withTable("deltaTbl") { withTempDir { dir => - JavaUtils.deleteRecursively(dir) - - val e = intercept[Exception] { + // path exists but is not a Delta table should fail + assert(dir.exists()) + var e = intercept[Exception] { runQuery( s""" |create external table deltaTbl(a string, b int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) - }.getMessage - assert(e.contains(s"delta.table.path(${dir.getCanonicalPath}) does not exist...")) - } - } - } - - test("when creating hive table on a partitioned delta, " + - "the partition columns should be after data columns") { - withTable("deltaTbl") { - withTempDir { dir => - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("a", "b").write.format("delta") - .partitionBy("b").save(dir.getCanonicalPath) } + assert(e.getMessage.contains("not a Delta table")) - val e = intercept[Exception] { + // path doesn't exist should fail as well + JavaUtils.deleteRecursively(dir) + assert(!dir.exists()) + e = intercept[Exception] { runQuery( s""" - |create external table deltaTbl(b string, a string) + |create external table deltaTbl(a string, b int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) - }.getMessage - assert(e.contains(s"The partition cols of Delta should be after data cols")) + } + assert(e.getMessage.contains("does not exist")) } } } - // check column number & column name test("Hive schema should match delta's schema") { withTable("deltaTbl") { withTempDir { dir => @@ -133,32 +120,96 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { withSparkSession { spark => import spark.implicits._ - val x = testData.toDS.toDF("a", "b", "c") testData.toDS.toDF("a", "b", "c").write.format("delta") .partitionBy("b").save(dir.getCanonicalPath) } // column number mismatch - val e1 = intercept[Exception] { + var e = intercept[Exception] { runQuery( s""" |create external table deltaTbl(a string, b string) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) - }.getMessage - assert(e1.contains(s"number does not match")) + } + assert(e.getMessage.contains(s"schema is not the same")) // column name mismatch - val e2 = intercept[Exception] { + e = intercept[Exception] { runQuery( s""" - |create external table deltaTbl(e string, c string, b string) + |create external table deltaTbl(e int, c string, b string) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) - }.getMessage - assert(e2.contains(s"name does not match")) + } + assert(e.getMessage.contains(s"schema is not the same")) + + // column order mismatch + e = intercept[Exception] { + runQuery( + s""" + |create external table deltaTbl(a int, c string, b string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + } + assert(e.getMessage.contains(s"schema is not the same")) + } + } + } + + test("detect schema changes outside Hive") { + withTable("deltaTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession { spark => + import spark.implicits._ + testData.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) + |STORED BY 'io.delta.hive.DeltaStorageHandler' + |LOCATION '${dir.getCanonicalPath}'""".stripMargin + ) + + checkAnswer("SELECT * FROM deltaTbl", testData) + + // Change the underlying Delta table to a different schema + val testData2 = testData.map(_.swap) + + withSparkSession { spark => + import spark.implicits._ + testData2.toDF("a", "b") + .write + .format("delta") + .mode("overwrite") + .option("overwriteSchema", "true") + .save(dir.getCanonicalPath) + } + + // Should detect the underlying schema change and fail the query + val e = intercept[Exception] { + runQuery("SELECT * FROM deltaTbl") + } + assert(e.getMessage.contains(s"schema is not the same")) + + // Re-create the table because Hive doesn't allow `ALTER TABLE` on a non-native table. + // TODO Investigate whether there is a more convenient way to update the table schema. + runQuery("DROP TABLE deltaTbl") + runQuery( + s""" + |CREATE EXTERNAL TABLE deltaTbl(a STRING, b INT) + |STORED BY 'io.delta.hive.DeltaStorageHandler' + |LOCATION '${dir.getCanonicalPath}'""".stripMargin + ) + + // After fixing the schema, the query should work again. + checkAnswer("SELECT * FROM deltaTbl", testData2) } } } @@ -181,9 +232,7 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { """.stripMargin ) - assert(runQuery( - "select * from deltaNonPartitionTbl").sorted === - testData.map(r => s"${r._1}\t${r._2}").sorted) + checkAnswer("select * from deltaNonPartitionTbl", testData) } } } @@ -207,18 +256,14 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { """.stripMargin ) - assert(runQuery( - "select * from deltaPartitionTbl").sorted === - testData.map(r => s"${r._1}\t${r._2}").sorted) + checkAnswer("select * from deltaPartitionTbl", testData) // select partition column order change - assert(runQuery( - "select c2, c1 from deltaPartitionTbl").sorted === - testData.map(r => s"${r._2}\t${r._1}").sorted) + checkAnswer("select c2, c1 from deltaPartitionTbl", testData.map(_.swap)) - assert(runQuery( - "select c2, c1, c2 as c3 from deltaPartitionTbl").sorted === - testData.map(r => s"${r._2}\t${r._1}\t${r._2}").sorted) + checkAnswer( + "select c2, c1, c2 as c3 from deltaPartitionTbl", + testData.map(r => (r._2, r._1, r._2))) } } } @@ -245,9 +290,9 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { // Delete the partition not needed in the below query to verify the partition pruning works JavaUtils.deleteRecursively(new File(dir, "c2=foo1")) assert(dir.listFiles.map(_.getName).sorted === Seq("_delta_log", "c2=foo0").sorted) - assert(runQuery( - "select * from deltaPartitionTbl where c2 = 'foo0'").sorted === - testData.filter(_._2 == "foo0").map(r => s"${r._1}\t${r._2}").sorted) + checkAnswer( + "select * from deltaPartitionTbl where c2 = 'foo0'", + testData.filter(_._2 == "foo0")) } } } @@ -271,7 +316,7 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { runQuery( s""" - |create external table deltaPartitionTbl(name string, cnt int, city string, `date` string) + |create external table deltaPartitionTbl(city string, `date` string, name string, cnt int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) @@ -280,87 +325,76 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'") .mkString(" ").contains("filterExpr: (date = '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'") - .toList.sorted === testData.filter(_._2 == "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'", + testData.filter(_._2 == "20180520")) assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'") .mkString(" ").contains("filterExpr: (date <> '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'") - .toList.sorted === testData.filter(_._2 != "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'", + testData.filter(_._2 != "20180520")) assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'") .mkString(" ").contains("filterExpr: (date > '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'") - .toList.sorted === testData.filter(_._2 > "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'", + testData.filter(_._2 > "20180520")) assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'") .mkString(" ").contains("filterExpr: (date >= '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'") - .toList.sorted === testData.filter(_._2 >= "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'", + testData.filter(_._2 >= "20180520")) assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'") .mkString(" ").contains("filterExpr: (date < '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'") - .toList.sorted === testData.filter(_._2 < "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'", + testData.filter(_._2 < "20180520")) assert(runQuery( "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'") .mkString(" ").contains("filterExpr: (date <= '20180520')")) - assert(runQuery( - "select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'") - .toList.sorted === testData.filter(_._2 <= "20180520") - .map(r => s"${r._1}\t${r._2}\t${r._3}\t${r._4}").sorted) + checkAnswer( + "select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'", + testData.filter(_._2 <= "20180520")) // expr(like) pushed down assert(runQuery( "explain select * from deltaPartitionTbl where `date` like '201805%'") .mkString(" ").contains("filterExpr: (date like '201805%')")) - assert(runQuery( - "select * from deltaPartitionTbl where `date` like '201805%'").toList.sorted === testData - .filter(_._2.contains("201805")).map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer( + "select * from deltaPartitionTbl where `date` like '201805%'", + testData.filter(_._2.contains("201805"))) // expr(in) pushed down assert(runQuery( "explain select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')") .mkString(" ").contains("filterExpr: (city) IN ('hz', 'sz')")) - assert(runQuery( - "select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')") - .toList.sorted === testData.filter(c => Seq("hz", "sz").contains(c._1)) - .map(r => s"${r._3}\t${r._2}\t${r._4}").sorted) + checkAnswer( + "select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')", + testData.filter(c => Seq("hz", "sz").contains(c._1)).map(r => (r._3, r._2, r._4))) // two partition column pushed down assert(runQuery( "explain select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')") .mkString(" ").contains("filterExpr: ((city) IN ('hz', 'sz') and (date = '20181212'))")) - assert(runQuery( - "select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')") - .toList.sorted === testData - .filter(c => Seq("hz", "sz").contains(c._1) && c._2 == "20181212") - .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer( + "select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')", + testData.filter(c => Seq("hz", "sz").contains(c._1) && c._2 == "20181212")) // data column not be pushed down assert(runQuery( "explain select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'") .mkString(" ").contains("filterExpr: (city = 'hz'")) - assert(runQuery( - "select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'") - .toList.sorted === testData - .filter(c => c._1 == "hz" && c._3 == "Jim") - .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer( + "select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'", + testData.filter(c => c._1 == "hz" && c._3 == "Jim")) } } } @@ -380,30 +414,24 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { runQuery( s""" - |create external table deltaPartitionTbl(name string, cnt int, city string, `date` string) + |create external table deltaPartitionTbl(city string, `date` string, name string, cnt int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) - assert(runQuery( - "select * from deltaPartitionTbl").toList.sorted === testData1 - .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer("select * from deltaPartitionTbl", testData1) // insert another partition data val testData2 = Seq(("bj", "20180520", "Trump", 1)) testData2.toDS.toDF("city", "date", "name", "cnt").write.mode("append").format("delta") .partitionBy("date", "city").save(dir.getCanonicalPath) val testData = testData1 ++ testData2 - assert(runQuery( - "select * from deltaPartitionTbl").toList.sorted === testData - .map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer("select * from deltaPartitionTbl", testData) // delete one partition val deltaTable = DeltaTable.forPath(spark, dir.getCanonicalPath) deltaTable.delete("city='hz'") - assert(runQuery( - "select * from deltaPartitionTbl").toList.sorted === testData - .filterNot(_._1 == "hz").map(r => s"${r._3}\t${r._4}\t${r._1}\t${r._2}").sorted) + checkAnswer("select * from deltaPartitionTbl", testData.filterNot(_._1 == "hz")) } } } @@ -427,10 +455,89 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { """.stripMargin ) - // TODO Read partition values from `AddFile.partitionValues` to fix incorrect escaped - // partition values. - runQuery("select * from deltaPartitionTbl") + checkAnswer("select * from deltaPartitionTbl", testData) + } + } + } + + test("map Spark types to Hive types correctly") { + withTable("deltaTbl") { + withTempDir { dir => + val testData = Seq( + TestClass( + 97.toByte, + Array(98.toByte, 99.toByte), + true, + 4, + 5L, + "foo", + 6.0f, + 7.0, + 8.toShort, + new java.sql.Date(60000000L), + new java.sql.Timestamp(60000000L), + new java.math.BigDecimal(12345.6789), + Array("foo", "bar"), + Map("foo" -> 123L), + TestStruct("foo", 456L) + ) + ) + + withSparkSession { spark => + import spark.implicits._ + testData.toDF.write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaTbl( + |c1 tinyint, c2 binary, c3 boolean, c4 int, c5 bigint, c6 string, c7 float, c8 double, + |c9 smallint, c10 date, c11 timestamp, c12 decimal(38, 18), c13 array, + |c14 map, c15 struct) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + val expected = ( + "97", + "bc", + "true", + "4", + "5", + "foo", + "6.0", + "7.0", + "8", + "1970-01-01", + "1970-01-01 08:40:00", + "12345.678900000000794535", + """["foo","bar"]""", + """{"foo":123}""", + """{"f1":"foo","f2":456}""" + ) + checkAnswer("select * from deltaTbl", Seq(expected)) } } } } + +case class TestStruct(f1: String, f2: Long) + +/** A special test class that covers all Spark types we support in the Hive connector. */ +case class TestClass( + c1: Byte, + c2: Array[Byte], + c3: Boolean, + c4: Int, + c5: Long, + c6: String, + c7: Float, + c8: Double, + c9: Short, + c10: java.sql.Date, + c11: java.sql.Timestamp, + c12: BigDecimal, + c13: Array[String], + c14: Map[String, Long], + c15: TestStruct +) diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index b96e8a8096f..894f91c053c 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -2,6 +2,7 @@ package io.delta.hive.test import java.io.File import java.nio.file.Files +import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ @@ -27,6 +28,11 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { private var driver: Driver = _ private var mr: MiniMRCluster = _ + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + override def beforeAll(): Unit = { super.beforeAll() val warehouseDir = new File(tempPath, "warehouse") @@ -88,6 +94,29 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { } } + /** Run the Hive query and check the result with the expected answer. */ + def checkAnswer[T <: Product](query: String, expected: Seq[T]): Unit = { + val actualAnswer = runQuery(query).sorted + val expectedAnswer = expected.map(_.productIterator.mkString("\t")).sorted + if (actualAnswer != expectedAnswer) { + fail( + s"""Answers do not match. + |Query: + | + |$query + | + |Expected: + | + |${expectedAnswer.mkString("\n")} + | + |Actual: + | + |${actualAnswer.mkString("\n")} + | + """.stripMargin) + } + } + /** * Drops table `tableName` after calling `f`. */ From 868388a0155deb2db9e79cebeb412821cb216946 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 17 Dec 2019 08:11:55 -0800 Subject: [PATCH 016/291] Add Scala stylecheck (#15) This PR applies Scala stylecheck plugin to enforce the same `scalastyle-config.xml` file from the delta-core project. The major changes in this PR include: - Copy `scalastyle-config.xml` from the delta-core project. - Apply the stylecheck plugin to all projects. - Add the missing license header. - Fix the long lines. - Remove unused methods: `DeltaHelper.parsePathPartition` and `DeltaHepler. parsePartitionColumn`. - Update Delta version to 0.5.0. --- build.sbt | 18 +- coreTest/src/main/scala/Test.scala | 22 +- .../java/io/delta/hive/DeltaInputSplit.java | 16 + .../io/delta/hive/DeltaInputFormat.scala | 39 +- .../io/delta/hive/DeltaOutputFormat.scala | 16 + .../delta/hive/DeltaRecordReaderWrapper.scala | 16 + .../io/delta/hive/DeltaStorageHandler.scala | 53 ++- .../scala/io/delta/hive/HiveInputFormat.scala | 22 +- .../io/delta/hive/PartitionColumnInfo.scala | 16 + .../apache/spark/sql/delta/DeltaHelper.scala | 72 +-- .../spark/sql/delta/DeltaPushFilter.scala | 16 + .../io/delta/hive/HiveConnectorSuite.scala | 40 +- .../scala/io/delta/hive/test/HiveTest.scala | 17 + .../io/delta/hive/test/HiveTestSuite.scala | 16 + scalastyle-config.xml | 418 ++++++++++++++++++ 15 files changed, 715 insertions(+), 82 deletions(-) create mode 100644 scalastyle-config.xml diff --git a/build.sbt b/build.sbt index 94d42058fbb..67db1a9dad9 100644 --- a/build.sbt +++ b/build.sbt @@ -1,4 +1,3 @@ - /* * Copyright 2019 Databricks, Inc. * @@ -16,13 +15,18 @@ */ parallelExecution in ThisBuild := false +scalastyleConfig in ThisBuild := baseDirectory.value / "scalastyle-config.xml" + +lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") +lazy val testScalastyle = taskKey[Unit]("testScalastyle") val sparkVersion = "2.4.3" val hadoopVersion = "2.7.2" val hiveVersion = "2.3.3" +val deltaVersion = "0.5.0" lazy val commonSettings = Seq( - version := "0.4.0", + version := deltaVersion, organization := "io.delta", scalaVersion := "2.12.8", fork := true, @@ -35,14 +39,18 @@ lazy val commonSettings = Seq( "-Ddelta.log.cacheSize=3", "-Dspark.sql.sources.parallelPartitionDiscovery.parallelism=5", "-Xmx1024m" - ) + ), + compileScalastyle := scalastyle.in(Compile).toTask("").value, + (compile in Compile) := ((compile in Compile) dependsOn compileScalastyle).value, + testScalastyle := scalastyle.in(Test).toTask("").value, + (test in Test) := ((test in Test) dependsOn testScalastyle).value ) lazy val core = (project in file("core")) .settings( name := "delta-core-shaded", libraryDependencies ++= Seq( - "io.delta" %% "delta-core" % "0.4.0" excludeAll ExclusionRule("org.apache.hadoop"), + "io.delta" %% "delta-core" % deltaVersion excludeAll ExclusionRule("org.apache.hadoop"), "org.apache.spark" %% "spark-sql" % sparkVersion excludeAll ExclusionRule("org.apache.hadoop"), "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" ), @@ -158,6 +166,6 @@ lazy val hive = (project in file("hive")) settings ( // TODO Figure out how this fixes some bad dependency "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % "0.4.0" % "test" excludeAll ExclusionRule("org.apache.hadoop") + "io.delta" %% "delta-core" % deltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") ) ) diff --git a/coreTest/src/main/scala/Test.scala b/coreTest/src/main/scala/Test.scala index e8fab5c9569..61ea85e33b6 100644 --- a/coreTest/src/main/scala/Test.scala +++ b/coreTest/src/main/scala/Test.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package test import shadedelta.io.delta.tables._ @@ -9,7 +25,7 @@ import org.apache.spark.sql.functions._ import java.io.File object Test { - + def main(args: Array[String]): Unit = { // Create a Spark Session val spark = SparkSession @@ -20,14 +36,18 @@ object Test { // Create a table + // scalastyle:off println println("Creating a table") + // scalastyle:on println val file = new File("/tmp/delta-table") val path = file.getCanonicalPath var data = spark.range(0, 5) data.write.format("delta").mode("overwrite").save(path) // Read table + // scalastyle:off println println("Reading the table") + // scalastyle:on println val df = spark.read.format("delta").load(path) df.show() diff --git a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java index a3abdb89e2b..30ae177a379 100644 --- a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java +++ b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive; import java.io.DataInput; diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index ea0ba4a36c5..cdd777c0593 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import java.io.IOException @@ -37,7 +53,8 @@ import org.slf4j.LoggerFactory * [[Writable]] for all partition values, and insert them to the raw row returned by * [[org.apache.parquet.hadoop.ParquetRecordReader]]. */ -class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends FileInputFormat[NullWritable, ArrayWritable] { +class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) + extends FileInputFormat[NullWritable, ArrayWritable] { private val LOG = LoggerFactory.getLogger(classOf[DeltaInputFormat]) @@ -51,7 +68,10 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends Fil this(new ParquetInputFormat[ArrayWritable](classOf[DataWritableReadSupport])) } - override def getRecordReader(split: InputSplit, job: JobConf, reporter: Reporter): RecordReader[NullWritable, ArrayWritable] = { + override def getRecordReader( + split: InputSplit, + job: JobConf, + reporter: Reporter): RecordReader[NullWritable, ArrayWritable] = { split match { case deltaSplit: DeltaInputSplit => if (Utilities.getUseVectorizedInputFileFormat(job)) { @@ -88,7 +108,12 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends Fil start: Long, length: Long, hosts: Array[String]): FileSplit = { - new DeltaInputSplit(file, start, length, hosts, fileToPartition.getOrElse(file.toUri, Array.empty)) + new DeltaInputSplit( + file, + start, + length, + hosts, + fileToPartition.getOrElse(file.toUri, Array.empty)) } override def makeSplit( @@ -97,7 +122,13 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) extends Fil length: Long, hosts: Array[String], inMemoryHosts: Array[String]): FileSplit = { - new DeltaInputSplit(file, start, length, hosts, inMemoryHosts, fileToPartition.getOrElse(file.toUri, Array.empty)) + new DeltaInputSplit( + file, + start, + length, + hosts, + inMemoryHosts, + fileToPartition.getOrElse(file.toUri, Array.empty)) } override def getSplits(job: JobConf, numSplits: Int): Array[InputSplit] = { diff --git a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala index 30f01814a2a..fa333fbd621 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import org.apache.hadoop.fs.FileSystem diff --git a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala index 90783d80702..8054f909891 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index d991dc5af68..952a124726b 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -1,5 +1,23 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive +import java.util.{ArrayList => JArrayList} + import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.fs.Path @@ -23,7 +41,8 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} import org.apache.spark.sql.delta.{DeltaHelper, DeltaPushFilter} import org.slf4j.LoggerFactory -class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with HiveStoragePredicateHandler { +class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook + with HiveStoragePredicateHandler { import DeltaStorageHandler._ @@ -40,13 +59,18 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H override def getSerDeClass(): Class[_ <: AbstractSerDe] = classOf[ParquetHiveSerDe] - override def configureInputJobProperties(tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { + override def configureInputJobProperties( + tableDesc: TableDesc, + jobProperties: java.util.Map[String, String]): Unit = { super.configureInputJobProperties(tableDesc, jobProperties) jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)) jobProperties.put(DELTA_TABLE_SCHEMA, tableDesc.getProperties().getProperty(DELTA_TABLE_SCHEMA)) } - override def decomposePredicate(jobConf: JobConf, deserializer: Deserializer, predicate: ExprNodeDesc): DecomposedPredicate = { + override def decomposePredicate( + jobConf: JobConf, + deserializer: Deserializer, + predicate: ExprNodeDesc): DecomposedPredicate = { // Get the delta root path val deltaRootPath = jobConf.get(DELTA_TABLE_PATH) // Get the partitionColumns of Delta @@ -57,7 +81,8 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H val conditions = new java.util.ArrayList[IndexSearchCondition]() var pushedPredicate: ExprNodeGenericFuncDesc = null - var residualPredicate = analyzer.analyzePredicate(predicate, conditions).asInstanceOf[ExprNodeGenericFuncDesc] + var residualPredicate = + analyzer.analyzePredicate(predicate, conditions).asInstanceOf[ExprNodeGenericFuncDesc] for (searchConditions <- decompose(conditions).values) { // still push back the pushedPredicate to residualPredicate residualPredicate = @@ -66,8 +91,9 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H extractStorageHandlerCondition(analyzer, searchConditions, pushedPredicate) } - LOG.info("pushedPredicate:" + (if (pushedPredicate == null) "null" else pushedPredicate.getExprString()) - + ",residualPredicate" + residualPredicate) + LOG.info("pushedPredicate:" + + (if (pushedPredicate == null) "null" else pushedPredicate.getExprString()) + + ",residualPredicate" + residualPredicate) val decomposedPredicate = new DecomposedPredicate() decomposedPredicate.pushedPredicate = pushedPredicate @@ -84,10 +110,13 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H analyzer } - private def decompose(searchConditions: java.util.ArrayList[IndexSearchCondition]): Map[String, java.util.ArrayList[IndexSearchCondition]] = { + private def decompose(searchConditions: JArrayList[IndexSearchCondition]): + Map[String, JArrayList[IndexSearchCondition]] = { val result = mutable.Map[String, java.util.ArrayList[IndexSearchCondition]]() for (condition <- searchConditions.asScala) { - val conditions = result.getOrElseUpdate(condition.getColumnDesc().getColumn(), new java.util.ArrayList[IndexSearchCondition]()) + val conditions = result.getOrElseUpdate( + condition.getColumnDesc().getColumn(), + new JArrayList[IndexSearchCondition]()) conditions.add(condition) } result.toMap @@ -100,7 +129,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H if (inputExpr == null) { analyzer.translateOriginalConditions(searchConditions) } else { - val children = new java.util.ArrayList[ExprNodeDesc] + val children = new JArrayList[ExprNodeDesc] children.add(analyzer.translateOriginalConditions(searchConditions)) children.add(inputExpr) new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, @@ -115,7 +144,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H if (inputExpr == null) { analyzer.translateSearchConditions(searchConditions) } else { - val children = new java.util.ArrayList[ExprNodeDesc] + val children = new JArrayList[ExprNodeDesc] children.add(analyzer.translateSearchConditions(searchConditions)) children.add(inputExpr) new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, @@ -154,8 +183,8 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook with H // TODO Investigate if we can get the table schema without manually storing it in the table // property. val cols = tbl.getSd.getCols - val columnNames = new java.util.ArrayList[String](cols.size) - val columnTypes = new java.util.ArrayList[TypeInfo](cols.size) + val columnNames = new JArrayList[String](cols.size) + val columnTypes = new JArrayList[TypeInfo](cols.size) cols.asScala.foreach { col => columnNames.add(col.getName) columnTypes.add(TypeInfoUtils.getTypeInfoFromTypeString(col.getType)) diff --git a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala index 12c728c1569..ddcd1e3eaec 100644 --- a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import org.apache.hadoop.fs.Path @@ -5,7 +21,11 @@ import org.apache.hadoop.mapred.JobConf class HiveInputFormat extends org.apache.hadoop.hive.ql.io.HiveInputFormat { - override def pushProjectionsAndFilters(jobConf: JobConf, inputFormatClass: Class[_], splitPath: Path, nonNative: Boolean) = { + override def pushProjectionsAndFilters( + jobConf: JobConf, + inputFormatClass: Class[_], + splitPath: Path, + nonNative: Boolean): Unit = { if (inputFormatClass == classOf[DeltaInputFormat]) { super.pushProjectionsAndFilters(jobConf, inputFormatClass, splitPath, false) } else { diff --git a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala index 325421f8ea9..8f8c6c293ad 100644 --- a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala +++ b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import java.io.{DataInput, DataOutput} diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index cf6ac51e971..2540ae20d06 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.sql.delta import java.net.URI @@ -16,57 +32,6 @@ import org.apache.spark.sql.types._ object DeltaHelper extends Logging { - def parsePathPartition(path: Path, partitionCols: Seq[String]): Map[String, String] = { - val columns = mutable.ArrayBuffer.empty[(String, String)] - // Old Hadoop versions don't have `Path.isRoot` - var finished = path.getParent == null - // currentPath is the current path that we will use to parse partition column value. - var currentPath: Path = path - - while (!finished) { - // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. - // Once we get the string, we try to parse it and find the partition column and value. - val fragment = currentPath.getName - val maybeColumn = - parsePartitionColumn(currentPath.getName) - - maybeColumn.foreach(columns += _) - - finished = - (maybeColumn.isEmpty && columns.nonEmpty) || currentPath.getParent == null - - if (!finished) { - // For the above example, currentPath will be "/table/". - currentPath = currentPath.getParent - } - } - - assert(columns.map(_._1).zip(partitionCols).forall(c => c._1 == c._2), - s""" - |partitionCols(${columns.map(_._1).mkString(",")}) parsed from $path - |does not match the created table partition(${partitionCols.mkString(",")}) - """.stripMargin) - - columns.toMap - } - - private def parsePartitionColumn(columnSpec: String): Option[(String, String)] = { - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName - - val equalSignIndex = columnSpec.indexOf('=') - if (equalSignIndex == -1) { - None - } else { - val columnName = unescapePathName(columnSpec.take(equalSignIndex)) - assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") - - val rawColumnValue = columnSpec.drop(equalSignIndex + 1) - assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - - Some(columnName, rawColumnValue) - } - } - /** * List the file paths in the Delta table. The provided `JobConf` may consider pushed partition * filters to do the partition pruning. @@ -133,7 +98,8 @@ object DeltaHelper extends Logging { .collect().map { f => logInfo(s"selected delta file ${f.path} under $rootPath") val status = toFileStatus(fs, rootPath, f, blockSize) - localFileToPartition += status.getPath.toUri -> partitionColumnWithIndex.map { case (t, index) => + localFileToPartition += + status.getPath.toUri -> partitionColumnWithIndex.map { case (t, index) => // TODO Is `catalogString` always correct? We may need to add our own conversion rather // than relying on Spark. new PartitionColumnInfo(index, t.dataType.catalogString, f.partitionValues(t.name)) @@ -291,4 +257,4 @@ object DeltaHelper extends Logging { .master("local[*]") .appName("HiveOnDelta Get Files") .getOrCreate() -} \ No newline at end of file +} diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala index f35945f8775..b75c0f7c851 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.sql.delta import scala.collection.immutable.HashSet diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index 99e0b0ef38a..9711b9be619 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive import java.io.File @@ -316,7 +332,11 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { runQuery( s""" - |create external table deltaPartitionTbl(city string, `date` string, name string, cnt int) + |create external table deltaPartitionTbl( + | city string, + | `date` string, + | name string, + | cnt int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) @@ -330,7 +350,8 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { testData.filter(_._2 == "20180520")) assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'") + "explain select city, `date`, name, cnt from deltaPartitionTbl " + + "where `date` != '20180520'") .mkString(" ").contains("filterExpr: (date <> '20180520')")) checkAnswer( "select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'", @@ -344,7 +365,8 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { testData.filter(_._2 > "20180520")) assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'") + "explain select city, `date`, name, cnt from deltaPartitionTbl " + + "where `date` >= '20180520'") .mkString(" ").contains("filterExpr: (date >= '20180520')")) checkAnswer( "select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'", @@ -358,7 +380,8 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { testData.filter(_._2 < "20180520")) assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'") + "explain select city, `date`, name, cnt from deltaPartitionTbl " + + "where `date` <= '20180520'") .mkString(" ").contains("filterExpr: (date <= '20180520')")) checkAnswer( "select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'", @@ -382,7 +405,8 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { // two partition column pushed down assert(runQuery( - "explain select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')") + "explain select * from deltaPartitionTbl " + + "where `date` = '20181212' and `city` in ('hz', 'sz')") .mkString(" ").contains("filterExpr: ((city) IN ('hz', 'sz') and (date = '20181212'))")) checkAnswer( "select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')", @@ -414,7 +438,11 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { runQuery( s""" - |create external table deltaPartitionTbl(city string, `date` string, name string, cnt int) + |create external table deltaPartitionTbl( + | city string, + | `date` string, + | name string, + | cnt int) |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' """.stripMargin ) diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 894f91c053c..6d313005225 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive.test import java.io.File @@ -19,6 +35,7 @@ import org.apache.spark.SparkConf import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog +// scalastyle:off funsuite import org.scalatest.{BeforeAndAfterAll, FunSuite} // TODO Yarn is using log4j2. Disable its verbose logs. diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala b/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala index c454a783e80..f920512a7d6 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive.test import org.scalatest.Ignore diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 00000000000..f88a3472897 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,418 @@ + + + + + + + Scalastyle standard configuration + + + + + + + + + + true + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + + + + ^FunSuite[A-Za-z]*$ + Tests must extend org.apache.spark.SparkFunSuite instead. + + + + + ^println$ + + + + + spark(.sqlContext)?.sparkContext.hadoopConfiguration + + + + + @VisibleForTesting + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + mutable\.SynchronizedBuffer + + + + + Class\.forName + + + + + Await\.result + + + + + Await\.ready + + + + + (\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\))) + + + + + throw new \w+Error\( + + + + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + + + extractOpt + Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter + is slower. + + + + + COMMA + + + + + + \)\{ + + + + + (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] + Use Javadoc style indentation for multiline comments + + + + case[^\n>]*=>\s*\{ + Omit braces in case clauses. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 800> + + + + + 30 + + + + + 10 + + + + + 50 + + + + + + + + + + + -1,0,1,2,3 + + + From f0dc24df5178e773b604811066d98c0306782594 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 10 Jan 2020 13:35:57 -0800 Subject: [PATCH 017/291] Add Hive connector instructions --- README.md | 58 +++++++++++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 54 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index c2f4d6c727e..2cd16d8b8c4 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ This is the repository for Delta Lake Connectors. It includes a library for quer The project is compiled using [SBT](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html). It has the following subprojects. -## 1. Delta uber jar +## Delta uber jar This project generates a single uber jar containing Delta Lake and all it transitive dependencies (except Hadoop and its dependencies). - Most of the dependencies are shaded to avoid version conflicts. See the file build.sbt for details on what are not shaded. @@ -20,15 +20,65 @@ This project generates a single uber jar containing Delta Lake and all it transi - To generate the uber jar, run `build/sbt core/compile` - To test the uber jar, run `build/sbt coreTest/test` -## 2. Hive connector +## Hive connector This project contains all the code needed to make Hive read Delta Lake tables. - To compile the project, run `build/sbt hive/compile` - To test the project, run `build/sbt hive/test` - To generate the connector jar run `build/sbt hive/package` -Config `HIVE_AUX_JARS_PATH` in hive-env.sh with above two jars(uber jar and Hive connector jar) +The above commands will generate two JARs in the following paths. -Refer to [SBT docs](https://www.scala-sbt.org/1.x/docs/Command-Line-Reference.html) for more commands. +``` +core/target/scala-2.12/delta-core-shaded-assembly-0.5.0.jar +hive/target/scala-2.12/hive-delta_2.12-0.5.0.jar +``` + +These two JARs include the Hive connector and all its dependencies. They need to be put in Hive’s classpath. + +### Setting up Hive + +This sections describes how to set up Hive to load the Delta Hive connector. + +Before starting your Hive CLI or running your Hive script, add the following special Hive config to the `hive-site.xml` file (Its location is `/etc/hive/conf/hive-site.xml` in a EMR cluster). + +```xml + + hive.input.format + io.delta.hive.HiveInputFormat + +``` + +The second step is to upload the above two JARs to the machine that runs Hive. Finally, add the paths of the JARs toHive’s environment variable, `HIVE_AUX_JARS_PATH`. You can find this environment variable in the `hive-env.sh` file, whose location is ``/etc/hive/conf/hive-env.sh` on an EMR cluster. This setting will tell Hive where to find the connector JARs. + +### Create a Hive table + +After finishing setup, you should be able to create a Delta table in Hive. + +Right now the connector supports only EXTERNAL Hive tables. The Delta table must be created using Spark before an external Hive table can reference it. + +Here is an example of a CREATE TABLE command that defines an external Hive table pointing to a Delta table on `s3://foo-bucket/bar-dir`. + +```SQL +CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) +STORED BY 'io.delta.hive.DeltaStorageHandler' +LOCATION 's3://foo-bucket/bar-dir’ +``` + +`io.delta.hive.DeltaStorageHandler` is the class that implements Hive data source APIs. It will know how to load a Delta table and extract its metadata. The table schema in the `CREATE TABLE` statement must be consistent with the underlying Delta metadata. Otherwise, the connector will throw an error to tell you about the inconsistency. + +### FAQs + +#### Supported Hive versions +Hive 2.x. Please report any incompatible issues. + +#### Do I need to specify the partition columns when creating a Delta table? +No. The partition columns are read from the underlying Delta metadata. The connector will know the partition columns and use this information to do the partition pruning automatically. + +#### Why do I need to specify the table schema? Shouldn’t it exist in the underlying Delta table metadata? +Unfortunately, the table schema is a core concept of Hive and Hive needs it before calling the connector. + +#### What if I change the underlying Delta table schema in Spark after creating the Hive table? +If the schema in the underlying Delta metadata is not consistent with the schema specified by `CREATE TABLE` statement, the connector will report an error when loading the table and ask you to fix the schema. You must drop the table and recreate it using the new schema. Hive 3.x exposes a new API to allow a data source to hook ALTER TABLE. You will be able to use ALTER TABLE to update a table schema when the connector supports Hive 3.x. # Reporting issues From 119e67e6128b186bdc610bb81634ba4efa3481d0 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 17 Jan 2020 10:36:14 -0800 Subject: [PATCH 018/291] Set 'path' in the storage properties as Spark is using it for its data sources (#16) Spark reads the `path` option from the storage properties. We should set it when creating a table in Hive so that the table can be read by Spark as well. I also removed `delta.table.path` from the table properties since we can get it from the `location` property. --- .../main/scala/io/delta/hive/DeltaStorageHandler.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 952a124726b..e99a06086e9 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaHook import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.metastore.api.Table +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.index.IndexSearchCondition import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe @@ -63,7 +64,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { super.configureInputJobProperties(tableDesc, jobProperties) - jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(DELTA_TABLE_PATH)) + jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(META_TABLE_LOCATION)) jobProperties.put(DELTA_TABLE_SCHEMA, tableDesc.getProperties().getProperty(DELTA_TABLE_SCHEMA)) } @@ -72,7 +73,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook deserializer: Deserializer, predicate: ExprNodeDesc): DecomposedPredicate = { // Get the delta root path - val deltaRootPath = jobConf.get(DELTA_TABLE_PATH) + val deltaRootPath = jobConf.get(META_TABLE_LOCATION) // Get the partitionColumns of Delta val partitionColumns = DeltaHelper.getPartitionCols(new Path(deltaRootPath)) LOG.info("delta partitionColumns is " + partitionColumns.mkString(", ")) @@ -192,9 +193,9 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook val hiveSchema = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes) .asInstanceOf[StructTypeInfo] DeltaHelper.checkTableSchema(snapshot.metadata.schema, hiveSchema) - tbl.getParameters.put(DELTA_TABLE_PATH, deltaRootString) tbl.getParameters.put(DELTA_TABLE_SCHEMA, hiveSchema.toString) tbl.getParameters.put("spark.sql.sources.provider", "DELTA") + tbl.getSd.getParameters.put("path", deltaRootString) } override def rollbackCreateTable(table: Table): Unit = { @@ -220,8 +221,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook object DeltaStorageHandler { /** - * The Delta table path we store in the table properties and it's also passed into `JobConf` so - * that `DeltaLog` can be accessed everywhere. + * The Delta table path passing into `JobConf` so that `DeltaLog` can be accessed everywhere. */ val DELTA_TABLE_PATH = "delta.table.path" From 7929edd2050f0f28f31614d4c599f8cb04f2576d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 25 Feb 2020 22:26:18 -0800 Subject: [PATCH 019/291] Column names should be case insensitive (#20) Hive's column names are case insensitive. We should ignore case when comparing the column names. This PR also improved the error message to make it easier to compare the schema differences. Closes #17 --- .../apache/spark/sql/delta/DeltaHelper.scala | 19 +++++-- .../io/delta/hive/HiveConnectorSuite.scala | 51 +++++++++++++++++++ 2 files changed, 66 insertions(+), 4 deletions(-) diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 2540ae20d06..b113c09c01f 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.delta import java.net.URI import scala.collection.mutable +import scala.collection.JavaConverters._ + import io.delta.hive.{DeltaStorageHandler, PartitionColumnInfo} import org.apache.hadoop.fs._ import org.apache.hadoop.hive.metastore.api.MetaException @@ -183,8 +185,8 @@ object DeltaHelper extends Logging { val sparkField = sparkStruct(i) val hiveFieldName = hiveStruct.getAllStructFieldNames.get(i) val hiveFieldType = hiveStruct.getAllStructFieldTypeInfos.get(i) - // TODO Do we need to respect case insensitive config? - sparkField.name == hiveFieldName && isSameType(sparkField.dataType, hiveFieldType) + sparkField.name.equalsIgnoreCase(hiveFieldName) && + isSameType(sparkField.dataType, hiveFieldType) } } else { false @@ -245,11 +247,20 @@ object DeltaHelper extends Logging { private def metaInconsistencyException( deltaSchema: StructType, hiveSchema: StructTypeInfo): MetaException = { + val hiveSchemaString = hiveSchema.getAllStructFieldNames + .asScala + .zip(hiveSchema.getAllStructFieldTypeInfos.asScala.map(_.getTypeName)) + .map(_.productIterator.mkString(": ")) + .mkString("\n") new MetaException( s"""The Delta table schema is not the same as the Hive schema. Please update your Hive |table's schema to match the Delta table schema. - |Delta table schema: $deltaSchema - |Hive schema: $hiveSchema""".stripMargin) + | + |Delta table schema: + |${deltaSchema.treeString} + | + |Hive schema: + |${hiveSchemaString}""".stripMargin) } // TODO Configure `spark` to pick up the right Hadoop configuration. diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index 9711b9be619..d0bf574ca90 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -547,6 +547,55 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { } } } + + test("column names should be case insensitive") { + // Create a Delta table + withTable("deltaCaseInsensitiveTest") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession{ spark => + import spark.implicits._ + testData.toDS.toDF("FooBar", "BarFoo").write.format("delta") + .partitionBy("BarFoo").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaCaseInsensitiveTest(fooBar int, Barfoo string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + checkAnswer("select * from deltaCaseInsensitiveTest", testData) + for ((col1, col2) <- + Seq("fooBar" -> "barFoo", "foobar" -> "barfoo", "FOOBAR" -> "BARFOO")) { + checkAnswer( + s"select $col1, $col2 from deltaCaseInsensitiveTest", + testData) + checkAnswer( + s"select $col2, $col1 from deltaCaseInsensitiveTest", + testData.map(_.swap)) + checkAnswer( + s"select $col1 from deltaCaseInsensitiveTest where $col2 = '2'", + testData.filter(_._2 == "2").map(x => OneItem(x._1))) + checkAnswer( + s"select $col2 from deltaCaseInsensitiveTest where $col1 = 2", + testData.filter(_._1 == 2).map(x => OneItem(x._2))) + } + for (col <- Seq("fooBar", "foobar", "FOOBAR")) { + checkAnswer( + s"select $col from deltaCaseInsensitiveTest", + testData.map(x => OneItem(x._1))) + } + for (col <- Seq("barFoo", "barfoo", "BARFOO")) { + checkAnswer( + s"select $col from deltaCaseInsensitiveTest", + testData.map(x => OneItem(x._2))) + } + } + } + } } case class TestStruct(f1: String, f2: Long) @@ -569,3 +618,5 @@ case class TestClass( c14: Map[String, Long], c15: TestStruct ) + +case class OneItem[T](t: T) From 7927a80d8bcc9079683f554bbd9402666cd2a3ae Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 9 Mar 2020 00:18:16 -0700 Subject: [PATCH 020/291] Use SchemaUtils.reportDifferences to report schema differences (#23) This is one example of the new error message: ``` MetaException(message:The Delta table schema is not the same as the Hive schema: Specified schema is missing field(s): a Specified schema has additional field(s): e Delta table schema: root -- a: integer (nullable = true) -- b: string (nullable = true) -- c: string (nullable = true) Hive schema: e: int c: string b: string Please update your Hive table's schema to match the Delta table schema.) ``` --- .../apache/spark/sql/delta/DeltaHelper.scala | 134 ++++++++++-------- 1 file changed, 77 insertions(+), 57 deletions(-) diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index b113c09c01f..cbd39f655e7 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.delta import java.net.URI +import java.util.Locale import scala.collection.mutable import scala.collection.JavaConverters._ @@ -30,6 +31,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.actions.{AddFile, SingleAction} +import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.types._ object DeltaHelper extends Logging { @@ -173,64 +175,78 @@ object DeltaHelper extends Logging { */ @throws(classOf[MetaException]) def checkTableSchema(deltaSchema: StructType, hiveSchema: StructTypeInfo): Unit = { - // TODO How to check column nullables? - if (!isSameStructType(deltaSchema, hiveSchema)) { - throw metaInconsistencyException(deltaSchema, hiveSchema) + val deltaType = normalizeSparkType(deltaSchema).asInstanceOf[StructType] + val hiveType = hiveTypeToSparkType(hiveSchema).asInstanceOf[StructType] + if (deltaType != hiveType) { + val diffs = + SchemaUtils.reportDifferences(existingSchema = deltaType, specifiedSchema = hiveType) + throw metaInconsistencyException( + deltaSchema, + hiveSchema, + // `reportDifferences` doesn't report the column order difference so we report a special + // error message for this case. + if (diffs.isEmpty) "Column order is different" else diffs.mkString("\n")) } } - private def isSameStructType(sparkStruct: StructType, hiveStruct: StructTypeInfo): Boolean = { - if (sparkStruct.size == hiveStruct.getAllStructFieldNames.size) { - (0 until sparkStruct.size).forall { i => - val sparkField = sparkStruct(i) - val hiveFieldName = hiveStruct.getAllStructFieldNames.get(i) - val hiveFieldType = hiveStruct.getAllStructFieldTypeInfos.get(i) - sparkField.name.equalsIgnoreCase(hiveFieldName) && - isSameType(sparkField.dataType, hiveFieldType) - } - } else { - false - } - } - - private def isSameType(sparkType: DataType, hiveType: TypeInfo): Boolean = { + /** + * Normalize the Spark type so that we can compare it with user specified Hive schema. + * - Field names will be converted to lower case. + * - Nullable will be set to `true` since Hive doesn't support non-null fields. + */ + private def normalizeSparkType(sparkType: DataType): DataType = { sparkType match { - case ByteType => hiveType == TypeInfoFactory.byteTypeInfo - case BinaryType => hiveType == TypeInfoFactory.binaryTypeInfo - case BooleanType => hiveType == TypeInfoFactory.booleanTypeInfo - case IntegerType => hiveType == TypeInfoFactory.intTypeInfo - case LongType => hiveType == TypeInfoFactory.longTypeInfo - case StringType => hiveType == TypeInfoFactory.stringTypeInfo - case FloatType => hiveType == TypeInfoFactory.floatTypeInfo - case DoubleType => hiveType == TypeInfoFactory.doubleTypeInfo - case ShortType => hiveType == TypeInfoFactory.shortTypeInfo - case DateType => hiveType == TypeInfoFactory.dateTypeInfo - case TimestampType => hiveType == TypeInfoFactory.timestampTypeInfo - case decimalType: DecimalType => - hiveType match { - case hiveDecimalType: DecimalTypeInfo => - decimalType.precision == hiveDecimalType.precision() && - decimalType.scale == hiveDecimalType.scale() - case _ => false - } + case structType: StructType => + StructType(structType.fields.map(f => StructField( + name = f.name.toLowerCase(Locale.ROOT), + dataType = normalizeSparkType(f.dataType), + nullable = true, + metadata = Metadata.empty + ))) case arrayType: ArrayType => - hiveType match { - case hiveListType: ListTypeInfo => - isSameType(arrayType.elementType, hiveListType.getListElementTypeInfo) - case _ => false - } + ArrayType(normalizeSparkType(arrayType.elementType), containsNull = true) case mapType: MapType => - hiveType match { - case hiveMapType: MapTypeInfo => - isSameType(mapType.keyType, hiveMapType.getMapKeyTypeInfo) && - isSameType(mapType.valueType, hiveMapType.getMapValueTypeInfo) - case _ => false - } - case structType: StructType => - hiveType match { - case hiveStructType: StructTypeInfo => isSameStructType(structType, hiveStructType) - case _ => false - } + MapType( + normalizeSparkType(mapType.keyType), + normalizeSparkType(mapType.valueType), + valueContainsNull = true) + case other => other + } + } + + /** + * Convert a Hive's type to a Spark type so that we can compare it with the underlying Delta Spark + * type. + */ + private def hiveTypeToSparkType(hiveType: TypeInfo): DataType = { + hiveType match { + case TypeInfoFactory.byteTypeInfo => ByteType + case TypeInfoFactory.binaryTypeInfo => BinaryType + case TypeInfoFactory.booleanTypeInfo => BooleanType + case TypeInfoFactory.intTypeInfo => IntegerType + case TypeInfoFactory.longTypeInfo => LongType + case TypeInfoFactory.stringTypeInfo => StringType + case TypeInfoFactory.floatTypeInfo => FloatType + case TypeInfoFactory.doubleTypeInfo => DoubleType + case TypeInfoFactory.shortTypeInfo => ShortType + case TypeInfoFactory.dateTypeInfo => DateType + case TypeInfoFactory.timestampTypeInfo => TimestampType + case hiveDecimalType: DecimalTypeInfo => + DecimalType(precision = hiveDecimalType.precision(), scale = hiveDecimalType.scale()) + case hiveListType: ListTypeInfo => + ArrayType(hiveTypeToSparkType(hiveListType.getListElementTypeInfo), containsNull = true) + case hiveMapType: MapTypeInfo => + MapType( + hiveTypeToSparkType(hiveMapType.getMapKeyTypeInfo), + hiveTypeToSparkType(hiveMapType.getMapValueTypeInfo), + valueContainsNull = true) + case hiveStructType: StructTypeInfo => + val size = hiveStructType.getAllStructFieldNames.size + StructType((0 until size) map { i => + val hiveFieldName = hiveStructType.getAllStructFieldNames.get(i) + val hiveFieldType = hiveStructType.getAllStructFieldTypeInfos.get(i) + StructField(hiveFieldName.toLowerCase(Locale.ROOT), hiveTypeToSparkType(hiveFieldType)) + }) case _ => // TODO More Hive types: // - void @@ -240,27 +256,31 @@ object DeltaHelper extends Logging { // - intervalDayTimeType // - UnionType // - Others? - throw new UnsupportedOperationException(s"Spark type $sparkType is not supported Hive") + throw new UnsupportedOperationException(s"Hive type $hiveType is not supported") } } private def metaInconsistencyException( deltaSchema: StructType, - hiveSchema: StructTypeInfo): MetaException = { + hiveSchema: StructTypeInfo, + diffs: String): MetaException = { val hiveSchemaString = hiveSchema.getAllStructFieldNames .asScala .zip(hiveSchema.getAllStructFieldTypeInfos.asScala.map(_.getTypeName)) .map(_.productIterator.mkString(": ")) .mkString("\n") new MetaException( - s"""The Delta table schema is not the same as the Hive schema. Please update your Hive - |table's schema to match the Delta table schema. + s"""The Delta table schema is not the same as the Hive schema: + | + |$diffs | |Delta table schema: |${deltaSchema.treeString} | |Hive schema: - |${hiveSchemaString}""".stripMargin) + |$hiveSchemaString + | + |Please update your Hive table's schema to match the Delta table schema.""".stripMargin) } // TODO Configure `spark` to pick up the right Hadoop configuration. From b85fdd6fbb574c700429a5d57da2387efdc9967b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 9 Mar 2020 00:18:56 -0700 Subject: [PATCH 021/291] Optimize the connector memory usage (#24) - Disable Spark UI and remove `SparkListener`s to reduce the memory usage of Spark. They are unnecessary for the Spark cluster in connector. - Set the `DeltaLog` cache size to 1 if the user doesn't specify it to avoid caching lots of `DeltaLog` objects. --- .../apache/spark/sql/delta/DeltaHelper.scala | 30 +++++++++++++++---- .../io/delta/hive/HiveConnectorSuite.scala | 9 ------ .../scala/io/delta/hive/test/HiveTest.scala | 9 ++---- 3 files changed, 27 insertions(+), 21 deletions(-) diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index cbd39f655e7..64b323719af 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -99,6 +99,8 @@ object DeltaHelper extends Logging { val files = DeltaLog.filterFileList( snapshotToUse.metadata.partitionColumns, snapshotToUse.allFiles.toDF(), convertedFilterExpr) .as[AddFile](SingleAction.addFileEncoder) + // Drop unused potential huge fields + .map(add => add.copy(stats = null, tags = null))(SingleAction.addFileEncoder) .collect().map { f => logInfo(s"selected delta file ${f.path} under $rootPath") val status = toFileStatus(fs, rootPath, f, blockSize) @@ -283,9 +285,27 @@ object DeltaHelper extends Logging { |Please update your Hive table's schema to match the Delta table schema.""".stripMargin) } - // TODO Configure `spark` to pick up the right Hadoop configuration. - def spark: SparkSession = SparkSession.builder() - .master("local[*]") - .appName("HiveOnDelta Get Files") - .getOrCreate() + /** + * Start a special Spark cluster using local mode to process Delta's metadata. The Spark UI has + * been disabled and `SparkListener`s have been removed to reduce the memory usage of Spark. + * `DeltaLog` cache size is also set to "1" if the user doesn't specify it, to cache only the + * recent accessed `DeltaLog`. + */ + def spark: SparkSession = { + // TODO Configure `spark` to pick up the right Hadoop configuration. + if (System.getProperty("delta.log.cacheSize") == null) { + System.setProperty("delta.log.cacheSize", "1") + } + val sparkSession = SparkSession.builder() + .master("local[*]") + .appName("Delta Connector") + .config("spark.ui.enabled", "false") + .getOrCreate() + // Trigger codes that add `SparkListener`s before stopping the listener bus. Otherwise, they + // would fail to add `SparkListener`s. + sparkSession.sharedState + sparkSession.sessionState + sparkSession.sparkContext.listenerBus.stop() + sparkSession + } } diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala index d0bf574ca90..fedde435859 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala @@ -22,19 +22,10 @@ import io.delta.hive.test.HiveTest import io.delta.tables.DeltaTable import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.delta.DeltaLog import org.scalatest.BeforeAndAfterEach class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { - override def beforeEach(): Unit = { - DeltaLog.clearCache() - } - - override def afterEach(): Unit = { - DeltaLog.clearCache() - } - test("should not allow to create a non external Delta table") { val e = intercept[Exception] { runQuery( diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 6d313005225..9073aae5d57 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -31,10 +31,10 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.mapred.{JobConf, MiniMRCluster} import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.spark.SparkConf import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.DeltaHelper // scalastyle:off funsuite import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -160,12 +160,7 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { } protected def withSparkSession(f: SparkSession => Unit): Unit = { - val conf = new SparkConf() - val spark = SparkSession.builder() - .appName("HiveConnectorSuite") - .master("local[2]") - .getOrCreate() - + val spark = DeltaHelper.spark try f(spark) finally { // Clean up resources so that we can use new DeltaLog and SparkSession spark.stop() From 85bdeddb68363ab0da28765541aa47b4fc348d7b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 9 Mar 2020 10:59:38 -0700 Subject: [PATCH 022/291] Cross build Scala 2.11 and 2.12 (#22) Add build support for Scala 2.11. Closes #18 --- .circleci/config.yml | 3 ++- README.md | 11 +++++++++-- build.sbt | 9 ++++++++- coreTest/src/main/scala/Test.scala | 27 +++++++++++++++------------ 4 files changed, 34 insertions(+), 16 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 9466cb1ec54..1b22d38ae61 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -8,4 +8,5 @@ jobs: - run: name: Run tests command: | - build/sbt coreTest/test hive/test + build/sbt "+ coreTest/test" + build/sbt "+ hive/test" diff --git a/README.md b/README.md index 2cd16d8b8c4..324aef18762 100644 --- a/README.md +++ b/README.md @@ -29,12 +29,19 @@ This project contains all the code needed to make Hive read Delta Lake tables. The above commands will generate two JARs in the following paths. ``` -core/target/scala-2.12/delta-core-shaded-assembly-0.5.0.jar +core/target/scala-2.12/delta-core-shaded-assembly_2.12-0.5.0.jar hive/target/scala-2.12/hive-delta_2.12-0.5.0.jar ``` These two JARs include the Hive connector and all its dependencies. They need to be put in Hive’s classpath. +Note: if you would like to build jars using Scala 2.11, you can run the SBT command `build/sbt "++ 2.11.12 hive/package"` and the generated JARS will be in the following paths. + +``` +core/target/scala-2.11/delta-core-shaded-assembly_2.11-0.5.0.jar +hive/target/scala-2.12/hive-delta_2.11-0.5.0.jar +``` + ### Setting up Hive This sections describes how to set up Hive to load the Delta Hive connector. @@ -48,7 +55,7 @@ Before starting your Hive CLI or running your Hive script, add the following spe ``` -The second step is to upload the above two JARs to the machine that runs Hive. Finally, add the paths of the JARs toHive’s environment variable, `HIVE_AUX_JARS_PATH`. You can find this environment variable in the `hive-env.sh` file, whose location is ``/etc/hive/conf/hive-env.sh` on an EMR cluster. This setting will tell Hive where to find the connector JARs. +The second step is to upload the above two JARs to the machine that runs Hive. Finally, add the paths of the JARs toHive’s environment variable, `HIVE_AUX_JARS_PATH`. You can find this environment variable in the `hive-env.sh` file, whose location is `/etc/hive/conf/hive-env.sh` on an EMR cluster. This setting will tell Hive where to find the connector JARs. ### Create a Hive table diff --git a/build.sbt b/build.sbt index 67db1a9dad9..f29e25c2e06 100644 --- a/build.sbt +++ b/build.sbt @@ -20,6 +20,8 @@ scalastyleConfig in ThisBuild := baseDirectory.value / "scalastyle-config.xml" lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") lazy val testScalastyle = taskKey[Unit]("testScalastyle") +crossScalaVersions := Seq("2.12.8", "2.11.12") + val sparkVersion = "2.4.3" val hadoopVersion = "2.7.2" val hiveVersion = "2.3.3" @@ -92,8 +94,13 @@ lazy val assemblySettings = Seq( case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first }, + assemblyJarName in assembly := s"${name.value}-assembly_${scalaBinaryVersion.value}-${version.value}.jar", - assemblyShadeRules in assembly := Seq( + assemblyShadeRules in assembly := + (if (scalaBinaryVersion.value == "2.11") Seq( + // json4s cannot be shaded when using Scala 2.11 + ShadeRule.rename("org.json4s.**" -> "@0").inAll + ) else Nil) ++ Seq( /* All org.apache.* before shading: arrow, avro, commons, curator, ivy, jute, log4j, orc, oro, parquet, spark, xbean, zookeeper diff --git a/coreTest/src/main/scala/Test.scala b/coreTest/src/main/scala/Test.scala index 61ea85e33b6..35f7612310c 100644 --- a/coreTest/src/main/scala/Test.scala +++ b/coreTest/src/main/scala/Test.scala @@ -18,12 +18,14 @@ package test import shadedelta.io.delta.tables._ +import java.nio.file.Files + +import scala.collection.JavaConverters._ + import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -// import shadedelta.org.apache.commons.io.FileUtils - -import java.io.File +// scalastyle:off println object Test { def main(args: Array[String]): Unit = { @@ -36,23 +38,24 @@ object Test { // Create a table - // scalastyle:off println - println("Creating a table") - // scalastyle:on println - val file = new File("/tmp/delta-table") - val path = file.getCanonicalPath + val dir = Files.createTempDirectory("delta-table") + println(s"Creating a table at $dir") + val path = dir.toFile.getCanonicalPath var data = spark.range(0, 5) data.write.format("delta").mode("overwrite").save(path) // Read table - // scalastyle:off println println("Reading the table") - // scalastyle:on println val df = spark.read.format("delta").load(path) df.show() - // Cleanup - // FileUtils.deleteDirectory(file) spark.stop() + + // Cleanup + println("Finished the test. Cleaning up the table") + Files.walk(dir).iterator.asScala.toSeq.reverse.foreach { f => + println(s"Deleting $f") + Files.delete(f) + } } } From a92d3722c6738e70789421917a5e18bd54ab71fe Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 24 Mar 2020 00:55:14 -0700 Subject: [PATCH 023/291] Run unit tests for Tez (#29) - Move the unit tests for Hive MR to HiveMR project - Add a new project HiveTez to run unit tests in Tez mode. - Document that `hive.tez.input.format` should also be set to `io.delta.hive.HiveInputFormat`. - Fix the dependency issue for Tez - Remove dependencies used by Spark UI because Tez has its own UI and its dependencies conflict with Spark's. - Exclude `org.xerial.snappy` and don't shade it. Tez uses a different version and doesn't work with the Spark one. Closes #21 --- .circleci/config.yml | 8 +- README.md | 11 ++ build.sbt | 102 +++++++++++++++- coreTest/src/main/scala/Test.scala | 1 + .../scala/io/delta/hive/HiveMRSuite.scala | 50 ++++++++ .../scala/io/delta/hive/HiveTezSuite.scala | 112 ++++++++++++++++++ ...torSuite.scala => HiveConnectorTest.scala} | 79 +++++------- .../scala/io/delta/hive/test/HiveTest.scala | 51 ++++---- .../io/delta/hive/test/HiveTestSuite.scala | 32 ----- 9 files changed, 340 insertions(+), 106 deletions(-) create mode 100644 hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala create mode 100644 hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala rename hive/src/test/scala/io/delta/hive/{HiveConnectorSuite.scala => HiveConnectorTest.scala} (88%) delete mode 100644 hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala diff --git a/.circleci/config.yml b/.circleci/config.yml index 1b22d38ae61..0e468ea31f3 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -8,5 +8,9 @@ jobs: - run: name: Run tests command: | - build/sbt "+ coreTest/test" - build/sbt "+ hive/test" + build/sbt "++ 2.12.8 coreTest/test" + build/sbt "++ 2.12.8 hiveMR/test" + build/sbt "++ 2.12.8 hiveTez/test" + build/sbt "++ 2.11.12 coreTest/test" + build/sbt "++ 2.11.12 hiveMR/test" + build/sbt "++ 2.11.12 hiveTez/test" diff --git a/README.md b/README.md index 324aef18762..972dd4bbd66 100644 --- a/README.md +++ b/README.md @@ -53,6 +53,17 @@ Before starting your Hive CLI or running your Hive script, add the following spe hive.input.format io.delta.hive.HiveInputFormat + + hive.tez.input.format + io.delta.hive.HiveInputFormat + +``` + +Alternatively, you can also run the following SQL commands in Hive CLI before reading Delta tables to set `io.delta.hive.HiveInputFormat`: + +``` +SET hive.input.format=io.delta.hive.HiveInputFormat; +SET hive.tez.input.format=io.delta.hive.HiveInputFormat; ``` The second step is to upload the above two JARs to the machine that runs Hive. Finally, add the paths of the JARs toHive’s environment variable, `HIVE_AUX_JARS_PATH`. You can find this environment variable in the `hive-env.sh` file, whose location is `/etc/hive/conf/hive-env.sh` on an EMR cluster. This setting will tell Hive where to find the connector JARs. diff --git a/build.sbt b/build.sbt index f29e25c2e06..cf71edbf2c1 100644 --- a/build.sbt +++ b/build.sbt @@ -53,7 +53,27 @@ lazy val core = (project in file("core")) name := "delta-core-shaded", libraryDependencies ++= Seq( "io.delta" %% "delta-core" % deltaVersion excludeAll ExclusionRule("org.apache.hadoop"), - "org.apache.spark" %% "spark-sql" % sparkVersion excludeAll ExclusionRule("org.apache.hadoop"), + "org.apache.spark" %% "spark-sql" % sparkVersion excludeAll( + ExclusionRule("org.apache.hadoop"), + // Remove all dependencies used by Spark UI. Spark UI is not needed and we have disabled it. + // So these dependencies are not used any more. + ExclusionRule("com.sun.jersey"), + ExclusionRule("org.glassfish"), + ExclusionRule("org.glassfish.jersey.bundles"), + ExclusionRule("org.glassfish.jersey.media"), + ExclusionRule("org.glassfish.hk2"), + ExclusionRule("org.glassfish.jersey.bundles.repackaged"), + ExclusionRule("org.glassfish.jersey.test-framework"), + ExclusionRule("org.glassfish.hk2.external"), + ExclusionRule("org.glassfish.jersey.containers"), + ExclusionRule("org.glassfish.jersey.test-framework.providers"), + ExclusionRule("org.glassfish.jaxb"), + ExclusionRule("org.glassfish.jersey.core"), + ExclusionRule("org.glassfish.jersey"), + ExclusionRule("org.glassfish.jersey.inject"), + ExclusionRule("javax.ws.rs"), + ExclusionRule("org.xerial.snappy") + ), "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" ), @@ -70,8 +90,10 @@ lazy val coreTest = (project in file("coreTest")) unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, // Only dependency not in the uber jar - libraryDependencies += "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll + libraryDependencies ++= Seq("org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll ExclusionRule("org.slf4j", "slf4j-log4j12"), + "org.xerial.snappy" % "snappy-java" % "1.1.7.3" + ), autoScalaLibrary := false, @@ -117,7 +139,7 @@ lazy val assemblySettings = Seq( ShadeRule.rename("org.apache.log4j.**" -> "@0").inAll, // Initialization via reflection fails when package changed ShadeRule.rename("org.slf4j.**" -> "@0").inAll, // Initialization via reflection fails when package changed ShadeRule.rename("org.apache.commons.**" -> "@0").inAll, // Initialization via reflection fails when package changed - ShadeRule.rename("org.xerial.snappy.*Native*" -> "@0").inAll, // JNI class fails to resolve native code when package changed + ShadeRule.rename("org.xerial.snappy.**" -> "@0").inAll, // Snappy fails to resolve native code when package changed ShadeRule.rename("com.databricks.**" -> "@0").inAll, // Scala package object does not resolve correctly when package changed // Shade everything else @@ -153,6 +175,38 @@ lazy val hive = (project in file("hive")) settings ( // Ensures that the connector core jar is compiled before compiling this project (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, + // Minimal dependencies to compile the codes. This project doesn't run any tests so we don't need + // any runtime dependencies. + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "com.google.protobuf") + ), + "org.apache.hive" % "hive-cli" % hiveVersion % "test" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("ch.qos.logback", "logback-classic"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "com.google.protobuf") + ), + "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", + "org.scalatest" %% "scalatest" % "3.0.5" % "test", + "io.delta" %% "delta-core" % deltaVersion % "test" + ) +) + +lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") settings ( + name := "hive-mr", + commonSettings, + unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, + autoScalaLibrary := false, + + // Ensures that the connector core jar is compiled before compiling this project + (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, + libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( @@ -176,3 +230,45 @@ lazy val hive = (project in file("hive")) settings ( "io.delta" %% "delta-core" % deltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") ) ) + +lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") settings ( + name := "hive-tez", + commonSettings, + unmanagedJars in Compile += (packageBin in(core, Compile, packageBin)).value, + autoScalaLibrary := false, + // Ensures that the connector core jar is compiled before compiling this project + (compile in Compile) := ((compile in Compile) dependsOn (packageBin in (core, Compile, packageBin))).value, + + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" excludeAll ( + ExclusionRule(organization = "com.google.protobuf") + ), + "com.google.protobuf" % "protobuf-java" % "2.5.0", + "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "com.google.protobuf") + ), + "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-mapreduce-client-hs" % hadoopVersion % "test", + "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion % "test" classifier "tests", + "org.apache.hadoop" % "hadoop-yarn-server-tests" % hadoopVersion % "test" classifier "tests", + "org.apache.hive" % "hive-cli" % hiveVersion % "test" excludeAll( + ExclusionRule(organization = "org.apache.spark"), + ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule("ch.qos.logback", "logback-classic"), + ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "com.google.protobuf") + ), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion % "test", + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion % "test", + "org.apache.tez" % "tez-mapreduce" % "0.8.4" % "test", + "org.apache.tez" % "tez-dag" % "0.8.4" % "test", + "org.apache.tez" % "tez-tests" % "0.8.4" % "test" classifier "tests", + // TODO Figure out how this fixes some bad dependency + "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", + "org.scalatest" %% "scalatest" % "3.0.5" % "test", + "io.delta" %% "delta-core" % deltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") + ) +) diff --git a/coreTest/src/main/scala/Test.scala b/coreTest/src/main/scala/Test.scala index 35f7612310c..479295ccdb3 100644 --- a/coreTest/src/main/scala/Test.scala +++ b/coreTest/src/main/scala/Test.scala @@ -34,6 +34,7 @@ object Test { .builder() .appName("Quickstart") .master("local[*]") + .config("spark.ui.enabled", "false") .getOrCreate() diff --git a/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala new file mode 100644 index 00000000000..b82f1921251 --- /dev/null +++ b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.hive + +import java.io.{Closeable, File} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{JobConf, MiniMRCluster} +import org.apache.hadoop.mapreduce.MRJobConfig +import org.apache.hadoop.yarn.conf.YarnConfiguration + +class HiveMRSuite extends HiveConnectorTest { + + override val engine: String = "mr" + + override def createCluster(namenode: String, conf: Configuration, tempPath: File): Closeable = { + val jConf = new JobConf(conf); + jConf.set("yarn.scheduler.capacity.root.queues", "default"); + jConf.set("yarn.scheduler.capacity.root.default.capacity", "100"); + jConf.setInt(MRJobConfig.MAP_MEMORY_MB, 512); + jConf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 512); + jConf.setInt(MRJobConfig.MR_AM_VMEM_MB, 128); + jConf.setInt(YarnConfiguration.YARN_MINICLUSTER_NM_PMEM_MB, 512); + jConf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128); + jConf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 512); + val mr = new MiniMRCluster(2, namenode, 1, null, null, jConf) + + new Closeable { + override def close(): Unit = { + mr.shutdown() + } + } + } +} diff --git a/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala new file mode 100644 index 00000000000..a6fd1930453 --- /dev/null +++ b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala @@ -0,0 +1,112 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.hive + +import java.io.{Closeable, File} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.MRJobConfig +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.tez.dag.api.TezConfiguration +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration +import org.apache.tez.test.MiniTezCluster + +class HiveTezSuite extends HiveConnectorTest { + + override val engine: String = "tez" + + private var tezConf: Configuration = _ + + // scalastyle:off + /** + * This method is based on + * https://github.com/apache/hive/blob/c660cba003f9b7fff29db2202b375982a8c03450/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java#L406 + */ + // scalastyle:on + override def createCluster( + namenode: String, + conf: Configuration, + tempPath: File): Closeable = new Closeable { + private val tez = { + assert(sys.env("JAVA_HOME") != null, "Cannot find JAVA_HOME") + val tez = new MiniTezCluster("hivetest", 2) + conf.setInt(YarnConfiguration.YARN_MINICLUSTER_NM_PMEM_MB, 256) + conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 256) + conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 256) + // Overrides values from the hive/tez-site. + conf.setInt("hive.tez.container.size", 256) + conf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 256) + conf.setInt(TezConfiguration.TEZ_TASK_RESOURCE_MEMORY_MB, 256) + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 24) + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB, 10) + conf.setFloat(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT, 0.4f) + conf.set("fs.defaultFS", namenode) + conf.set("tez.am.log.level", "DEBUG") + conf.set( + MRJobConfig.MR_AM_STAGING_DIR, + new File(tempPath, "apps_staging_dir").getAbsolutePath) + // - Set `spark.testing.reservedMemory` in the test so that Spark doesn't check the physical + // memory size. We are using a very small container and that's enough for testing. + // - Reduce the partition number to 1 to reduce the memory usage of Spark because CircleCI has + // a small physical memory limit. + // - Set the default timezone so that the answers of tests using timestamp is not changed when + // running in CircleCI. + conf.set("tez.am.launch.cmd-opts", + "-Dspark.testing.reservedMemory=0 " + + "-Dspark.sql.shuffle.partitions=1 " + + "-Dspark.databricks.delta.snapshotPartitions=1 " + + "-Duser.timezone=America/Los_Angeles") + conf.set("tez.task.launch.cmd-opts", "-Duser.timezone=America/Los_Angeles") + // Disable disk health check and authorization + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 100.0F) + conf.setBoolean(YarnConfiguration.NM_DISK_HEALTH_CHECK_ENABLE, false) + conf.setBoolean("hadoop.security.authorization", false) + tez.init(conf) + tez.start() + tezConf = tez.getConfig + tez + } + + override def close(): Unit = { + tez.stop() + } + } + + // scalastyle:off + /** + * The method is based on + * https://github.com/apache/hive/blob/c660cba003f9b7fff29db2202b375982a8c03450/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java#L446 + */ + // scalastyle:on + override def setupConfiguration(conf: Configuration): Unit = { + tezConf.asScala.foreach { e => + conf.set(e.getKey, e.getValue) + } + // Overrides values from the hive/tez-site. + conf.setInt("hive.tez.container.size", 256) + conf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 256) + conf.setInt(TezConfiguration.TEZ_TASK_RESOURCE_MEMORY_MB, 256) + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 24) + conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_UNORDERED_OUTPUT_BUFFER_SIZE_MB, 10) + conf.setFloat(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT, 0.4f) + conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true) + } +} diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala similarity index 88% rename from hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala rename to hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala index fedde435859..0851d7a3518 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorSuite.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala @@ -24,7 +24,7 @@ import io.delta.tables.DeltaTable import org.apache.spark.network.util.JavaUtils import org.scalatest.BeforeAndAfterEach -class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { +abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("should not allow to create a non external Delta table") { val e = intercept[Exception] { @@ -83,8 +83,15 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { val e = intercept[Exception] { runQuery("INSERT INTO deltaTbl(a, b) VALUES(123, 'foo')") } - assert(e.getMessage != null && e.getMessage.contains( - "Writing to a Delta table in Hive is not supported")) + if (engine == "tez") { + // We cannot get the root cause in Tez mode because of HIVE-20974. Currently it's only in + // the log so we cannot verify it. + // TODO Remove this `if` branch once we upgrade to a new Hive version containing the fix + // for HIVE-20974 + } else { + assert(e.getMessage != null && e.getMessage.contains( + "Writing to a Delta table in Hive is not supported")) + } } } } @@ -333,82 +340,58 @@ class HiveConnectorSuite extends HiveTest with BeforeAndAfterEach { ) // equal pushed down - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'") - .mkString(" ").contains("filterExpr: (date = '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` = '20180520'", + "(date = '20180520')", testData.filter(_._2 == "20180520")) - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl " + - "where `date` != '20180520'") - .mkString(" ").contains("filterExpr: (date <> '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` != '20180520'", + "(date <> '20180520')", testData.filter(_._2 != "20180520")) - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'") - .mkString(" ").contains("filterExpr: (date > '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` > '20180520'", + "(date > '20180520')", testData.filter(_._2 > "20180520")) - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl " + - "where `date` >= '20180520'") - .mkString(" ").contains("filterExpr: (date >= '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` >= '20180520'", + "(date >= '20180520')", testData.filter(_._2 >= "20180520")) - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'") - .mkString(" ").contains("filterExpr: (date < '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` < '20180520'", + "(date < '20180520')", testData.filter(_._2 < "20180520")) - assert(runQuery( - "explain select city, `date`, name, cnt from deltaPartitionTbl " + - "where `date` <= '20180520'") - .mkString(" ").contains("filterExpr: (date <= '20180520')")) - checkAnswer( + checkFilterPushdown( "select city, `date`, name, cnt from deltaPartitionTbl where `date` <= '20180520'", + "(date <= '20180520')", testData.filter(_._2 <= "20180520")) // expr(like) pushed down - assert(runQuery( - "explain select * from deltaPartitionTbl where `date` like '201805%'") - .mkString(" ").contains("filterExpr: (date like '201805%')")) - checkAnswer( + checkFilterPushdown( "select * from deltaPartitionTbl where `date` like '201805%'", - testData.filter(_._2.contains("201805"))) + "(date like '201805%')", + testData.filter(_._2.startsWith("201805"))) // expr(in) pushed down - assert(runQuery( - "explain select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')") - .mkString(" ").contains("filterExpr: (city) IN ('hz', 'sz')")) - checkAnswer( + checkFilterPushdown( "select name, `date`, cnt from deltaPartitionTbl where `city` in ('hz', 'sz')", + "(city) IN ('hz', 'sz')", testData.filter(c => Seq("hz", "sz").contains(c._1)).map(r => (r._3, r._2, r._4))) // two partition column pushed down - assert(runQuery( - "explain select * from deltaPartitionTbl " + - "where `date` = '20181212' and `city` in ('hz', 'sz')") - .mkString(" ").contains("filterExpr: ((city) IN ('hz', 'sz') and (date = '20181212'))")) - checkAnswer( + checkFilterPushdown( "select * from deltaPartitionTbl where `date` = '20181212' and `city` in ('hz', 'sz')", + "((city) IN ('hz', 'sz') and (date = '20181212'))", testData.filter(c => Seq("hz", "sz").contains(c._1) && c._2 == "20181212")) // data column not be pushed down - assert(runQuery( - "explain select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'") - .mkString(" ").contains("filterExpr: (city = 'hz'")) - checkAnswer( + checkFilterPushdown( "select * from deltaPartitionTbl where city = 'hz' and name = 'Jim'", + "(city = 'hz')", testData.filter(c => c._1 == "hz" && c._3 == "Jim")) } } diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 9073aae5d57..2816800918f 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -16,21 +16,20 @@ package io.delta.hive.test -import java.io.File +import java.io.{Closeable, File} import java.nio.file.Files import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hive.cli.CliSessionState import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.mapred.{JobConf, MiniMRCluster} -import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.yarn.conf.YarnConfiguration + import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog @@ -43,7 +42,7 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { private val tempPath = Files.createTempDirectory(this.getClass.getSimpleName).toFile private var driver: Driver = _ - private var mr: MiniMRCluster = _ + private var cluster: Closeable = _ // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) @@ -54,42 +53,42 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { super.beforeAll() val warehouseDir = new File(tempPath, "warehouse") val metastoreDir = new File(tempPath, "metastore_db") + val hiveJarsDir = new File(tempPath, "hive_jars") val conf = new HiveConf() - + conf.set("hive.execution.engine", engine) + conf.set("hive.user.install.directory", hiveJarsDir.getCanonicalPath) // Disable schema verification and allow schema auto-creation in the // Derby database, in case the config for the metastore is set otherwise. // Without these settings, starting the client fails with // MetaException(message:Version information not found in metastore.)t conf.set("hive.metastore.schema.verification", "false") conf.set("datanucleus.schema.autoCreateAll", "true") - // if hive.fetch.task.conversion set to none, the hive.input.format should be - // io.delta.hive.HiveInputFormat + // if hive.fetch.task.conversion set to none, "hive.input.format" and "hive.tez.input.format" + // should be "io.delta.hive.HiveInputFormat". conf.set("hive.fetch.task.conversion", "none") conf.set("hive.input.format", "io.delta.hive.HiveInputFormat") + conf.set("hive.tez.input.format", "io.delta.hive.HiveInputFormat") conf.set( "javax.jdo.option.ConnectionURL", s"jdbc:derby:memory:;databaseName=${metastoreDir.getCanonicalPath};create=true") conf.set("hive.metastore.warehouse.dir", warehouseDir.getCanonicalPath) val fs = FileSystem.getLocal(conf) - val jConf = new JobConf(conf) - jConf.set("yarn.scheduler.capacity.root.queues", "default") - jConf.set("yarn.scheduler.capacity.root.default.capacity", "100") - jConf.setInt(MRJobConfig.MAP_MEMORY_MB, 512) - jConf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 512) - jConf.setInt(MRJobConfig.MR_AM_VMEM_MB, 128) - jConf.setInt(YarnConfiguration.YARN_MINICLUSTER_NM_PMEM_MB, 512) - jConf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128) - jConf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 512) - mr = new MiniMRCluster(2, fs.getUri.toString, 1, null, null, jConf) - + cluster = createCluster(fs.getUri.toString, conf, tempPath) + setupConfiguration(conf) val db = Hive.get(conf) SessionState.start(new CliSessionState(conf)) driver = new Driver(conf) } + def engine: String + + def createCluster(namenode: String, conf: Configuration, tempPath: File): Closeable + + def setupConfiguration(conf: Configuration): Unit = {} + override def afterAll() { - if (mr != null) { - mr.shutdown() + if (cluster != null) { + cluster.close() } driver.close() driver.destroy() @@ -134,6 +133,16 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { } } + /** + * Check whether the `filter` is pushed into TableScan's filterExpr field and also verify the + * answer. + */ + def checkFilterPushdown[T <: Product](query: String, filter: String, expected: Seq[T]): Unit = { + // `explain` in Tez doesn't show TableScan's filterExpr field, so we use `explain extended`. + assert(runQuery(s"explain extended $query").mkString("\n").contains(s"filterExpr: $filter")) + checkAnswer(query, expected) + } + /** * Drops table `tableName` after calling `f`. */ diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala b/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala deleted file mode 100644 index f920512a7d6..00000000000 --- a/hive/src/test/scala/io/delta/hive/test/HiveTestSuite.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Copyright 2019 Databricks, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.hive.test - -import org.scalatest.Ignore - -// TODO Figure out why running this test will cause other tests fail. Probably due to some unknonw -// dependency conflicts. -@Ignore -class HiveTestSuite extends HiveTest { - - test("basic hive query") { - runQuery("create database testdb1") - runQuery("use testdb1") - runQuery("create table testtbl1 as select 'foo' as key") - assert(runQuery("select * from testdb1.testtbl1") === Seq("foo")) - } -} From 1f8b4700b8a6591e65c876bf739947069badf016 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 24 Mar 2020 22:10:12 -0700 Subject: [PATCH 024/291] Fail incorrect format and throw an error rather then returning NULL values (#30) When a user doesn't set the input format to `io.delta.hive.HiveInputFormat`, we will return NULL values right now. This is pretty bad as the user may not notice it. It's better to throw an error when the input format is not set. --- .../io/delta/hive/DeltaInputFormat.scala | 41 +++++++++++++++++++ .../io/delta/hive/HiveConnectorTest.scala | 30 ++++++++++++++ .../scala/io/delta/hive/test/HiveTest.scala | 17 ++++++++ 3 files changed, 88 insertions(+) diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index cdd777c0593..a0361e7d6ff 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -21,6 +21,7 @@ import java.net.URI import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport @@ -87,6 +88,7 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) @throws(classOf[IOException]) override def listStatus(job: JobConf): Array[FileStatus] = { + checkHiveConf(job) val deltaRootPath = new Path(job.get(DeltaStorageHandler.DELTA_TABLE_PATH)) TokenCache.obtainTokensForNamenodes(job.getCredentials(), Array(deltaRootPath), job) val (files, partitions) = @@ -103,6 +105,45 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) files } + private def checkHiveConf(job: JobConf): Unit = { + val engine = HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE) + val deltaFormat = classOf[HiveInputFormat].getName + engine match { + case "mr" => + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT) != deltaFormat) { + throw deltaFormatError(engine, HiveConf.ConfVars.HIVEINPUTFORMAT.varname, deltaFormat) + } + case "tez" => + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVETEZINPUTFORMAT) != deltaFormat) { + throw deltaFormatError(engine, HiveConf.ConfVars.HIVETEZINPUTFORMAT.varname, deltaFormat) + } + case other => + throw new UnsupportedOperationException(s"The execution engine '$other' is not supported." + + s" Please set '${HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname}' to 'mr' or 'tez'") + } + } + + private def deltaFormatError( + engine: String, + formatConfig: String, + deltaFormat: String): Throwable = { + val message = + s"""'$formatConfig' must be set to '$deltaFormat' when reading a Delta table using + |'$engine' execution engine. You can run the following SQL command in Hive CLI + |before reading a Delta table, + | + |> SET $formatConfig=$deltaFormat; + | + |or add the following config to the "hive-site.xml" file. + | + | + | $formatConfig + | $deltaFormat + | + """.stripMargin + new IllegalArgumentException(message) + } + override def makeSplit( file: Path, start: Long, diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala index 0851d7a3518..7fdacfe96f6 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala @@ -570,6 +570,36 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { } } } + + test("fail incorrect format config") { + val formatKey = engine match { + case "mr" => "hive.input.format" + case "tez" => "hive.tez.input.format" + case other => throw new UnsupportedOperationException(s"Unsupported engine: $other") + } + withTable("deltaTbl") { + withTempDir { dir => + withSparkSession { spark => + import spark.implicits._ + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + testData.toDS.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) + |STORED BY 'io.delta.hive.DeltaStorageHandler' + |LOCATION '${dir.getCanonicalPath}'""".stripMargin) + } + withHiveConf(formatKey, "org.apache.hadoop.hive.ql.io.HiveInputFormat") { + val e = intercept[Exception] { + runQuery("SELECT * from deltaTbl") + } + assert(e.getMessage.contains(formatKey)) + assert(e.getMessage.contains(classOf[HiveInputFormat].getName)) + } + } + } } case class TestStruct(f1: String, f2: Long) diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 2816800918f..38c406e7d55 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -176,4 +176,21 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { DeltaLog.clearCache() } } + + protected def withHiveConf(key: String, value: String)(body: => Unit): Unit = { + val hiveConfField = driver.getClass.getDeclaredField("conf") + hiveConfField.setAccessible(true) + val hiveConf = hiveConfField.get(driver).asInstanceOf[HiveConf] + val original = hiveConf.get(key) + try { + hiveConf.set(key, value) + body + } finally { + if (original == null) { + hiveConf.unset(key) + } else { + hiveConf.set(key, original) + } + } + } } From aeff6c8df40beb02950df12055585b5dce77073f Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 25 Mar 2020 09:41:00 -0700 Subject: [PATCH 025/291] Get table schema from table properties directly (#31) so that we don't need to store the property `delta.table.schema` in Hive Metastore. --- .../io/delta/hive/DeltaStorageHandler.scala | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index e99a06086e9..81d30caf6e7 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -28,6 +28,8 @@ import org.apache.hadoop.hive.metastore.api.Table import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.index.IndexSearchCondition +import org.apache.hadoop.hive.ql.io.IOConstants +import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler @@ -64,8 +66,15 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook tableDesc: TableDesc, jobProperties: java.util.Map[String, String]): Unit = { super.configureInputJobProperties(tableDesc, jobProperties) - jobProperties.put(DELTA_TABLE_PATH, tableDesc.getProperties().getProperty(META_TABLE_LOCATION)) - jobProperties.put(DELTA_TABLE_SCHEMA, tableDesc.getProperties().getProperty(DELTA_TABLE_SCHEMA)) + val tableProps = tableDesc.getProperties() + val columnNames = + DataWritableReadSupport.getColumnNames(tableProps.getProperty(IOConstants.COLUMNS)) + val columnTypes = + DataWritableReadSupport.getColumnTypes(tableProps.getProperty(IOConstants.COLUMNS_TYPES)) + val hiveSchema = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes) + .asInstanceOf[StructTypeInfo] + jobProperties.put(DELTA_TABLE_PATH, tableProps.getProperty(META_TABLE_LOCATION)) + jobProperties.put(DELTA_TABLE_SCHEMA, hiveSchema.toString) } override def decomposePredicate( @@ -179,10 +188,8 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook throw new MetaException(s"$deltaRootString does not exist or it's not a Delta table") } - // Extract the table schema in Hive and put it into the table property. Then we can compare it - // with the latest table schema in Delta logs and fail the query if it was changed. - // TODO Investigate if we can get the table schema without manually storing it in the table - // property. + // Extract the table schema in Hive to compare it with the latest table schema in Delta logs, + // and fail the query if it was changed. val cols = tbl.getSd.getCols val columnNames = new JArrayList[String](cols.size) val columnTypes = new JArrayList[TypeInfo](cols.size) @@ -193,7 +200,6 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook val hiveSchema = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes) .asInstanceOf[StructTypeInfo] DeltaHelper.checkTableSchema(snapshot.metadata.schema, hiveSchema) - tbl.getParameters.put(DELTA_TABLE_SCHEMA, hiveSchema.toString) tbl.getParameters.put("spark.sql.sources.provider", "DELTA") tbl.getSd.getParameters.put("path", deltaRootString) } @@ -226,10 +232,7 @@ object DeltaStorageHandler { val DELTA_TABLE_PATH = "delta.table.path" /** - * A config we use to remember the table schema in the table properties. - * - * TODO Maybe Hive can tell us this in the `configureInputJobProperties` method. Then we don't - * need to store this extra information. + * The Hive table schema passing into `JobConf` so that `DeltaLog` can be accessed everywhere. */ val DELTA_TABLE_SCHEMA = "delta.table.schema" } From cb2bbf8df3d55f62aab7c55195cd36de2487580c Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 30 Mar 2020 10:49:27 -0700 Subject: [PATCH 026/291] Log Delta operation duration and the table stats (#32) Log Delta operation duration and the table stats so that we have a better understanding on how long it takes to load Delta metadata. --- .../apache/spark/sql/delta/DeltaHelper.scala | 46 ++++++++++++++++++- 1 file changed, 44 insertions(+), 2 deletions(-) diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 64b323719af..9ba709dd840 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -21,6 +21,7 @@ import java.util.Locale import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import io.delta.hive.{DeltaStorageHandler, PartitionColumnInfo} import org.apache.hadoop.fs._ @@ -46,6 +47,7 @@ object DeltaHelper extends Logging { def listDeltaFiles( nonNormalizedPath: Path, job: JobConf): (Array[FileStatus], Map[URI, Array[PartitionColumnInfo]]) = { + val loadStartMs = System.currentTimeMillis() val fs = nonNormalizedPath.getFileSystem(job) // We need to normalize the table path so that all paths we return to Hive will be normalized // This is necessary because `HiveInputFormat.pushProjectionsAndFilters` will try to figure out @@ -102,7 +104,6 @@ object DeltaHelper extends Logging { // Drop unused potential huge fields .map(add => add.copy(stats = null, tags = null))(SingleAction.addFileEncoder) .collect().map { f => - logInfo(s"selected delta file ${f.path} under $rootPath") val status = toFileStatus(fs, rootPath, f, blockSize) localFileToPartition += status.getPath.toUri -> partitionColumnWithIndex.map { case (t, index) => @@ -112,6 +113,10 @@ object DeltaHelper extends Logging { } status } + val loadEndMs = System.currentTimeMillis() + logOperationDuration("fetching file list", rootPath, snapshotToUse, loadEndMs - loadStartMs) + logInfo(s"Found ${files.size} files to process " + + s"in the Delta Lake table ${hideUserInfoInPath(rootPath)}") (files, localFileToPartition.toMap) } @@ -169,7 +174,11 @@ object DeltaHelper extends Logging { /** Load the latest Delta [[Snapshot]] from the path. */ def loadDeltaLatestSnapshot(rootPath: Path): Snapshot = { - DeltaLog.forTable(spark, rootPath).update() + val loadStartMs = System.currentTimeMillis() + val snapshot = DeltaLog.forTable(spark, rootPath).update() + val loadEndMs = System.currentTimeMillis() + logOperationDuration("loading snapshot", rootPath, snapshot, loadEndMs - loadStartMs) + snapshot } /** @@ -285,6 +294,39 @@ object DeltaHelper extends Logging { |Please update your Hive table's schema to match the Delta table schema.""".stripMargin) } + private def logOperationDuration( + ops: String, + path: Path, + snapshot: Snapshot, + durationMs: Long): Unit = { + logInfo(s"Delta Lake table '${hideUserInfoInPath(path)}' (" + + s"version: ${snapshot.version}, " + + s"size: ${snapshot.sizeInBytes}, " + + s"add: ${snapshot.numOfFiles}, " + + s"remove: ${snapshot.numOfRemoves}, " + + s"metadata: ${snapshot.numOfMetadata}, " + + s"protocol: ${snapshot.numOfProtocol}, " + + s"transactions: ${snapshot.numOfSetTransactions}, " + + s"partitions: ${snapshot.metadata.partitionColumns.mkString("[", ", ", "]")}" + + s") spent ${durationMs} ms on $ops.") + } + + /** Strip out user information to avoid printing credentials to logs. */ + private def hideUserInfoInPath(path: Path): Path = { + try { + val uri = path.toUri + val newUri = new URI(uri.getScheme, null, uri.getHost, uri.getPort, uri.getPath, + uri.getQuery, uri.getFragment) + new Path(newUri) + } catch { + case NonFatal(e) => + // This path may have illegal format, and we can not remove its user info and reassemble the + // uri. + logError("Path contains illegal format: " + path, e) + path + } + } + /** * Start a special Spark cluster using local mode to process Delta's metadata. The Spark UI has * been disabled and `SparkListener`s have been removed to reduce the memory usage of Spark. From 3645d9eaa101b521ae18a62727b3612993710806 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 06:48:44 -0700 Subject: [PATCH 027/291] update FAQ --- README.md | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 972dd4bbd66..a594798c950 100644 --- a/README.md +++ b/README.md @@ -84,10 +84,19 @@ LOCATION 's3://foo-bucket/bar-dir’ `io.delta.hive.DeltaStorageHandler` is the class that implements Hive data source APIs. It will know how to load a Delta table and extract its metadata. The table schema in the `CREATE TABLE` statement must be consistent with the underlying Delta metadata. Otherwise, the connector will throw an error to tell you about the inconsistency. -### FAQs +### Frequently asked questions (FAQ) #### Supported Hive versions -Hive 2.x. Please report any incompatible issues. +Hive 2.x. + +#### Can I use this connector in Apache Spark or Presto? +No. The connector **must** be used with Apache Hive. It doesn't work in other systems, such as Apache Spark or Presto. +- https://github.com/delta-io/delta/issues/85 is tracking the work to support Hive Metastore for Apache Spark. +- There is no native connector for Presto. But you can generate a manifest file to load a Delta table in Presto. See https://docs.delta.io/latest/presto-integration.html. +- Other system supoprt can be found in https://docs.delta.io/latest/integrations.html. + +#### If I create a table using the connector in Hive, can I query it in Apache Spark or Presto? +No. The table created by this connector in Hive cannot be read in any other systems right now. We recommend to create different tables in different systems but point to the same path. Although you need to use different table names to query the same Delta table, the underlying data will be shared by all of systems. #### Do I need to specify the partition columns when creating a Delta table? No. The partition columns are read from the underlying Delta metadata. The connector will know the partition columns and use this information to do the partition pruning automatically. From 965f3fd93ad1e1426b4c8fc8d13348840331e8c5 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 04:24:36 -0700 Subject: [PATCH 028/291] Save path in SerdeInfo's parameters (spark's data source table reads from it) --- hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 81d30caf6e7..8fdab3e948f 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -201,7 +201,7 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook .asInstanceOf[StructTypeInfo] DeltaHelper.checkTableSchema(snapshot.metadata.schema, hiveSchema) tbl.getParameters.put("spark.sql.sources.provider", "DELTA") - tbl.getSd.getParameters.put("path", deltaRootString) + tbl.getSd.getSerdeInfo.getParameters.put("path", deltaRootString) } override def rollbackCreateTable(table: Table): Unit = { From fd9ef41e954d142e1728b23906d4843c8f50a94b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 06:47:54 -0700 Subject: [PATCH 029/291] The query should fail when a path is deleted after a table is created --- .../io/delta/hive/DeltaStorageHandler.scala | 8 ++-- .../apache/spark/sql/delta/DeltaHelper.scala | 4 ++ .../io/delta/hive/HiveConnectorTest.scala | 46 +++++++++++++++---- 3 files changed, 46 insertions(+), 12 deletions(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 8fdab3e948f..c319e7e250a 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -73,7 +73,10 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook DataWritableReadSupport.getColumnTypes(tableProps.getProperty(IOConstants.COLUMNS_TYPES)) val hiveSchema = TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes) .asInstanceOf[StructTypeInfo] - jobProperties.put(DELTA_TABLE_PATH, tableProps.getProperty(META_TABLE_LOCATION)) + val rootPath = tableProps.getProperty(META_TABLE_LOCATION) + val snapshot = DeltaHelper.loadDeltaLatestSnapshot(new Path(rootPath)) + DeltaHelper.checkTableSchema(snapshot.metadata.schema, hiveSchema) + jobProperties.put(DELTA_TABLE_PATH, rootPath) jobProperties.put(DELTA_TABLE_SCHEMA, hiveSchema.toString) } @@ -184,9 +187,6 @@ class DeltaStorageHandler extends DefaultStorageHandler with HiveMetaHook } val snapshot = DeltaHelper.loadDeltaLatestSnapshot(new Path(deltaRootString)) - if (snapshot.version < 0) { - throw new MetaException(s"$deltaRootString does not exist or it's not a Delta table") - } // Extract the table schema in Hive to compare it with the latest table schema in Delta logs, // and fail the query if it was changed. diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 9ba709dd840..1080c3fa399 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -178,6 +178,10 @@ object DeltaHelper extends Logging { val snapshot = DeltaLog.forTable(spark, rootPath).update() val loadEndMs = System.currentTimeMillis() logOperationDuration("loading snapshot", rootPath, snapshot, loadEndMs - loadStartMs) + if (snapshot.version < 0) { + throw new MetaException( + s"${hideUserInfoInPath(rootPath)} does not exist or it's not a Delta table") + } snapshot } diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala index 7fdacfe96f6..634b258e2ea 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala @@ -571,14 +571,43 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { } } + test("fail the query when the path is deleted after the table is created") { + withTable("deltaTbl") { + withTempDir { dir => + val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) + + withSparkSession{ spark => + import spark.implicits._ + testData.toDS.toDF("c1", "c2").write.format("delta").save(dir.getCanonicalPath) + } + + runQuery( + s""" + |create external table deltaTbl(c1 int, c2 string) + |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + """.stripMargin + ) + + checkAnswer("select * from deltaTbl", testData) + + JavaUtils.deleteRecursively(dir) + + val e = intercept[Exception] { + checkAnswer("select * from deltaTbl", testData) + } + assert(e.getMessage.contains("not a Delta table")) + } + } + } + test("fail incorrect format config") { val formatKey = engine match { case "mr" => "hive.input.format" case "tez" => "hive.tez.input.format" case other => throw new UnsupportedOperationException(s"Unsupported engine: $other") } - withTable("deltaTbl") { - withTempDir { dir => + withTempDir { dir => + withTable("deltaTbl") { withSparkSession { spark => import spark.implicits._ val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) @@ -590,13 +619,14 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) |STORED BY 'io.delta.hive.DeltaStorageHandler' |LOCATION '${dir.getCanonicalPath}'""".stripMargin) - } - withHiveConf(formatKey, "org.apache.hadoop.hive.ql.io.HiveInputFormat") { - val e = intercept[Exception] { - runQuery("SELECT * from deltaTbl") + + withHiveConf(formatKey, "org.apache.hadoop.hive.ql.io.HiveInputFormat") { + val e = intercept[Exception] { + runQuery("SELECT * from deltaTbl") + } + assert(e.getMessage.contains(formatKey)) + assert(e.getMessage.contains(classOf[HiveInputFormat].getName)) } - assert(e.getMessage.contains(formatKey)) - assert(e.getMessage.contains(classOf[HiveInputFormat].getName)) } } } From 9bf77cbffdf57505c69c80ddc467b86a1ab8e399 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 07:07:28 -0700 Subject: [PATCH 030/291] set version to 0.1.0 --- build.sbt | 1 - version.sbt | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) create mode 100644 version.sbt diff --git a/build.sbt b/build.sbt index cf71edbf2c1..a86a74c468c 100644 --- a/build.sbt +++ b/build.sbt @@ -28,7 +28,6 @@ val hiveVersion = "2.3.3" val deltaVersion = "0.5.0" lazy val commonSettings = Seq( - version := deltaVersion, organization := "io.delta", scalaVersion := "2.12.8", fork := true, diff --git a/version.sbt b/version.sbt new file mode 100644 index 00000000000..e7654440516 --- /dev/null +++ b/version.sbt @@ -0,0 +1 @@ +version in ThisBuild := "0.1.0" From 16d3b5158a3c3879264da86775b375a57ae909d2 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 07:29:29 -0700 Subject: [PATCH 031/291] update README --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index a594798c950..ecdc5a2050a 100644 --- a/README.md +++ b/README.md @@ -29,8 +29,8 @@ This project contains all the code needed to make Hive read Delta Lake tables. The above commands will generate two JARs in the following paths. ``` -core/target/scala-2.12/delta-core-shaded-assembly_2.12-0.5.0.jar -hive/target/scala-2.12/hive-delta_2.12-0.5.0.jar +core/target/scala-2.12/delta-core-shaded-assembly_2.12-0.1.0.jar +hive/target/scala-2.12/hive-delta_2.12-0.1.0.jar ``` These two JARs include the Hive connector and all its dependencies. They need to be put in Hive’s classpath. @@ -38,8 +38,8 @@ These two JARs include the Hive connector and all its dependencies. They need to Note: if you would like to build jars using Scala 2.11, you can run the SBT command `build/sbt "++ 2.11.12 hive/package"` and the generated JARS will be in the following paths. ``` -core/target/scala-2.11/delta-core-shaded-assembly_2.11-0.5.0.jar -hive/target/scala-2.12/hive-delta_2.11-0.5.0.jar +core/target/scala-2.11/delta-core-shaded-assembly_2.11-0.1.0.jar +hive/target/scala-2.12/hive-delta_2.11-0.1.0.jar ``` ### Setting up Hive From 58768fa2f3d25a3dc90beeb1565c17f8955e1827 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 1 Apr 2020 08:08:46 -0700 Subject: [PATCH 032/291] Changed copyright in license headers based on linux foundation guidelines --- LICENSE.txt | 2 +- NOTICE.txt | 2 +- build.sbt | 2 +- build/sbt | 2 +- coreTest/src/main/resources/log4j.properties | 2 +- coreTest/src/main/scala/Test.scala | 2 +- .../scala/io/delta/hive/HiveMRSuite.scala | 2 +- .../scala/io/delta/hive/HiveTezSuite.scala | 2 +- .../java/io/delta/hive/DeltaInputSplit.java | 2 +- .../io/delta/hive/IndexPredicateAnalyzer.java | 24 ++++++++++++++++++- .../io/delta/hive/DeltaInputFormat.scala | 2 +- .../io/delta/hive/DeltaOutputFormat.scala | 2 +- .../delta/hive/DeltaRecordReaderWrapper.scala | 2 +- .../io/delta/hive/DeltaStorageHandler.scala | 2 +- .../scala/io/delta/hive/HiveInputFormat.scala | 2 +- .../io/delta/hive/PartitionColumnInfo.scala | 2 +- .../apache/spark/sql/delta/DeltaHelper.scala | 2 +- .../spark/sql/delta/DeltaPushFilter.scala | 2 +- hive/src/test/resources/log4j.properties | 2 +- .../io/delta/hive/HiveConnectorTest.scala | 2 +- .../scala/io/delta/hive/test/HiveTest.scala | 2 +- project/build.properties | 2 +- project/plugins.sbt | 2 +- scalastyle-config.xml | 4 ++-- 24 files changed, 47 insertions(+), 25 deletions(-) diff --git a/LICENSE.txt b/LICENSE.txt index 472fbf91414..480682c93c5 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -1,4 +1,4 @@ -Copyright 2019 Databricks, Inc. All rights reserved. +Copyright (2020) The Delta Lake Project Authors. All rights reserved. Apache License diff --git a/NOTICE.txt b/NOTICE.txt index f2744176c88..4fe11e3f96f 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Delta Lake Connectors -Copyright 2019 Databricks, Inc. +Copyright (2020) The Delta Lake Project Authors. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. diff --git a/build.sbt b/build.sbt index a86a74c468c..837ac3f0084 100644 --- a/build.sbt +++ b/build.sbt @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/build/sbt b/build/sbt index 53dc447fede..3a2a4e72222 100755 --- a/build/sbt +++ b/build/sbt @@ -23,7 +23,7 @@ # # -# Copyright 2019 Databricks, Inc. +# Copyright (2020) The Delta Lake Project Authors. # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at diff --git a/coreTest/src/main/resources/log4j.properties b/coreTest/src/main/resources/log4j.properties index 10887f221ce..ed93eca2537 100644 --- a/coreTest/src/main/resources/log4j.properties +++ b/coreTest/src/main/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright 2019 Databricks, Inc. +# Copyright (2020) The Delta Lake Project Authors. # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at diff --git a/coreTest/src/main/scala/Test.scala b/coreTest/src/main/scala/Test.scala index 479295ccdb3..c207bd714a0 100644 --- a/coreTest/src/main/scala/Test.scala +++ b/coreTest/src/main/scala/Test.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala index b82f1921251..10e637f14ff 100644 --- a/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala +++ b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala index a6fd1930453..37f78cac03a 100644 --- a/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala +++ b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java index 30ae177a379..38d3492df7b 100644 --- a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java +++ b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java index 46635b97f35..6777e0fca06 100644 --- a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java +++ b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,6 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +/* + * This file contains code from the Apache Spark project (original license above). + * It contains modifications, which are licensed as follows: + */ + +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.hive; import java.util.*; diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index a0361e7d6ff..845d8e9156f 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala index fa333fbd621..c5734c102a1 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala index 8054f909891..bc49ebb8685 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index c319e7e250a..34c6608b3dd 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala index ddcd1e3eaec..835c8d7a76a 100644 --- a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala index 8f8c6c293ad..b7367dd8451 100644 --- a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala +++ b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala index 1080c3fa399..f5090d14dd4 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala index b75c0f7c851..4369bd22a09 100644 --- a/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala +++ b/hive/src/main/scala/org/apache/spark/sql/delta/DeltaPushFilter.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/test/resources/log4j.properties b/hive/src/test/resources/log4j.properties index 10887f221ce..ed93eca2537 100644 --- a/hive/src/test/resources/log4j.properties +++ b/hive/src/test/resources/log4j.properties @@ -1,5 +1,5 @@ # -# Copyright 2019 Databricks, Inc. +# Copyright (2020) The Delta Lake Project Authors. # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala index 634b258e2ea..bda0baa91b6 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 38c406e7d55..7d69f1bb902 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/project/build.properties b/project/build.properties index d1e0fe6a892..a3c4896e6e7 100644 --- a/project/build.properties +++ b/project/build.properties @@ -21,7 +21,7 @@ # # -# Copyright 2019 Databricks, Inc. +# Copyright (2020) The Delta Lake Project Authors. # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at diff --git a/project/plugins.sbt b/project/plugins.sbt index 0f2b309dd45..1fd5033cf17 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,5 +1,5 @@ /* - * Copyright 2019 Databricks, Inc. + * Copyright (2020) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/scalastyle-config.xml b/scalastyle-config.xml index f88a3472897..bc5d4444b7c 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -19,7 +19,7 @@ ~It contains modifications, which are licensed as follows: --> + + + +All Classes (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

All Classes

+ + + diff --git a/docs/0.2.0/delta-standalone/api/java/allclasses-noframe.html b/docs/0.2.0/delta-standalone/api/java/allclasses-noframe.html new file mode 100644 index 00000000000..5558c8da999 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/allclasses-noframe.html @@ -0,0 +1,46 @@ + + + + + +All Classes (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

All Classes

+ + + diff --git a/docs/0.2.0/delta-standalone/api/java/constant-values.html b/docs/0.2.0/delta-standalone/api/java/constant-values.html new file mode 100644 index 00000000000..84691b5a5ae --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/constant-values.html @@ -0,0 +1,122 @@ + + + + + +Constant Field Values (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Constant Field Values

+

Contents

+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/deprecated-list.html b/docs/0.2.0/delta-standalone/api/java/deprecated-list.html new file mode 100644 index 00000000000..a5b8fd4d9a4 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/deprecated-list.html @@ -0,0 +1,122 @@ + + + + + +Deprecated List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

Deprecated API

+

Contents

+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.2.0/delta-standalone/api/java/help-doc.html b/docs/0.2.0/delta-standalone/api/java/help-doc.html new file mode 100644 index 00000000000..5472a288ff1 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/help-doc.html @@ -0,0 +1,223 @@ + + + + + +API Help (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

How This API Document Is Organized

+
This API (Application Programming Interface) document has pages corresponding to the items in the navigation bar, described as follows.
+
+
+
    +
  • +

    Overview

    +

    The Overview page is the front page of this API document and provides a list of all packages with a summary for each. This page can also contain an overall description of the set of packages.

    +
  • +
  • +

    Package

    +

    Each package has a page that contains a list of its classes and interfaces, with a summary for each. This page can contain six categories:

    +
      +
    • Interfaces (italic)
    • +
    • Classes
    • +
    • Enums
    • +
    • Exceptions
    • +
    • Errors
    • +
    • Annotation Types
    • +
    +
  • +
  • +

    Class/Interface

    +

    Each class, interface, nested class and nested interface has its own separate page. Each of these pages has three sections consisting of a class/interface description, summary tables, and detailed member descriptions:

    +
      +
    • Class inheritance diagram
    • +
    • Direct Subclasses
    • +
    • All Known Subinterfaces
    • +
    • All Known Implementing Classes
    • +
    • Class/interface declaration
    • +
    • Class/interface description
    • +
    +
      +
    • Nested Class Summary
    • +
    • Field Summary
    • +
    • Constructor Summary
    • +
    • Method Summary
    • +
    +
      +
    • Field Detail
    • +
    • Constructor Detail
    • +
    • Method Detail
    • +
    +

    Each summary entry contains the first sentence from the detailed description for that item. The summary entries are alphabetical, while the detailed descriptions are in the order they appear in the source code. This preserves the logical groupings established by the programmer.

    +
  • +
  • +

    Annotation Type

    +

    Each annotation type has its own separate page with the following sections:

    +
      +
    • Annotation Type declaration
    • +
    • Annotation Type description
    • +
    • Required Element Summary
    • +
    • Optional Element Summary
    • +
    • Element Detail
    • +
    +
  • +
  • +

    Enum

    +

    Each enum has its own separate page with the following sections:

    +
      +
    • Enum declaration
    • +
    • Enum description
    • +
    • Enum Constant Summary
    • +
    • Enum Constant Detail
    • +
    +
  • +
  • +

    Tree (Class Hierarchy)

    +

    There is a Class Hierarchy page for all packages, plus a hierarchy for each package. Each hierarchy page contains a list of classes and a list of interfaces. The classes are organized by inheritance structure starting with java.lang.Object. The interfaces do not inherit from java.lang.Object.

    +
      +
    • When viewing the Overview page, clicking on "Tree" displays the hierarchy for all packages.
    • +
    • When viewing a particular package, class or interface page, clicking "Tree" displays the hierarchy for only that package.
    • +
    +
  • +
  • +

    Deprecated API

    +

    The Deprecated API page lists all of the API that have been deprecated. A deprecated API is not recommended for use, generally due to improvements, and a replacement API is usually given. Deprecated APIs may be removed in future implementations.

    +
  • +
  • +

    Index

    +

    The Index contains an alphabetic list of all classes, interfaces, constructors, methods, and fields.

    +
  • +
  • +

    Prev/Next

    +

    These links take you to the next or previous class, interface, package, or related page.

    +
  • +
  • +

    Frames/No Frames

    +

    These links show and hide the HTML frames. All pages are available with or without frames.

    +
  • +
  • +

    All Classes

    +

    The All Classes link shows all classes and interfaces except non-static nested types.

    +
  • +
  • +

    Serialized Form

    +

    Each serializable or externalizable class has a description of its serialization fields and methods. This information is of interest to re-implementors, not to developers using the API. While there is no link in the navigation bar, you can get to this information by going to any serialized class and clicking "Serialized Form" in the "See also" section of the class description.

    +
  • +
  • +

    Constant Field Values

    +

    The Constant Field Values page lists the static final fields and their values.

    +
  • +
+This help file applies to API documentation generated using the standard doclet.
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.2.0/delta-standalone/api/java/index-all.html b/docs/0.2.0/delta-standalone/api/java/index-all.html new file mode 100644 index 00000000000..b9e778b127b --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/index-all.html @@ -0,0 +1,677 @@ + + + + + +Index (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
A B C D E F G H I J L M N O R S T U V  + + +

A

+
+
AddFile - Class in io.delta.standalone.actions
+
+
Represents an action that adds a new file to the table.
+
+
AddFile(String, Map<String, String>, long, long, boolean, String, Map<String, String>) - Constructor for class io.delta.standalone.actions.AddFile
+
 
+
ArrayType - Class in io.delta.standalone.types
+
+
The data type for collections of multiple values.
+
+
ArrayType(DataType, boolean) - Constructor for class io.delta.standalone.types.ArrayType
+
 
+
+ + + +

B

+
+
BinaryType - Class in io.delta.standalone.types
+
+
The data type representing byte[] values.
+
+
BinaryType() - Constructor for class io.delta.standalone.types.BinaryType
+
 
+
BooleanType - Class in io.delta.standalone.types
+
+
The data type representing boolean values.
+
+
BooleanType() - Constructor for class io.delta.standalone.types.BooleanType
+
 
+
ByteType - Class in io.delta.standalone.types
+
+
The data type representing byte values.
+
+
ByteType() - Constructor for class io.delta.standalone.types.ByteType
+
 
+
+ + + +

C

+
+
CloseableIterator<T> - Interface in io.delta.standalone.data
+
+
An Iterator that also need to implement the Closeable interface.
+
+
CommitInfo - Class in io.delta.standalone.actions
+
+
Holds provenance information about changes to the table.
+
+
CommitInfo(Optional<Long>, Timestamp, Optional<String>, Optional<String>, String, Map<String, String>, Optional<JobInfo>, Optional<NotebookInfo>, Optional<String>, Optional<Long>, Optional<String>, Optional<Boolean>, Optional<Map<String, String>>, Optional<String>) - Constructor for class io.delta.standalone.actions.CommitInfo
+
 
+
containsNull() - Method in class io.delta.standalone.types.ArrayType
+
 
+
+ + + +

D

+
+
DataType - Class in io.delta.standalone.types
+
+
The base type of all io.delta.standalone data types.
+
+
DataType() - Constructor for class io.delta.standalone.types.DataType
+
 
+
DateType - Class in io.delta.standalone.types
+
+
A date type, supporting "0001-01-01" through "9999-12-31".
+
+
DateType() - Constructor for class io.delta.standalone.types.DateType
+
 
+
DecimalType - Class in io.delta.standalone.types
+
+
The data type representing java.math.BigDecimal values.
+
+
DecimalType(int, int) - Constructor for class io.delta.standalone.types.DecimalType
+
 
+
DeltaLog - Interface in io.delta.standalone
+
+
DeltaLog is the representation of the transaction logs of a Delta table.
+
+
DoubleType - Class in io.delta.standalone.types
+
+
The data type representing double values.
+
+
DoubleType() - Constructor for class io.delta.standalone.types.DoubleType
+
 
+
+ + + +

E

+
+
equals(Object) - Method in class io.delta.standalone.actions.AddFile
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Format
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.JobInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Metadata
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.types.ArrayType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DataType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DecimalType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.MapType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructField
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructType
+
 
+
+ + + +

F

+
+
FloatType - Class in io.delta.standalone.types
+
+
The data type representing float values.
+
+
FloatType() - Constructor for class io.delta.standalone.types.FloatType
+
 
+
Format - Class in io.delta.standalone.actions
+
+
A specification of the encoding for the files stored in a table.
+
+
Format(String, Map<String, String>) - Constructor for class io.delta.standalone.actions.Format
+
 
+
forTable(Configuration, String) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provided path.
+
+
forTable(Configuration, Path) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provide path.
+
+
+ + + +

G

+
+
get(String) - Method in class io.delta.standalone.types.StructType
+
 
+
getAllFiles() - Method in interface io.delta.standalone.Snapshot
+
 
+
getBigDecimal(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.math.BigDecimal.
+
+
getBinary(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as binary (byte array).
+
+
getBoolean(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive boolean.
+
+
getByte(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive byte.
+
+
getCatalogString() - Method in class io.delta.standalone.types.DataType
+
 
+
getClusterId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getCommitInfoAt(long) - Method in interface io.delta.standalone.DeltaLog
+
 
+
getConfiguration() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getCreatedTime() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDataType() - Method in class io.delta.standalone.types.StructField
+
 
+
getDate(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Date.
+
+
getDescription() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDouble(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive double.
+
+
getElementType() - Method in class io.delta.standalone.types.ArrayType
+
 
+
getFieldNames() - Method in class io.delta.standalone.types.StructType
+
 
+
getFields() - Method in class io.delta.standalone.types.StructType
+
 
+
getFloat(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive float.
+
+
getFormat() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getId() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getInt(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive int.
+
+
getIsBlindAppend() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getIsolationLevel() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobName() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobOwnerId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getKeyType() - Method in class io.delta.standalone.types.MapType
+
 
+
getLength() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getList(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.List<T> object.
+
+
getLong(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive long.
+
+
getMap(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
+
+
getMetadata() - Method in interface io.delta.standalone.Snapshot
+
 
+
getModificationTime() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getName() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getName() - Method in class io.delta.standalone.types.StructField
+
 
+
getNotebookId() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
getNotebookInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperation() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationMetrics() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationParameters() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOptions() - Method in class io.delta.standalone.actions.Format
+
 
+
getPartitionColumns() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getPartitionValues() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPath() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPath() - Method in interface io.delta.standalone.DeltaLog
+
 
+
getPrecision() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getProvider() - Method in class io.delta.standalone.actions.Format
+
 
+
getReadVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getRecord(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a RowRecord object.
+
+
getRunId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getScale() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getSchema() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getSchema() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getShort(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive short.
+
+
getSimpleString() - Method in class io.delta.standalone.types.ByteType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.DataType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.IntegerType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.LongType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.ShortType
+
 
+
getSize() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getSnapshotForTimestampAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the latest Snapshot that was generated at or before timestamp.
+
+
getSnapshotForVersionAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the Snapshot with the provided version number.
+
+
getStats() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getString(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a String object.
+
+
getTags() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getTimestamp() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getTimestamp(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Timestamp.
+
+
getTreeString() - Method in class io.delta.standalone.types.StructType
+
 
+
getTriggerType() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getTypeName() - Method in class io.delta.standalone.types.DataType
+
 
+
getUserId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserMetadata() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserName() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getValueType() - Method in class io.delta.standalone.types.MapType
+
 
+
getVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getVersion() - Method in interface io.delta.standalone.Snapshot
+
 
+
+ + + +

H

+
+
hashCode() - Method in class io.delta.standalone.actions.AddFile
+
 
+
hashCode() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Format
+
 
+
hashCode() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Metadata
+
 
+
hashCode() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
hashCode() - Method in class io.delta.standalone.types.ArrayType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DataType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DecimalType
+
 
+
hashCode() - Method in class io.delta.standalone.types.MapType
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructField
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructType
+
 
+
+ + + +

I

+
+
IntegerType - Class in io.delta.standalone.types
+
+
The data type representing int values.
+
+
IntegerType() - Constructor for class io.delta.standalone.types.IntegerType
+
 
+
io.delta.standalone - package io.delta.standalone
+
 
+
io.delta.standalone.actions - package io.delta.standalone.actions
+
 
+
io.delta.standalone.data - package io.delta.standalone.data
+
 
+
io.delta.standalone.types - package io.delta.standalone.types
+
 
+
isDataChange() - Method in class io.delta.standalone.actions.AddFile
+
 
+
isNullable() - Method in class io.delta.standalone.types.StructField
+
 
+
isNullAt(String) - Method in interface io.delta.standalone.data.RowRecord
+
 
+
+ + + +

J

+
+
JobInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Job information that committed to the Delta table.
+
+
JobInfo(String, String, String, String, String) - Constructor for class io.delta.standalone.actions.JobInfo
+
 
+
+ + + +

L

+
+
LongType - Class in io.delta.standalone.types
+
+
The data type representing long values.
+
+
LongType() - Constructor for class io.delta.standalone.types.LongType
+
 
+
+ + + +

M

+
+
MapType - Class in io.delta.standalone.types
+
+
The data type for Maps.
+
+
MapType(DataType, DataType, boolean) - Constructor for class io.delta.standalone.types.MapType
+
 
+
Metadata - Class in io.delta.standalone.actions
+
+
Updates the metadata of the table.
+
+
Metadata(String, String, String, Format, List<String>, Map<String, String>, Optional<Long>, StructType) - Constructor for class io.delta.standalone.actions.Metadata
+
 
+
+ + + +

N

+
+
NotebookInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Notebook information that committed to the Delta table.
+
+
NotebookInfo(String) - Constructor for class io.delta.standalone.actions.NotebookInfo
+
 
+
NullType - Class in io.delta.standalone.types
+
+
The data type representing null values.
+
+
NullType() - Constructor for class io.delta.standalone.types.NullType
+
 
+
+ + + +

O

+
+
open() - Method in interface io.delta.standalone.Snapshot
+
+
Creates a CloseableIterator which can iterate over data belonging to this snapshot.
+
+
+ + + +

R

+
+
RowRecord - Interface in io.delta.standalone.data
+
+
Represents one row of data containing a non-empty collection of fieldName - value pairs.
+
+
+ + + +

S

+
+
ShortType - Class in io.delta.standalone.types
+
+
The data type representing short values.
+
+
ShortType() - Constructor for class io.delta.standalone.types.ShortType
+
 
+
snapshot() - Method in interface io.delta.standalone.DeltaLog
+
 
+
Snapshot - Interface in io.delta.standalone
+
+
Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version.
+
+
StringType - Class in io.delta.standalone.types
+
+
The data type representing String values.
+
+
StringType() - Constructor for class io.delta.standalone.types.StringType
+
 
+
StructField - Class in io.delta.standalone.types
+
+
A field inside a StructType.
+
+
StructField(String, DataType, boolean) - Constructor for class io.delta.standalone.types.StructField
+
 
+
StructField(String, DataType) - Constructor for class io.delta.standalone.types.StructField
+
+
Constructor with default nullable = true.
+
+
StructType - Class in io.delta.standalone.types
+
+
The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
+
+
StructType(StructField[]) - Constructor for class io.delta.standalone.types.StructType
+
 
+
+ + + +

T

+
+
TimestampType - Class in io.delta.standalone.types
+
+
The data type representing java.sql.Timestamp values.
+
+
TimestampType() - Constructor for class io.delta.standalone.types.TimestampType
+
 
+
+ + + +

U

+
+
update() - Method in interface io.delta.standalone.DeltaLog
+
+
Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
+
+
USER_DEFAULT - Static variable in class io.delta.standalone.types.DecimalType
+
 
+
+ + + +

V

+
+
valueContainsNull() - Method in class io.delta.standalone.types.MapType
+
 
+
+A B C D E F G H I J L M N O R S T U V 
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.2.0/delta-standalone/api/java/index.html b/docs/0.2.0/delta-standalone/api/java/index.html new file mode 100644 index 00000000000..ac45cef6309 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/index.html @@ -0,0 +1,75 @@ + + + + + +Delta Standalone Reader 0.2.1 JavaDoc + + + + + + + + + +<noscript> +<div>JavaScript is disabled on your browser.</div> +</noscript> +<h2>Frame Alert</h2> +<p>This document is designed to be viewed using the frames feature. If you see this message, you are using a non-frame-capable web client. Link to <a href="overview-summary.html">Non-frame version</a>.</p> + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/DeltaLog.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/DeltaLog.html new file mode 100644 index 00000000000..16f2bf1ab33 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/DeltaLog.html @@ -0,0 +1,366 @@ + + + + + +DeltaLog (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface DeltaLog

+
+
+
+
    +
  • +
    +
    +
    public interface DeltaLog
    +
    DeltaLog is the representation of the transaction logs of a Delta table. It provides APIs + to access the states of a Delta table. + + You can use the following codes to create a DeltaLog instance. +
    
    +   Configuration conf = ... // Create your own Hadoop Configuration instance
    +   DeltaLog deltaLog = DeltaLog.forTable(conf, "/the/delta/table/path");
    + 
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        update

        +
        Snapshot update()
        +
        Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
        +
      • +
      + + + +
        +
      • +

        getSnapshotForVersionAsOf

        +
        Snapshot getSnapshotForVersionAsOf(long version)
        +
        Travel back in time to the Snapshot with the provided version number.
        +
        +
        Parameters:
        +
        version - the snapshot version to generate
        +
        Throws:
        +
        IllegalArgumentException - if the version is outside the range of available versions
        +
        +
      • +
      + + + +
        +
      • +

        getSnapshotForTimestampAsOf

        +
        Snapshot getSnapshotForTimestampAsOf(long timestamp)
        +
        Travel back in time to the latest Snapshot that was generated at or before timestamp.
        +
        +
        Parameters:
        +
        timestamp - the number of milliseconds since midnight, January 1, 1970 UTC
        +
        Throws:
        +
        RuntimeException - if the snapshot is unable to be recreated
        +
        IllegalArgumentException - if the timestamp is before the earliest possible snapshot or after the latest possible snapshot
        +
        +
      • +
      + + + +
        +
      • +

        getCommitInfoAt

        +
        CommitInfo getCommitInfoAt(long version)
        +
        +
        Parameters:
        +
        version - the commit version to retrieve CommitInfo
        +
        +
      • +
      + + + +
        +
      • +

        getPath

        +
        org.apache.hadoop.fs.Path getPath()
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         String path)
        +
        Create a DeltaLog instance representing the table located at the provided path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         org.apache.hadoop.fs.Path path)
        +
        Create a DeltaLog instance representing the table located at the provide path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/Snapshot.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/Snapshot.html new file mode 100644 index 00000000000..60696ccc426 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/Snapshot.html @@ -0,0 +1,267 @@ + + + + + +Snapshot (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface Snapshot

+
+
+
+
    +
  • +
    +
    +
    public interface Snapshot
    +
    Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version. + + See Delta Transaction Log Protocol + for more details about the transaction logs.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getAllFiles

        +
        java.util.List<AddFile> getAllFiles()
        +
      • +
      + + + +
        +
      • +

        getMetadata

        +
        Metadata getMetadata()
        +
      • +
      + + + +
        +
      • +

        getVersion

        +
        long getVersion()
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html new file mode 100644 index 00000000000..6eefa24e5a3 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html @@ -0,0 +1,406 @@ + + + + + +AddFile (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class AddFile

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.AddFile
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class AddFile
    +extends Object
    +
    Represents an action that adds a new file to the table. The path of a file acts as the primary + key for the entry in the set of files. + + Note: since actions within a given Delta file are not guaranteed to be applied in order, it is + not valid for multiple file operations with the same path to exist in a single version.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      AddFile(String path, + java.util.Map<String,String> partitionValues, + long size, + long modificationTime, + boolean dataChange, + String stats, + java.util.Map<String,String> tags) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        AddFile

        +
        public AddFile(String path,
        +               java.util.Map<String,String> partitionValues,
        +               long size,
        +               long modificationTime,
        +               boolean dataChange,
        +               String stats,
        +               java.util.Map<String,String> tags)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        public String getPath()
        +
      • +
      + + + + + + + +
        +
      • +

        getSize

        +
        public long getSize()
        +
      • +
      + + + +
        +
      • +

        getModificationTime

        +
        public long getModificationTime()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        public boolean isDataChange()
        +
      • +
      + + + +
        +
      • +

        getStats

        +
        public String getStats()
        +
      • +
      + + + +
        +
      • +

        getTags

        +
        public java.util.Map<String,String> getTags()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html new file mode 100644 index 00000000000..e89ac395090 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html @@ -0,0 +1,505 @@ + + + + + +CommitInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class CommitInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.CommitInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class CommitInfo
    +extends Object
    +
    Holds provenance information about changes to the table. This CommitInfo + is not stored in the checkpoint and has reduced compatibility guarantees. + Information stored in it is best effort (i.e. can be falsified by a writer).
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      CommitInfo(java.util.Optional<Long> version, + java.sql.Timestamp timestamp, + java.util.Optional<String> userId, + java.util.Optional<String> userName, + String operation, + java.util.Map<String,String> operationParameters, + java.util.Optional<JobInfo> jobInfo, + java.util.Optional<NotebookInfo> notebookInfo, + java.util.Optional<String> clusterId, + java.util.Optional<Long> readVersion, + java.util.Optional<String> isolationLevel, + java.util.Optional<Boolean> isBlindAppend, + java.util.Optional<java.util.Map<String,String>> operationMetrics, + java.util.Optional<String> userMetadata) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        CommitInfo

        +
        public CommitInfo(java.util.Optional<Long> version,
        +                  java.sql.Timestamp timestamp,
        +                  java.util.Optional<String> userId,
        +                  java.util.Optional<String> userName,
        +                  String operation,
        +                  java.util.Map<String,String> operationParameters,
        +                  java.util.Optional<JobInfo> jobInfo,
        +                  java.util.Optional<NotebookInfo> notebookInfo,
        +                  java.util.Optional<String> clusterId,
        +                  java.util.Optional<Long> readVersion,
        +                  java.util.Optional<String> isolationLevel,
        +                  java.util.Optional<Boolean> isBlindAppend,
        +                  java.util.Optional<java.util.Map<String,String>> operationMetrics,
        +                  java.util.Optional<String> userMetadata)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getVersion

        +
        public java.util.Optional<Long> getVersion()
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        public java.sql.Timestamp getTimestamp()
        +
      • +
      + + + +
        +
      • +

        getUserId

        +
        public java.util.Optional<String> getUserId()
        +
      • +
      + + + +
        +
      • +

        getUserName

        +
        public java.util.Optional<String> getUserName()
        +
      • +
      + + + +
        +
      • +

        getOperation

        +
        public String getOperation()
        +
      • +
      + + + +
        +
      • +

        getOperationParameters

        +
        public java.util.Map<String,String> getOperationParameters()
        +
      • +
      + + + +
        +
      • +

        getJobInfo

        +
        public java.util.Optional<JobInfo> getJobInfo()
        +
      • +
      + + + +
        +
      • +

        getNotebookInfo

        +
        public java.util.Optional<NotebookInfo> getNotebookInfo()
        +
      • +
      + + + +
        +
      • +

        getClusterId

        +
        public java.util.Optional<String> getClusterId()
        +
      • +
      + + + +
        +
      • +

        getReadVersion

        +
        public java.util.Optional<Long> getReadVersion()
        +
      • +
      + + + +
        +
      • +

        getIsolationLevel

        +
        public java.util.Optional<String> getIsolationLevel()
        +
      • +
      + + + +
        +
      • +

        getIsBlindAppend

        +
        public java.util.Optional<Boolean> getIsBlindAppend()
        +
      • +
      + + + +
        +
      • +

        getOperationMetrics

        +
        public java.util.Optional<java.util.Map<String,String>> getOperationMetrics()
        +
      • +
      + + + +
        +
      • +

        getUserMetadata

        +
        public java.util.Optional<String> getUserMetadata()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Format.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Format.html new file mode 100644 index 00000000000..218bd9d1949 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Format.html @@ -0,0 +1,323 @@ + + + + + +Format (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Format

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Format
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Format
    +extends Object
    +
    A specification of the encoding for the files stored in a table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      Format(String provider, + java.util.Map<String,String> options) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Format

        +
        public Format(String provider,
        +              java.util.Map<String,String> options)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getProvider

        +
        public String getProvider()
        +
      • +
      + + + +
        +
      • +

        getOptions

        +
        public java.util.Map<String,String> getOptions()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html new file mode 100644 index 00000000000..d909febb3ff --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html @@ -0,0 +1,364 @@ + + + + + +JobInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class JobInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.JobInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class JobInfo
    +extends Object
    +
    Represents the Databricks Job information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      JobInfo(String jobId, + String jobName, + String runId, + String jobOwnerId, + String triggerType) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        JobInfo

        +
        public JobInfo(String jobId,
        +               String jobName,
        +               String runId,
        +               String jobOwnerId,
        +               String triggerType)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getJobId

        +
        public String getJobId()
        +
      • +
      + + + +
        +
      • +

        getJobName

        +
        public String getJobName()
        +
      • +
      + + + +
        +
      • +

        getRunId

        +
        public String getRunId()
        +
      • +
      + + + +
        +
      • +

        getJobOwnerId

        +
        public String getJobOwnerId()
        +
      • +
      + + + +
        +
      • +

        getTriggerType

        +
        public String getTriggerType()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html new file mode 100644 index 00000000000..b3c0521fc7b --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html @@ -0,0 +1,418 @@ + + + + + +Metadata (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Metadata

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Metadata
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Metadata
    +extends Object
    +
    Updates the metadata of the table. The first version of a table must contain + a Metadata action. Subsequent Metadata actions completely + overwrite the current metadata of the table. It is the responsibility of the + writer to ensure that any data already present in the table is still valid + after any change. There can be at most one Metadata action in a + given version of the table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Metadata

        +
        public Metadata(String id,
        +                String name,
        +                String description,
        +                Format format,
        +                java.util.List<String> partitionColumns,
        +                java.util.Map<String,String> configuration,
        +                java.util.Optional<Long> createdTime,
        +                StructType schema)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getId

        +
        public String getId()
        +
      • +
      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDescription

        +
        public String getDescription()
        +
      • +
      + + + +
        +
      • +

        getFormat

        +
        public Format getFormat()
        +
      • +
      + + + +
        +
      • +

        getPartitionColumns

        +
        public java.util.List<String> getPartitionColumns()
        +
      • +
      + + + +
        +
      • +

        getConfiguration

        +
        public java.util.Map<String,String> getConfiguration()
        +
      • +
      + + + +
        +
      • +

        getCreatedTime

        +
        public java.util.Optional<Long> getCreatedTime()
        +
      • +
      + + + +
        +
      • +

        getSchema

        +
        public StructType getSchema()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html new file mode 100644 index 00000000000..63d70d13a71 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html @@ -0,0 +1,304 @@ + + + + + +NotebookInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class NotebookInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.NotebookInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class NotebookInfo
    +extends Object
    +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      NotebookInfo(String notebookId) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NotebookInfo

        +
        public NotebookInfo(String notebookId)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getNotebookId

        +
        public String getNotebookId()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html new file mode 100644 index 00000000000..6ddc8240a59 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html @@ -0,0 +1,25 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.actions

+ + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html new file mode 100644 index 00000000000..4d51d96d560 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html @@ -0,0 +1,172 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.actions

+
+
+
    +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    AddFile +
    Represents an action that adds a new file to the table.
    +
    CommitInfo +
    Holds provenance information about changes to the table.
    +
    Format +
    A specification of the encoding for the files stored in a table.
    +
    JobInfo +
    Represents the Databricks Job information that committed to the Delta table.
    +
    Metadata +
    Updates the metadata of the table.
    +
    NotebookInfo +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html new file mode 100644 index 00000000000..8ae14c76b1e --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html @@ -0,0 +1,140 @@ + + + + + +io.delta.standalone.actions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.actions

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html new file mode 100644 index 00000000000..42f203a8c33 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html @@ -0,0 +1,200 @@ + + + + + +CloseableIterator (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface CloseableIterator<T>

+
+
+
+
    +
  • +
    +
    All Superinterfaces:
    +
    AutoCloseable, java.io.Closeable, java.util.Iterator<T>
    +
    +
    +
    +
    public interface CloseableIterator<T>
    +extends java.util.Iterator<T>, java.io.Closeable
    +
    An Iterator that also need to implement the Closeable interface. The caller + should call Closeable.close() method to free all resources properly after using the iterator.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from interface java.util.Iterator

        +forEachRemaining, hasNext, next, remove
      • +
      +
        +
      • + + +

        Methods inherited from interface java.io.Closeable

        +close
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html new file mode 100644 index 00000000000..1c6f2ea001e --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html @@ -0,0 +1,634 @@ + + + + + +RowRecord (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface RowRecord

+
+
+
+
    +
  • +
    +
    +
    public interface RowRecord
    +
    Represents one row of data containing a non-empty collection of fieldName - value pairs. + It provides APIs to allow retrieval of values through fieldName lookup. For example, + +
    
    +   if (row.isNullAt("int_field")) {
    +     // handle the null value.
    +   } else {
    +     int x = getInt("int_field");
    +   }
    + 
    +
    +
    See Also:
    +
    StructType, +StructField
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      All Methods Instance Methods Abstract Methods 
      Modifier and TypeMethod and Description
      java.math.BigDecimalgetBigDecimal(String fieldName) +
      Retrieves value from data record and returns the value as a java.math.BigDecimal.
      +
      byte[]getBinary(String fieldName) +
      Retrieves value from data record and returns the value as binary (byte array).
      +
      booleangetBoolean(String fieldName) +
      Retrieves value from data record and returns the value as a primitive boolean.
      +
      bytegetByte(String fieldName) +
      Retrieves value from data record and returns the value as a primitive byte.
      +
      java.sql.DategetDate(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Date.
      +
      doublegetDouble(String fieldName) +
      Retrieves value from data record and returns the value as a primitive double.
      +
      floatgetFloat(String fieldName) +
      Retrieves value from data record and returns the value as a primitive float.
      +
      intgetInt(String fieldName) +
      Retrieves value from data record and returns the value as a primitive int.
      +
      intgetLength() 
      <T> java.util.List<T>getList(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.List<T> object.
      +
      longgetLong(String fieldName) +
      Retrieves value from data record and returns the value as a primitive long.
      +
      <K,V> java.util.Map<K,V>getMap(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
      +
      RowRecordgetRecord(String fieldName) +
      Retrieves value from data record and returns the value as a RowRecord object.
      +
      StructTypegetSchema() 
      shortgetShort(String fieldName) +
      Retrieves value from data record and returns the value as a primitive short.
      +
      StringgetString(String fieldName) +
      Retrieves value from data record and returns the value as a String object.
      +
      java.sql.TimestampgetTimestamp(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Timestamp.
      +
      booleanisNullAt(String fieldName) 
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getLength

        +
        int getLength()
        +
      • +
      + + + +
        +
      • +

        isNullAt

        +
        boolean isNullAt(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        +
      • +
      + + + +
        +
      • +

        getInt

        +
        int getInt(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive int.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getLong

        +
        long getLong(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive long.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getByte

        +
        byte getByte(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive byte.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getShort

        +
        short getShort(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive short.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBoolean

        +
        boolean getBoolean(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive boolean.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getFloat

        +
        float getFloat(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive float.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDouble

        +
        double getDouble(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive double.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getString

        +
        String getString(String fieldName)
        +
        Retrieves value from data record and returns the value as a String object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBinary

        +
        byte[] getBinary(String fieldName)
        +
        Retrieves value from data record and returns the value as binary (byte array).
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBigDecimal

        +
        java.math.BigDecimal getBigDecimal(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.math.BigDecimal.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        java.sql.Timestamp getTimestamp(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Timestamp.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDate

        +
        java.sql.Date getDate(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Date.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getRecord

        +
        RowRecord getRecord(String fieldName)
        +
        Retrieves value from data record and returns the value as a RowRecord object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any nested field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getList

        +
        <T> java.util.List<T> getList(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.List<T> object.
        +
        +
        Type Parameters:
        +
        T - element type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any element field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getMap

        +
        <K,V> java.util.Map<K,V> getMap(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
        +
        +
        Type Parameters:
        +
        K - key type
        +
        V - value type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any key/value field, if that field is not + nullable and null data value read
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-frame.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-frame.html new file mode 100644 index 00000000000..044cd9dfddb --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-frame.html @@ -0,0 +1,21 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.data

+
+

Interfaces

+ +
+ + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-summary.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-summary.html new file mode 100644 index 00000000000..e32652d82ea --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-summary.html @@ -0,0 +1,148 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.data

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    CloseableIterator<T> +
    An Iterator that also need to implement the Closeable interface.
    +
    RowRecord +
    Represents one row of data containing a non-empty collection of fieldName - value pairs.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-tree.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-tree.html new file mode 100644 index 00000000000..16daf1856a5 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/data/package-tree.html @@ -0,0 +1,145 @@ + + + + + +io.delta.standalone.data Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.data

+Package Hierarchies: + +
+
+

Interface Hierarchy

+
    +
  • AutoCloseable +
      +
    • java.io.Closeable +
        +
      • io.delta.standalone.data.CloseableIterator<T> (also extends java.util.Iterator<E>)
      • +
      +
    • +
    +
  • +
  • java.util.Iterator<E> + +
  • +
  • io.delta.standalone.data.RowRecord
  • +
+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-frame.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-frame.html new file mode 100644 index 00000000000..71246a3e2e5 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-frame.html @@ -0,0 +1,21 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone

+
+

Interfaces

+ +
+ + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-summary.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-summary.html new file mode 100644 index 00000000000..947354b4a3a --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-summary.html @@ -0,0 +1,149 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    DeltaLog +
    DeltaLog is the representation of the transaction logs of a Delta table.
    +
    Snapshot +
    Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-tree.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-tree.html new file mode 100644 index 00000000000..5a1a8050147 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/package-tree.html @@ -0,0 +1,132 @@ + + + + + +io.delta.standalone Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone

+Package Hierarchies: + +
+
+

Interface Hierarchy

+ +
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html new file mode 100644 index 00000000000..63779f3deb8 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html @@ -0,0 +1,336 @@ + + + + + +ArrayType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ArrayType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ArrayType
    +extends DataType
    +
    The data type for collections of multiple values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ArrayType

        +
        public ArrayType(DataType elementType,
        +                 boolean containsNull)
        +
        +
        Parameters:
        +
        elementType - the data type of values
        +
        containsNull - indicates if values have null value
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getElementType

        +
        public DataType getElementType()
        +
      • +
      + + + +
        +
      • +

        containsNull

        +
        public boolean containsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html new file mode 100644 index 00000000000..866608eb730 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html @@ -0,0 +1,248 @@ + + + + + +BinaryType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BinaryType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BinaryType
    +extends DataType
    +
    The data type representing byte[] values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BinaryType

        +
        public BinaryType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html new file mode 100644 index 00000000000..edf7af06335 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html @@ -0,0 +1,248 @@ + + + + + +BooleanType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BooleanType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BooleanType
    +extends DataType
    +
    The data type representing boolean values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BooleanType

        +
        public BooleanType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ByteType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ByteType.html new file mode 100644 index 00000000000..5ad1ab2673e --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ByteType.html @@ -0,0 +1,286 @@ + + + + + +ByteType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ByteType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ByteType
    +extends DataType
    +
    The data type representing byte values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ByteType

        +
        public ByteType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DataType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DataType.html new file mode 100644 index 00000000000..69e6f370e1d --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DataType.html @@ -0,0 +1,337 @@ + + + + + +DataType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DataType

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.DataType
    • +
    +
  • +
+
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DataType

        +
        public DataType()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getTypeName

        +
        public String getTypeName()
        +
      • +
      + + + +
        +
      • +

        getSimpleString

        +
        public String getSimpleString()
        +
      • +
      + + + +
        +
      • +

        getCatalogString

        +
        public String getCatalogString()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DateType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DateType.html new file mode 100644 index 00000000000..2e5e0c2a1b5 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DateType.html @@ -0,0 +1,249 @@ + + + + + +DateType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DateType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DateType
    +extends DataType
    +
    A date type, supporting "0001-01-01" through "9999-12-31". + Internally, this is represented as the number of days from 1970-01-01.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DateType

        +
        public DateType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html new file mode 100644 index 00000000000..84eabab6941 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html @@ -0,0 +1,373 @@ + + + + + +DecimalType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DecimalType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DecimalType
    +extends DataType
    +
    The data type representing java.math.BigDecimal values. + A Decimal that must have fixed precision (the maximum number of digits) and scale (the number + of digits on right side of dot). + + The precision can be up to 38, scale can also be up to 38 (less or equal to precision). + + The default precision and scale is (10, 0).
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        USER_DEFAULT

        +
        public static final DecimalType USER_DEFAULT
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DecimalType

        +
        public DecimalType(int precision,
        +                   int scale)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPrecision

        +
        public int getPrecision()
        +
      • +
      + + + +
        +
      • +

        getScale

        +
        public int getScale()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html new file mode 100644 index 00000000000..437c2acae2e --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html @@ -0,0 +1,248 @@ + + + + + +DoubleType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DoubleType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DoubleType
    +extends DataType
    +
    The data type representing double values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DoubleType

        +
        public DoubleType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/FloatType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/FloatType.html new file mode 100644 index 00000000000..e35b4d7b127 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/FloatType.html @@ -0,0 +1,248 @@ + + + + + +FloatType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class FloatType

+
+
+ +
+
    +
  • +
    +
    +
    public final class FloatType
    +extends DataType
    +
    The data type representing float values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        FloatType

        +
        public FloatType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html new file mode 100644 index 00000000000..31a88fc17d4 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html @@ -0,0 +1,286 @@ + + + + + +IntegerType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class IntegerType

+
+
+ +
+
    +
  • +
    +
    +
    public final class IntegerType
    +extends DataType
    +
    The data type representing int values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        IntegerType

        +
        public IntegerType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/LongType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/LongType.html new file mode 100644 index 00000000000..75d8dcd0590 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/LongType.html @@ -0,0 +1,286 @@ + + + + + +LongType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class LongType

+
+
+ +
+
    +
  • +
    +
    +
    public final class LongType
    +extends DataType
    +
    The data type representing long values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        LongType

        +
        public LongType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/MapType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/MapType.html new file mode 100644 index 00000000000..c814ae9f046 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/MapType.html @@ -0,0 +1,352 @@ + + + + + +MapType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class MapType

+
+
+ +
+
    +
  • +
    +
    +
    public final class MapType
    +extends DataType
    +
    The data type for Maps. Keys in a map are not allowed to have null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        MapType

        +
        public MapType(DataType keyType,
        +               DataType valueType,
        +               boolean valueContainsNull)
        +
        +
        Parameters:
        +
        keyType - the data type of map keys
        +
        valueType - the data type of map values
        +
        valueContainsNull - indicates if map values have null values
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getKeyType

        +
        public DataType getKeyType()
        +
      • +
      + + + +
        +
      • +

        getValueType

        +
        public DataType getValueType()
        +
      • +
      + + + +
        +
      • +

        valueContainsNull

        +
        public boolean valueContainsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/NullType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/NullType.html new file mode 100644 index 00000000000..f8e8a8fae62 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/NullType.html @@ -0,0 +1,248 @@ + + + + + +NullType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class NullType

+
+
+ +
+
    +
  • +
    +
    +
    public final class NullType
    +extends DataType
    +
    The data type representing null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NullType

        +
        public NullType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ShortType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ShortType.html new file mode 100644 index 00000000000..aa995b02759 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/ShortType.html @@ -0,0 +1,286 @@ + + + + + +ShortType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ShortType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ShortType
    +extends DataType
    +
    The data type representing short values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ShortType

        +
        public ShortType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StringType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StringType.html new file mode 100644 index 00000000000..eab20c6cadd --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StringType.html @@ -0,0 +1,248 @@ + + + + + +StringType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StringType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StringType
    +extends DataType
    +
    The data type representing String values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StringType

        +
        public StringType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructField.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructField.html new file mode 100644 index 00000000000..38b613fe61c --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructField.html @@ -0,0 +1,362 @@ + + + + + +StructField (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructField

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.StructField
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class StructField
    +extends Object
    +
    A field inside a StructType.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType,
        +                   boolean nullable)
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        nullable - indicates if values of this field can be null values
        +
        +
      • +
      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType)
        +
        Constructor with default nullable = true.
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDataType

        +
        public DataType getDataType()
        +
      • +
      + + + +
        +
      • +

        isNullable

        +
        public boolean isNullable()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructType.html new file mode 100644 index 00000000000..861ffdd605b --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/StructType.html @@ -0,0 +1,366 @@ + + + + + +StructType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StructType
    +extends DataType
    +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    +
    See Also:
    +
    StructField
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructType

        +
        public StructType(StructField[] fields)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getFieldNames

        +
        public String[] getFieldNames()
        +
      • +
      + + + +
        +
      • +

        get

        +
        public StructField get(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - the name of the desired StructField, not null
        +
        Throws:
        +
        IllegalArgumentException - if a field with the given name does not exist
        +
        +
      • +
      + + + +
        +
      • +

        getTreeString

        +
        public String getTreeString()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html new file mode 100644 index 00000000000..49cb01e0ce7 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html @@ -0,0 +1,248 @@ + + + + + +TimestampType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class TimestampType

+
+
+ +
+
    +
  • +
    +
    +
    public final class TimestampType
    +extends DataType
    +
    The data type representing java.sql.Timestamp values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        TimestampType

        +
        public TimestampType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-frame.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-frame.html new file mode 100644 index 00000000000..e081f233f13 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-frame.html @@ -0,0 +1,37 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.types

+ + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-summary.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-summary.html new file mode 100644 index 00000000000..aead501f7f3 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-summary.html @@ -0,0 +1,245 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.types

+
+
+
    +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    ArrayType +
    The data type for collections of multiple values.
    +
    BinaryType +
    The data type representing byte[] values.
    +
    BooleanType +
    The data type representing boolean values.
    +
    ByteType +
    The data type representing byte values.
    +
    DataType +
    The base type of all io.delta.standalone data types.
    +
    DateType +
    A date type, supporting "0001-01-01" through "9999-12-31".
    +
    DecimalType +
    The data type representing java.math.BigDecimal values.
    +
    DoubleType +
    The data type representing double values.
    +
    FloatType +
    The data type representing float values.
    +
    IntegerType +
    The data type representing int values.
    +
    LongType +
    The data type representing long values.
    +
    MapType +
    The data type for Maps.
    +
    NullType +
    The data type representing null values.
    +
    ShortType +
    The data type representing short values.
    +
    StringType +
    The data type representing String values.
    +
    StructField +
    A field inside a StructType.
    +
    StructType +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    TimestampType +
    The data type representing java.sql.Timestamp values.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-tree.html b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-tree.html new file mode 100644 index 00000000000..4e52a7f4997 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/io/delta/standalone/types/package-tree.html @@ -0,0 +1,155 @@ + + + + + +io.delta.standalone.types Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.types

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/0.2.0/delta-standalone/api/java/overview-frame.html b/docs/0.2.0/delta-standalone/api/java/overview-frame.html new file mode 100644 index 00000000000..0187b771272 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/overview-frame.html @@ -0,0 +1,24 @@ + + + + + +Overview List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + +

 

+ + diff --git a/docs/0.2.0/delta-standalone/api/java/overview-summary.html b/docs/0.2.0/delta-standalone/api/java/overview-summary.html new file mode 100644 index 00000000000..69ddb947786 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/overview-summary.html @@ -0,0 +1,145 @@ + + + + + +Overview (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+ + + + + + + + + + + + + + + + + + + + + + + + +
Packages 
PackageDescription
io.delta.standalone 
io.delta.standalone.actions 
io.delta.standalone.data 
io.delta.standalone.types 
+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.2.0/delta-standalone/api/java/overview-tree.html b/docs/0.2.0/delta-standalone/api/java/overview-tree.html new file mode 100644 index 00000000000..1461dd63ed0 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/overview-tree.html @@ -0,0 +1,184 @@ + + + + + +Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

Hierarchy For All Packages

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +

Interface Hierarchy

+
    +
  • AutoCloseable +
      +
    • java.io.Closeable +
        +
      • io.delta.standalone.data.CloseableIterator<T> (also extends java.util.Iterator<E>)
      • +
      +
    • +
    +
  • +
  • io.delta.standalone.DeltaLog
  • +
  • java.util.Iterator<E> + +
  • +
  • io.delta.standalone.data.RowRecord
  • +
  • io.delta.standalone.Snapshot
  • +
+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.2.0/delta-standalone/api/java/package-list b/docs/0.2.0/delta-standalone/api/java/package-list new file mode 100644 index 00000000000..5dab2c7aadd --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/package-list @@ -0,0 +1,4 @@ +io.delta.standalone +io.delta.standalone.actions +io.delta.standalone.data +io.delta.standalone.types diff --git a/docs/0.2.0/delta-standalone/api/java/script.js b/docs/0.2.0/delta-standalone/api/java/script.js new file mode 100644 index 00000000000..b3463569314 --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/script.js @@ -0,0 +1,30 @@ +function show(type) +{ + count = 0; + for (var key in methods) { + var row = document.getElementById(key); + if ((methods[key] & type) != 0) { + row.style.display = ''; + row.className = (count++ % 2) ? rowColor : altColor; + } + else + row.style.display = 'none'; + } + updateTabs(type); +} + +function updateTabs(type) +{ + for (var value in tabs) { + var sNode = document.getElementById(tabs[value][0]); + var spanNode = sNode.firstChild; + if (value == type) { + sNode.className = activeTableTab; + spanNode.innerHTML = tabs[value][1]; + } + else { + sNode.className = tableTab; + spanNode.innerHTML = "" + tabs[value][1] + ""; + } + } +} diff --git a/docs/0.2.0/delta-standalone/api/java/stylesheet.css b/docs/0.2.0/delta-standalone/api/java/stylesheet.css new file mode 100644 index 00000000000..98055b22d6d --- /dev/null +++ b/docs/0.2.0/delta-standalone/api/java/stylesheet.css @@ -0,0 +1,574 @@ +/* Javadoc style sheet */ +/* +Overall document style +*/ + +@import url('resources/fonts/dejavu.css'); + +body { + background-color:#ffffff; + color:#353833; + font-family:'DejaVu Sans', Arial, Helvetica, sans-serif; + font-size:14px; + margin:0; +} +a:link, a:visited { + text-decoration:none; + color:#4A6782; +} +a:hover, a:focus { + text-decoration:none; + color:#bb7a2a; +} +a:active { + text-decoration:none; + color:#4A6782; +} +a[name] { + color:#353833; +} +a[name]:hover { + text-decoration:none; + color:#353833; +} +pre { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; +} +h1 { + font-size:20px; +} +h2 { + font-size:18px; +} +h3 { + font-size:16px; + font-style:italic; +} +h4 { + font-size:13px; +} +h5 { + font-size:12px; +} +h6 { + font-size:11px; +} +ul { + list-style-type:disc; +} +code, tt { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; + margin-top:8px; + line-height:1.4em; +} +dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; +} +table tr td dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + vertical-align:top; + padding-top:4px; +} +sup { + font-size:8px; +} +/* +Document title and Copyright styles +*/ +.clear { + clear:both; + height:0px; + overflow:hidden; +} +.aboutLanguage { + float:right; + padding:0px 21px; + font-size:11px; + z-index:200; + margin-top:-9px; +} +.legalCopy { + margin-left:.5em; +} +.bar a, .bar a:link, .bar a:visited, .bar a:active { + color:#FFFFFF; + text-decoration:none; +} +.bar a:hover, .bar a:focus { + color:#bb7a2a; +} +.tab { + background-color:#0066FF; + color:#ffffff; + padding:8px; + width:5em; + font-weight:bold; +} +/* +Navigation bar styles +*/ +.bar { + background-color:#4D7A97; + color:#FFFFFF; + padding:.8em .5em .4em .8em; + height:auto;/*height:1.8em;*/ + font-size:11px; + margin:0; +} +.topNav { + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.bottomNav { + margin-top:10px; + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.subNav { + background-color:#dee3e9; + float:left; + width:100%; + overflow:hidden; + font-size:12px; +} +.subNav div { + clear:left; + float:left; + padding:0 0 5px 6px; + text-transform:uppercase; +} +ul.navList, ul.subNavList { + float:left; + margin:0 25px 0 0; + padding:0; +} +ul.navList li{ + list-style:none; + float:left; + padding: 5px 6px; + text-transform:uppercase; +} +ul.subNavList li{ + list-style:none; + float:left; +} +.topNav a:link, .topNav a:active, .topNav a:visited, .bottomNav a:link, .bottomNav a:active, .bottomNav a:visited { + color:#FFFFFF; + text-decoration:none; + text-transform:uppercase; +} +.topNav a:hover, .bottomNav a:hover { + text-decoration:none; + color:#bb7a2a; + text-transform:uppercase; +} +.navBarCell1Rev { + background-color:#F8981D; + color:#253441; + margin: auto 5px; +} +.skipNav { + position:absolute; + top:auto; + left:-9999px; + overflow:hidden; +} +/* +Page header and footer styles +*/ +.header, .footer { + clear:both; + margin:0 20px; + padding:5px 0 0 0; +} +.indexHeader { + margin:10px; + position:relative; +} +.indexHeader span{ + margin-right:15px; +} +.indexHeader h1 { + font-size:13px; +} +.title { + color:#2c4557; + margin:10px 0; +} +.subTitle { + margin:5px 0 0 0; +} +.header ul { + margin:0 0 15px 0; + padding:0; +} +.footer ul { + margin:20px 0 5px 0; +} +.header ul li, .footer ul li { + list-style:none; + font-size:13px; +} +/* +Heading styles +*/ +div.details ul.blockList ul.blockList ul.blockList li.blockList h4, div.details ul.blockList ul.blockList ul.blockListLast li.blockList h4 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList ul.blockList li.blockList h3 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList li.blockList h3 { + padding:0; + margin:15px 0; +} +ul.blockList li.blockList h2 { + padding:0px 0 20px 0; +} +/* +Page layout container styles +*/ +.contentContainer, .sourceContainer, .classUseContainer, .serializedFormContainer, .constantValuesContainer { + clear:both; + padding:10px 20px; + position:relative; +} +.indexContainer { + margin:10px; + position:relative; + font-size:12px; +} +.indexContainer h2 { + font-size:13px; + padding:0 0 3px 0; +} +.indexContainer ul { + margin:0; + padding:0; +} +.indexContainer ul li { + list-style:none; + padding-top:2px; +} +.contentContainer .description dl dt, .contentContainer .details dl dt, .serializedFormContainer dl dt { + font-size:12px; + font-weight:bold; + margin:10px 0 0 0; + color:#4E4E4E; +} +.contentContainer .description dl dd, .contentContainer .details dl dd, .serializedFormContainer dl dd { + margin:5px 0 10px 0px; + font-size:14px; + font-family:'DejaVu Sans Mono',monospace; +} +.serializedFormContainer dl.nameValue dt { + margin-left:1px; + font-size:1.1em; + display:inline; + font-weight:bold; +} +.serializedFormContainer dl.nameValue dd { + margin:0 0 0 1px; + font-size:1.1em; + display:inline; +} +/* +List styles +*/ +ul.horizontal li { + display:inline; + font-size:0.9em; +} +ul.inheritance { + margin:0; + padding:0; +} +ul.inheritance li { + display:inline; + list-style:none; +} +ul.inheritance li ul.inheritance { + margin-left:15px; + padding-left:15px; + padding-top:1px; +} +ul.blockList, ul.blockListLast { + margin:10px 0 10px 0; + padding:0; +} +ul.blockList li.blockList, ul.blockListLast li.blockList { + list-style:none; + margin-bottom:15px; + line-height:1.4; +} +ul.blockList ul.blockList li.blockList, ul.blockList ul.blockListLast li.blockList { + padding:0px 20px 5px 10px; + border:1px solid #ededed; + background-color:#f8f8f8; +} +ul.blockList ul.blockList ul.blockList li.blockList, ul.blockList ul.blockList ul.blockListLast li.blockList { + padding:0 0 5px 8px; + background-color:#ffffff; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockList { + margin-left:0; + padding-left:0; + padding-bottom:15px; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockListLast { + list-style:none; + border-bottom:none; + padding-bottom:0; +} +table tr td dl, table tr td dl dt, table tr td dl dd { + margin-top:0; + margin-bottom:1px; +} +/* +Table styles +*/ +.overviewSummary, .memberSummary, .typeSummary, .useSummary, .constantsSummary, .deprecatedSummary { + width:100%; + border-left:1px solid #EEE; + border-right:1px solid #EEE; + border-bottom:1px solid #EEE; +} +.overviewSummary, .memberSummary { + padding:0px; +} +.overviewSummary caption, .memberSummary caption, .typeSummary caption, +.useSummary caption, .constantsSummary caption, .deprecatedSummary caption { + position:relative; + text-align:left; + background-repeat:no-repeat; + color:#253441; + font-weight:bold; + clear:none; + overflow:hidden; + padding:0px; + padding-top:10px; + padding-left:1px; + margin:0px; + white-space:pre; +} +.overviewSummary caption a:link, .memberSummary caption a:link, .typeSummary caption a:link, +.useSummary caption a:link, .constantsSummary caption a:link, .deprecatedSummary caption a:link, +.overviewSummary caption a:hover, .memberSummary caption a:hover, .typeSummary caption a:hover, +.useSummary caption a:hover, .constantsSummary caption a:hover, .deprecatedSummary caption a:hover, +.overviewSummary caption a:active, .memberSummary caption a:active, .typeSummary caption a:active, +.useSummary caption a:active, .constantsSummary caption a:active, .deprecatedSummary caption a:active, +.overviewSummary caption a:visited, .memberSummary caption a:visited, .typeSummary caption a:visited, +.useSummary caption a:visited, .constantsSummary caption a:visited, .deprecatedSummary caption a:visited { + color:#FFFFFF; +} +.overviewSummary caption span, .memberSummary caption span, .typeSummary caption span, +.useSummary caption span, .constantsSummary caption span, .deprecatedSummary caption span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + padding-bottom:7px; + display:inline-block; + float:left; + background-color:#F8981D; + border: none; + height:16px; +} +.memberSummary caption span.activeTableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#F8981D; + height:16px; +} +.memberSummary caption span.tableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#4D7A97; + height:16px; +} +.memberSummary caption span.tableTab, .memberSummary caption span.activeTableTab { + padding-top:0px; + padding-left:0px; + padding-right:0px; + background-image:none; + float:none; + display:inline; +} +.overviewSummary .tabEnd, .memberSummary .tabEnd, .typeSummary .tabEnd, +.useSummary .tabEnd, .constantsSummary .tabEnd, .deprecatedSummary .tabEnd { + display:none; + width:5px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .activeTableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .tableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + background-color:#4D7A97; + float:left; + +} +.overviewSummary td, .memberSummary td, .typeSummary td, +.useSummary td, .constantsSummary td, .deprecatedSummary td { + text-align:left; + padding:0px 0px 12px 10px; +} +th.colOne, th.colFirst, th.colLast, .useSummary th, .constantsSummary th, +td.colOne, td.colFirst, td.colLast, .useSummary td, .constantsSummary td{ + vertical-align:top; + padding-right:0px; + padding-top:8px; + padding-bottom:3px; +} +th.colFirst, th.colLast, th.colOne, .constantsSummary th { + background:#dee3e9; + text-align:left; + padding:8px 3px 3px 7px; +} +td.colFirst, th.colFirst { + white-space:nowrap; + font-size:13px; +} +td.colLast, th.colLast { + font-size:13px; +} +td.colOne, th.colOne { + font-size:13px; +} +.overviewSummary td.colFirst, .overviewSummary th.colFirst, +.useSummary td.colFirst, .useSummary th.colFirst, +.overviewSummary td.colOne, .overviewSummary th.colOne, +.memberSummary td.colFirst, .memberSummary th.colFirst, +.memberSummary td.colOne, .memberSummary th.colOne, +.typeSummary td.colFirst{ + width:25%; + vertical-align:top; +} +td.colOne a:link, td.colOne a:active, td.colOne a:visited, td.colOne a:hover, td.colFirst a:link, td.colFirst a:active, td.colFirst a:visited, td.colFirst a:hover, td.colLast a:link, td.colLast a:active, td.colLast a:visited, td.colLast a:hover, .constantValuesContainer td a:link, .constantValuesContainer td a:active, .constantValuesContainer td a:visited, .constantValuesContainer td a:hover { + font-weight:bold; +} +.tableSubHeadingColor { + background-color:#EEEEFF; +} +.altColor { + background-color:#FFFFFF; +} +.rowColor { + background-color:#EEEEEF; +} +/* +Content styles +*/ +.description pre { + margin-top:0; +} +.deprecatedContent { + margin:0; + padding:10px 0; +} +.docSummary { + padding:0; +} + +ul.blockList ul.blockList ul.blockList li.blockList h3 { + font-style:normal; +} + +div.block { + font-size:14px; + font-family:'DejaVu Serif', Georgia, "Times New Roman", Times, serif; +} + +td.colLast div { + padding-top:0px; +} + + +td.colLast a { + padding-bottom:3px; +} +/* +Formatting effect styles +*/ +.sourceLineNo { + color:green; + padding:0 30px 0 0; +} +h1.hidden { + visibility:hidden; + overflow:hidden; + font-size:10px; +} +.block { + display:block; + margin:3px 10px 2px 0px; + color:#474747; +} +.deprecatedLabel, .descfrmTypeLabel, .memberNameLabel, .memberNameLink, +.overrideSpecifyLabel, .packageHierarchyLabel, .paramLabel, .returnLabel, +.seeLabel, .simpleTagLabel, .throwsLabel, .typeNameLabel, .typeNameLink { + font-weight:bold; +} +.deprecationComment, .emphasizedPhrase, .interfaceName { + font-style:italic; +} + +div.block div.block span.deprecationComment, div.block div.block span.emphasizedPhrase, +div.block div.block span.interfaceName { + font-style:normal; +} + +div.contentContainer ul.blockList li.blockList h2{ + padding-bottom:0px; +} diff --git a/docs/latest/delta-standalone/api/java/allclasses-frame.html b/docs/latest/delta-standalone/api/java/allclasses-frame.html new file mode 100644 index 00000000000..c65c68fba35 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/allclasses-frame.html @@ -0,0 +1,46 @@ + + + + + +All Classes (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

All Classes

+ + + diff --git a/docs/latest/delta-standalone/api/java/allclasses-noframe.html b/docs/latest/delta-standalone/api/java/allclasses-noframe.html new file mode 100644 index 00000000000..5558c8da999 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/allclasses-noframe.html @@ -0,0 +1,46 @@ + + + + + +All Classes (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

All Classes

+ + + diff --git a/docs/latest/delta-standalone/api/java/constant-values.html b/docs/latest/delta-standalone/api/java/constant-values.html new file mode 100644 index 00000000000..84691b5a5ae --- /dev/null +++ b/docs/latest/delta-standalone/api/java/constant-values.html @@ -0,0 +1,122 @@ + + + + + +Constant Field Values (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Constant Field Values

+

Contents

+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/deprecated-list.html b/docs/latest/delta-standalone/api/java/deprecated-list.html new file mode 100644 index 00000000000..a5b8fd4d9a4 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/deprecated-list.html @@ -0,0 +1,122 @@ + + + + + +Deprecated List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

Deprecated API

+

Contents

+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/latest/delta-standalone/api/java/help-doc.html b/docs/latest/delta-standalone/api/java/help-doc.html new file mode 100644 index 00000000000..5472a288ff1 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/help-doc.html @@ -0,0 +1,223 @@ + + + + + +API Help (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

How This API Document Is Organized

+
This API (Application Programming Interface) document has pages corresponding to the items in the navigation bar, described as follows.
+
+
+
    +
  • +

    Overview

    +

    The Overview page is the front page of this API document and provides a list of all packages with a summary for each. This page can also contain an overall description of the set of packages.

    +
  • +
  • +

    Package

    +

    Each package has a page that contains a list of its classes and interfaces, with a summary for each. This page can contain six categories:

    +
      +
    • Interfaces (italic)
    • +
    • Classes
    • +
    • Enums
    • +
    • Exceptions
    • +
    • Errors
    • +
    • Annotation Types
    • +
    +
  • +
  • +

    Class/Interface

    +

    Each class, interface, nested class and nested interface has its own separate page. Each of these pages has three sections consisting of a class/interface description, summary tables, and detailed member descriptions:

    +
      +
    • Class inheritance diagram
    • +
    • Direct Subclasses
    • +
    • All Known Subinterfaces
    • +
    • All Known Implementing Classes
    • +
    • Class/interface declaration
    • +
    • Class/interface description
    • +
    +
      +
    • Nested Class Summary
    • +
    • Field Summary
    • +
    • Constructor Summary
    • +
    • Method Summary
    • +
    +
      +
    • Field Detail
    • +
    • Constructor Detail
    • +
    • Method Detail
    • +
    +

    Each summary entry contains the first sentence from the detailed description for that item. The summary entries are alphabetical, while the detailed descriptions are in the order they appear in the source code. This preserves the logical groupings established by the programmer.

    +
  • +
  • +

    Annotation Type

    +

    Each annotation type has its own separate page with the following sections:

    +
      +
    • Annotation Type declaration
    • +
    • Annotation Type description
    • +
    • Required Element Summary
    • +
    • Optional Element Summary
    • +
    • Element Detail
    • +
    +
  • +
  • +

    Enum

    +

    Each enum has its own separate page with the following sections:

    +
      +
    • Enum declaration
    • +
    • Enum description
    • +
    • Enum Constant Summary
    • +
    • Enum Constant Detail
    • +
    +
  • +
  • +

    Tree (Class Hierarchy)

    +

    There is a Class Hierarchy page for all packages, plus a hierarchy for each package. Each hierarchy page contains a list of classes and a list of interfaces. The classes are organized by inheritance structure starting with java.lang.Object. The interfaces do not inherit from java.lang.Object.

    +
      +
    • When viewing the Overview page, clicking on "Tree" displays the hierarchy for all packages.
    • +
    • When viewing a particular package, class or interface page, clicking "Tree" displays the hierarchy for only that package.
    • +
    +
  • +
  • +

    Deprecated API

    +

    The Deprecated API page lists all of the API that have been deprecated. A deprecated API is not recommended for use, generally due to improvements, and a replacement API is usually given. Deprecated APIs may be removed in future implementations.

    +
  • +
  • +

    Index

    +

    The Index contains an alphabetic list of all classes, interfaces, constructors, methods, and fields.

    +
  • +
  • +

    Prev/Next

    +

    These links take you to the next or previous class, interface, package, or related page.

    +
  • +
  • +

    Frames/No Frames

    +

    These links show and hide the HTML frames. All pages are available with or without frames.

    +
  • +
  • +

    All Classes

    +

    The All Classes link shows all classes and interfaces except non-static nested types.

    +
  • +
  • +

    Serialized Form

    +

    Each serializable or externalizable class has a description of its serialization fields and methods. This information is of interest to re-implementors, not to developers using the API. While there is no link in the navigation bar, you can get to this information by going to any serialized class and clicking "Serialized Form" in the "See also" section of the class description.

    +
  • +
  • +

    Constant Field Values

    +

    The Constant Field Values page lists the static final fields and their values.

    +
  • +
+This help file applies to API documentation generated using the standard doclet.
+ +
+ + + + + + + +
+ + + + diff --git a/docs/latest/delta-standalone/api/java/index-all.html b/docs/latest/delta-standalone/api/java/index-all.html new file mode 100644 index 00000000000..b9e778b127b --- /dev/null +++ b/docs/latest/delta-standalone/api/java/index-all.html @@ -0,0 +1,677 @@ + + + + + +Index (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
A B C D E F G H I J L M N O R S T U V  + + +

A

+
+
AddFile - Class in io.delta.standalone.actions
+
+
Represents an action that adds a new file to the table.
+
+
AddFile(String, Map<String, String>, long, long, boolean, String, Map<String, String>) - Constructor for class io.delta.standalone.actions.AddFile
+
 
+
ArrayType - Class in io.delta.standalone.types
+
+
The data type for collections of multiple values.
+
+
ArrayType(DataType, boolean) - Constructor for class io.delta.standalone.types.ArrayType
+
 
+
+ + + +

B

+
+
BinaryType - Class in io.delta.standalone.types
+
+
The data type representing byte[] values.
+
+
BinaryType() - Constructor for class io.delta.standalone.types.BinaryType
+
 
+
BooleanType - Class in io.delta.standalone.types
+
+
The data type representing boolean values.
+
+
BooleanType() - Constructor for class io.delta.standalone.types.BooleanType
+
 
+
ByteType - Class in io.delta.standalone.types
+
+
The data type representing byte values.
+
+
ByteType() - Constructor for class io.delta.standalone.types.ByteType
+
 
+
+ + + +

C

+
+
CloseableIterator<T> - Interface in io.delta.standalone.data
+
+
An Iterator that also need to implement the Closeable interface.
+
+
CommitInfo - Class in io.delta.standalone.actions
+
+
Holds provenance information about changes to the table.
+
+
CommitInfo(Optional<Long>, Timestamp, Optional<String>, Optional<String>, String, Map<String, String>, Optional<JobInfo>, Optional<NotebookInfo>, Optional<String>, Optional<Long>, Optional<String>, Optional<Boolean>, Optional<Map<String, String>>, Optional<String>) - Constructor for class io.delta.standalone.actions.CommitInfo
+
 
+
containsNull() - Method in class io.delta.standalone.types.ArrayType
+
 
+
+ + + +

D

+
+
DataType - Class in io.delta.standalone.types
+
+
The base type of all io.delta.standalone data types.
+
+
DataType() - Constructor for class io.delta.standalone.types.DataType
+
 
+
DateType - Class in io.delta.standalone.types
+
+
A date type, supporting "0001-01-01" through "9999-12-31".
+
+
DateType() - Constructor for class io.delta.standalone.types.DateType
+
 
+
DecimalType - Class in io.delta.standalone.types
+
+
The data type representing java.math.BigDecimal values.
+
+
DecimalType(int, int) - Constructor for class io.delta.standalone.types.DecimalType
+
 
+
DeltaLog - Interface in io.delta.standalone
+
+
DeltaLog is the representation of the transaction logs of a Delta table.
+
+
DoubleType - Class in io.delta.standalone.types
+
+
The data type representing double values.
+
+
DoubleType() - Constructor for class io.delta.standalone.types.DoubleType
+
 
+
+ + + +

E

+
+
equals(Object) - Method in class io.delta.standalone.actions.AddFile
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Format
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.JobInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Metadata
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.types.ArrayType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DataType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DecimalType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.MapType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructField
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructType
+
 
+
+ + + +

F

+
+
FloatType - Class in io.delta.standalone.types
+
+
The data type representing float values.
+
+
FloatType() - Constructor for class io.delta.standalone.types.FloatType
+
 
+
Format - Class in io.delta.standalone.actions
+
+
A specification of the encoding for the files stored in a table.
+
+
Format(String, Map<String, String>) - Constructor for class io.delta.standalone.actions.Format
+
 
+
forTable(Configuration, String) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provided path.
+
+
forTable(Configuration, Path) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provide path.
+
+
+ + + +

G

+
+
get(String) - Method in class io.delta.standalone.types.StructType
+
 
+
getAllFiles() - Method in interface io.delta.standalone.Snapshot
+
 
+
getBigDecimal(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.math.BigDecimal.
+
+
getBinary(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as binary (byte array).
+
+
getBoolean(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive boolean.
+
+
getByte(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive byte.
+
+
getCatalogString() - Method in class io.delta.standalone.types.DataType
+
 
+
getClusterId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getCommitInfoAt(long) - Method in interface io.delta.standalone.DeltaLog
+
 
+
getConfiguration() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getCreatedTime() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDataType() - Method in class io.delta.standalone.types.StructField
+
 
+
getDate(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Date.
+
+
getDescription() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDouble(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive double.
+
+
getElementType() - Method in class io.delta.standalone.types.ArrayType
+
 
+
getFieldNames() - Method in class io.delta.standalone.types.StructType
+
 
+
getFields() - Method in class io.delta.standalone.types.StructType
+
 
+
getFloat(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive float.
+
+
getFormat() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getId() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getInt(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive int.
+
+
getIsBlindAppend() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getIsolationLevel() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobName() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobOwnerId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getKeyType() - Method in class io.delta.standalone.types.MapType
+
 
+
getLength() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getList(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.List<T> object.
+
+
getLong(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive long.
+
+
getMap(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
+
+
getMetadata() - Method in interface io.delta.standalone.Snapshot
+
 
+
getModificationTime() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getName() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getName() - Method in class io.delta.standalone.types.StructField
+
 
+
getNotebookId() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
getNotebookInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperation() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationMetrics() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationParameters() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOptions() - Method in class io.delta.standalone.actions.Format
+
 
+
getPartitionColumns() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getPartitionValues() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPath() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPath() - Method in interface io.delta.standalone.DeltaLog
+
 
+
getPrecision() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getProvider() - Method in class io.delta.standalone.actions.Format
+
 
+
getReadVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getRecord(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a RowRecord object.
+
+
getRunId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getScale() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getSchema() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getSchema() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getShort(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive short.
+
+
getSimpleString() - Method in class io.delta.standalone.types.ByteType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.DataType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.IntegerType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.LongType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.ShortType
+
 
+
getSize() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getSnapshotForTimestampAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the latest Snapshot that was generated at or before timestamp.
+
+
getSnapshotForVersionAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the Snapshot with the provided version number.
+
+
getStats() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getString(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a String object.
+
+
getTags() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getTimestamp() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getTimestamp(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Timestamp.
+
+
getTreeString() - Method in class io.delta.standalone.types.StructType
+
 
+
getTriggerType() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getTypeName() - Method in class io.delta.standalone.types.DataType
+
 
+
getUserId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserMetadata() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserName() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getValueType() - Method in class io.delta.standalone.types.MapType
+
 
+
getVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getVersion() - Method in interface io.delta.standalone.Snapshot
+
 
+
+ + + +

H

+
+
hashCode() - Method in class io.delta.standalone.actions.AddFile
+
 
+
hashCode() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Format
+
 
+
hashCode() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Metadata
+
 
+
hashCode() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
hashCode() - Method in class io.delta.standalone.types.ArrayType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DataType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DecimalType
+
 
+
hashCode() - Method in class io.delta.standalone.types.MapType
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructField
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructType
+
 
+
+ + + +

I

+
+
IntegerType - Class in io.delta.standalone.types
+
+
The data type representing int values.
+
+
IntegerType() - Constructor for class io.delta.standalone.types.IntegerType
+
 
+
io.delta.standalone - package io.delta.standalone
+
 
+
io.delta.standalone.actions - package io.delta.standalone.actions
+
 
+
io.delta.standalone.data - package io.delta.standalone.data
+
 
+
io.delta.standalone.types - package io.delta.standalone.types
+
 
+
isDataChange() - Method in class io.delta.standalone.actions.AddFile
+
 
+
isNullable() - Method in class io.delta.standalone.types.StructField
+
 
+
isNullAt(String) - Method in interface io.delta.standalone.data.RowRecord
+
 
+
+ + + +

J

+
+
JobInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Job information that committed to the Delta table.
+
+
JobInfo(String, String, String, String, String) - Constructor for class io.delta.standalone.actions.JobInfo
+
 
+
+ + + +

L

+
+
LongType - Class in io.delta.standalone.types
+
+
The data type representing long values.
+
+
LongType() - Constructor for class io.delta.standalone.types.LongType
+
 
+
+ + + +

M

+
+
MapType - Class in io.delta.standalone.types
+
+
The data type for Maps.
+
+
MapType(DataType, DataType, boolean) - Constructor for class io.delta.standalone.types.MapType
+
 
+
Metadata - Class in io.delta.standalone.actions
+
+
Updates the metadata of the table.
+
+
Metadata(String, String, String, Format, List<String>, Map<String, String>, Optional<Long>, StructType) - Constructor for class io.delta.standalone.actions.Metadata
+
 
+
+ + + +

N

+
+
NotebookInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Notebook information that committed to the Delta table.
+
+
NotebookInfo(String) - Constructor for class io.delta.standalone.actions.NotebookInfo
+
 
+
NullType - Class in io.delta.standalone.types
+
+
The data type representing null values.
+
+
NullType() - Constructor for class io.delta.standalone.types.NullType
+
 
+
+ + + +

O

+
+
open() - Method in interface io.delta.standalone.Snapshot
+
+
Creates a CloseableIterator which can iterate over data belonging to this snapshot.
+
+
+ + + +

R

+
+
RowRecord - Interface in io.delta.standalone.data
+
+
Represents one row of data containing a non-empty collection of fieldName - value pairs.
+
+
+ + + +

S

+
+
ShortType - Class in io.delta.standalone.types
+
+
The data type representing short values.
+
+
ShortType() - Constructor for class io.delta.standalone.types.ShortType
+
 
+
snapshot() - Method in interface io.delta.standalone.DeltaLog
+
 
+
Snapshot - Interface in io.delta.standalone
+
+
Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version.
+
+
StringType - Class in io.delta.standalone.types
+
+
The data type representing String values.
+
+
StringType() - Constructor for class io.delta.standalone.types.StringType
+
 
+
StructField - Class in io.delta.standalone.types
+
+
A field inside a StructType.
+
+
StructField(String, DataType, boolean) - Constructor for class io.delta.standalone.types.StructField
+
 
+
StructField(String, DataType) - Constructor for class io.delta.standalone.types.StructField
+
+
Constructor with default nullable = true.
+
+
StructType - Class in io.delta.standalone.types
+
+
The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
+
+
StructType(StructField[]) - Constructor for class io.delta.standalone.types.StructType
+
 
+
+ + + +

T

+
+
TimestampType - Class in io.delta.standalone.types
+
+
The data type representing java.sql.Timestamp values.
+
+
TimestampType() - Constructor for class io.delta.standalone.types.TimestampType
+
 
+
+ + + +

U

+
+
update() - Method in interface io.delta.standalone.DeltaLog
+
+
Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
+
+
USER_DEFAULT - Static variable in class io.delta.standalone.types.DecimalType
+
 
+
+ + + +

V

+
+
valueContainsNull() - Method in class io.delta.standalone.types.MapType
+
 
+
+A B C D E F G H I J L M N O R S T U V 
+ +
+ + + + + + + +
+ + + + diff --git a/docs/latest/delta-standalone/api/java/index.html b/docs/latest/delta-standalone/api/java/index.html new file mode 100644 index 00000000000..ac45cef6309 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/index.html @@ -0,0 +1,75 @@ + + + + + +Delta Standalone Reader 0.2.1 JavaDoc + + + + + + + + + +<noscript> +<div>JavaScript is disabled on your browser.</div> +</noscript> +<h2>Frame Alert</h2> +<p>This document is designed to be viewed using the frames feature. If you see this message, you are using a non-frame-capable web client. Link to <a href="overview-summary.html">Non-frame version</a>.</p> + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/DeltaLog.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/DeltaLog.html new file mode 100644 index 00000000000..16f2bf1ab33 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/DeltaLog.html @@ -0,0 +1,366 @@ + + + + + +DeltaLog (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface DeltaLog

+
+
+
+
    +
  • +
    +
    +
    public interface DeltaLog
    +
    DeltaLog is the representation of the transaction logs of a Delta table. It provides APIs + to access the states of a Delta table. + + You can use the following codes to create a DeltaLog instance. +
    
    +   Configuration conf = ... // Create your own Hadoop Configuration instance
    +   DeltaLog deltaLog = DeltaLog.forTable(conf, "/the/delta/table/path");
    + 
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        update

        +
        Snapshot update()
        +
        Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
        +
      • +
      + + + +
        +
      • +

        getSnapshotForVersionAsOf

        +
        Snapshot getSnapshotForVersionAsOf(long version)
        +
        Travel back in time to the Snapshot with the provided version number.
        +
        +
        Parameters:
        +
        version - the snapshot version to generate
        +
        Throws:
        +
        IllegalArgumentException - if the version is outside the range of available versions
        +
        +
      • +
      + + + +
        +
      • +

        getSnapshotForTimestampAsOf

        +
        Snapshot getSnapshotForTimestampAsOf(long timestamp)
        +
        Travel back in time to the latest Snapshot that was generated at or before timestamp.
        +
        +
        Parameters:
        +
        timestamp - the number of milliseconds since midnight, January 1, 1970 UTC
        +
        Throws:
        +
        RuntimeException - if the snapshot is unable to be recreated
        +
        IllegalArgumentException - if the timestamp is before the earliest possible snapshot or after the latest possible snapshot
        +
        +
      • +
      + + + +
        +
      • +

        getCommitInfoAt

        +
        CommitInfo getCommitInfoAt(long version)
        +
        +
        Parameters:
        +
        version - the commit version to retrieve CommitInfo
        +
        +
      • +
      + + + +
        +
      • +

        getPath

        +
        org.apache.hadoop.fs.Path getPath()
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         String path)
        +
        Create a DeltaLog instance representing the table located at the provided path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         org.apache.hadoop.fs.Path path)
        +
        Create a DeltaLog instance representing the table located at the provide path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/Snapshot.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/Snapshot.html new file mode 100644 index 00000000000..60696ccc426 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/Snapshot.html @@ -0,0 +1,267 @@ + + + + + +Snapshot (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface Snapshot

+
+
+
+
    +
  • +
    +
    +
    public interface Snapshot
    +
    Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version. + + See Delta Transaction Log Protocol + for more details about the transaction logs.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getAllFiles

        +
        java.util.List<AddFile> getAllFiles()
        +
      • +
      + + + +
        +
      • +

        getMetadata

        +
        Metadata getMetadata()
        +
      • +
      + + + +
        +
      • +

        getVersion

        +
        long getVersion()
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html new file mode 100644 index 00000000000..6eefa24e5a3 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html @@ -0,0 +1,406 @@ + + + + + +AddFile (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class AddFile

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.AddFile
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class AddFile
    +extends Object
    +
    Represents an action that adds a new file to the table. The path of a file acts as the primary + key for the entry in the set of files. + + Note: since actions within a given Delta file are not guaranteed to be applied in order, it is + not valid for multiple file operations with the same path to exist in a single version.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      AddFile(String path, + java.util.Map<String,String> partitionValues, + long size, + long modificationTime, + boolean dataChange, + String stats, + java.util.Map<String,String> tags) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        AddFile

        +
        public AddFile(String path,
        +               java.util.Map<String,String> partitionValues,
        +               long size,
        +               long modificationTime,
        +               boolean dataChange,
        +               String stats,
        +               java.util.Map<String,String> tags)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        public String getPath()
        +
      • +
      + + + + + + + +
        +
      • +

        getSize

        +
        public long getSize()
        +
      • +
      + + + +
        +
      • +

        getModificationTime

        +
        public long getModificationTime()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        public boolean isDataChange()
        +
      • +
      + + + +
        +
      • +

        getStats

        +
        public String getStats()
        +
      • +
      + + + +
        +
      • +

        getTags

        +
        public java.util.Map<String,String> getTags()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html new file mode 100644 index 00000000000..e89ac395090 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html @@ -0,0 +1,505 @@ + + + + + +CommitInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class CommitInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.CommitInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class CommitInfo
    +extends Object
    +
    Holds provenance information about changes to the table. This CommitInfo + is not stored in the checkpoint and has reduced compatibility guarantees. + Information stored in it is best effort (i.e. can be falsified by a writer).
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      CommitInfo(java.util.Optional<Long> version, + java.sql.Timestamp timestamp, + java.util.Optional<String> userId, + java.util.Optional<String> userName, + String operation, + java.util.Map<String,String> operationParameters, + java.util.Optional<JobInfo> jobInfo, + java.util.Optional<NotebookInfo> notebookInfo, + java.util.Optional<String> clusterId, + java.util.Optional<Long> readVersion, + java.util.Optional<String> isolationLevel, + java.util.Optional<Boolean> isBlindAppend, + java.util.Optional<java.util.Map<String,String>> operationMetrics, + java.util.Optional<String> userMetadata) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        CommitInfo

        +
        public CommitInfo(java.util.Optional<Long> version,
        +                  java.sql.Timestamp timestamp,
        +                  java.util.Optional<String> userId,
        +                  java.util.Optional<String> userName,
        +                  String operation,
        +                  java.util.Map<String,String> operationParameters,
        +                  java.util.Optional<JobInfo> jobInfo,
        +                  java.util.Optional<NotebookInfo> notebookInfo,
        +                  java.util.Optional<String> clusterId,
        +                  java.util.Optional<Long> readVersion,
        +                  java.util.Optional<String> isolationLevel,
        +                  java.util.Optional<Boolean> isBlindAppend,
        +                  java.util.Optional<java.util.Map<String,String>> operationMetrics,
        +                  java.util.Optional<String> userMetadata)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getVersion

        +
        public java.util.Optional<Long> getVersion()
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        public java.sql.Timestamp getTimestamp()
        +
      • +
      + + + +
        +
      • +

        getUserId

        +
        public java.util.Optional<String> getUserId()
        +
      • +
      + + + +
        +
      • +

        getUserName

        +
        public java.util.Optional<String> getUserName()
        +
      • +
      + + + +
        +
      • +

        getOperation

        +
        public String getOperation()
        +
      • +
      + + + +
        +
      • +

        getOperationParameters

        +
        public java.util.Map<String,String> getOperationParameters()
        +
      • +
      + + + +
        +
      • +

        getJobInfo

        +
        public java.util.Optional<JobInfo> getJobInfo()
        +
      • +
      + + + +
        +
      • +

        getNotebookInfo

        +
        public java.util.Optional<NotebookInfo> getNotebookInfo()
        +
      • +
      + + + +
        +
      • +

        getClusterId

        +
        public java.util.Optional<String> getClusterId()
        +
      • +
      + + + +
        +
      • +

        getReadVersion

        +
        public java.util.Optional<Long> getReadVersion()
        +
      • +
      + + + +
        +
      • +

        getIsolationLevel

        +
        public java.util.Optional<String> getIsolationLevel()
        +
      • +
      + + + +
        +
      • +

        getIsBlindAppend

        +
        public java.util.Optional<Boolean> getIsBlindAppend()
        +
      • +
      + + + +
        +
      • +

        getOperationMetrics

        +
        public java.util.Optional<java.util.Map<String,String>> getOperationMetrics()
        +
      • +
      + + + +
        +
      • +

        getUserMetadata

        +
        public java.util.Optional<String> getUserMetadata()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Format.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Format.html new file mode 100644 index 00000000000..218bd9d1949 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Format.html @@ -0,0 +1,323 @@ + + + + + +Format (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Format

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Format
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Format
    +extends Object
    +
    A specification of the encoding for the files stored in a table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      Format(String provider, + java.util.Map<String,String> options) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Format

        +
        public Format(String provider,
        +              java.util.Map<String,String> options)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getProvider

        +
        public String getProvider()
        +
      • +
      + + + +
        +
      • +

        getOptions

        +
        public java.util.Map<String,String> getOptions()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html new file mode 100644 index 00000000000..d909febb3ff --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html @@ -0,0 +1,364 @@ + + + + + +JobInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class JobInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.JobInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class JobInfo
    +extends Object
    +
    Represents the Databricks Job information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      JobInfo(String jobId, + String jobName, + String runId, + String jobOwnerId, + String triggerType) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        JobInfo

        +
        public JobInfo(String jobId,
        +               String jobName,
        +               String runId,
        +               String jobOwnerId,
        +               String triggerType)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getJobId

        +
        public String getJobId()
        +
      • +
      + + + +
        +
      • +

        getJobName

        +
        public String getJobName()
        +
      • +
      + + + +
        +
      • +

        getRunId

        +
        public String getRunId()
        +
      • +
      + + + +
        +
      • +

        getJobOwnerId

        +
        public String getJobOwnerId()
        +
      • +
      + + + +
        +
      • +

        getTriggerType

        +
        public String getTriggerType()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html new file mode 100644 index 00000000000..b3c0521fc7b --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html @@ -0,0 +1,418 @@ + + + + + +Metadata (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Metadata

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Metadata
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Metadata
    +extends Object
    +
    Updates the metadata of the table. The first version of a table must contain + a Metadata action. Subsequent Metadata actions completely + overwrite the current metadata of the table. It is the responsibility of the + writer to ensure that any data already present in the table is still valid + after any change. There can be at most one Metadata action in a + given version of the table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Metadata

        +
        public Metadata(String id,
        +                String name,
        +                String description,
        +                Format format,
        +                java.util.List<String> partitionColumns,
        +                java.util.Map<String,String> configuration,
        +                java.util.Optional<Long> createdTime,
        +                StructType schema)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getId

        +
        public String getId()
        +
      • +
      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDescription

        +
        public String getDescription()
        +
      • +
      + + + +
        +
      • +

        getFormat

        +
        public Format getFormat()
        +
      • +
      + + + +
        +
      • +

        getPartitionColumns

        +
        public java.util.List<String> getPartitionColumns()
        +
      • +
      + + + +
        +
      • +

        getConfiguration

        +
        public java.util.Map<String,String> getConfiguration()
        +
      • +
      + + + +
        +
      • +

        getCreatedTime

        +
        public java.util.Optional<Long> getCreatedTime()
        +
      • +
      + + + +
        +
      • +

        getSchema

        +
        public StructType getSchema()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html new file mode 100644 index 00000000000..63d70d13a71 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html @@ -0,0 +1,304 @@ + + + + + +NotebookInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class NotebookInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.NotebookInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class NotebookInfo
    +extends Object
    +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      NotebookInfo(String notebookId) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NotebookInfo

        +
        public NotebookInfo(String notebookId)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getNotebookId

        +
        public String getNotebookId()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html new file mode 100644 index 00000000000..6ddc8240a59 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html @@ -0,0 +1,25 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.actions

+ + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html new file mode 100644 index 00000000000..4d51d96d560 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html @@ -0,0 +1,172 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.actions

+
+
+
    +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    AddFile +
    Represents an action that adds a new file to the table.
    +
    CommitInfo +
    Holds provenance information about changes to the table.
    +
    Format +
    A specification of the encoding for the files stored in a table.
    +
    JobInfo +
    Represents the Databricks Job information that committed to the Delta table.
    +
    Metadata +
    Updates the metadata of the table.
    +
    NotebookInfo +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html new file mode 100644 index 00000000000..8ae14c76b1e --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html @@ -0,0 +1,140 @@ + + + + + +io.delta.standalone.actions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.actions

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html new file mode 100644 index 00000000000..42f203a8c33 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html @@ -0,0 +1,200 @@ + + + + + +CloseableIterator (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface CloseableIterator<T>

+
+
+
+
    +
  • +
    +
    All Superinterfaces:
    +
    AutoCloseable, java.io.Closeable, java.util.Iterator<T>
    +
    +
    +
    +
    public interface CloseableIterator<T>
    +extends java.util.Iterator<T>, java.io.Closeable
    +
    An Iterator that also need to implement the Closeable interface. The caller + should call Closeable.close() method to free all resources properly after using the iterator.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from interface java.util.Iterator

        +forEachRemaining, hasNext, next, remove
      • +
      +
        +
      • + + +

        Methods inherited from interface java.io.Closeable

        +close
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html new file mode 100644 index 00000000000..1c6f2ea001e --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html @@ -0,0 +1,634 @@ + + + + + +RowRecord (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface RowRecord

+
+
+
+
    +
  • +
    +
    +
    public interface RowRecord
    +
    Represents one row of data containing a non-empty collection of fieldName - value pairs. + It provides APIs to allow retrieval of values through fieldName lookup. For example, + +
    
    +   if (row.isNullAt("int_field")) {
    +     // handle the null value.
    +   } else {
    +     int x = getInt("int_field");
    +   }
    + 
    +
    +
    See Also:
    +
    StructType, +StructField
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      All Methods Instance Methods Abstract Methods 
      Modifier and TypeMethod and Description
      java.math.BigDecimalgetBigDecimal(String fieldName) +
      Retrieves value from data record and returns the value as a java.math.BigDecimal.
      +
      byte[]getBinary(String fieldName) +
      Retrieves value from data record and returns the value as binary (byte array).
      +
      booleangetBoolean(String fieldName) +
      Retrieves value from data record and returns the value as a primitive boolean.
      +
      bytegetByte(String fieldName) +
      Retrieves value from data record and returns the value as a primitive byte.
      +
      java.sql.DategetDate(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Date.
      +
      doublegetDouble(String fieldName) +
      Retrieves value from data record and returns the value as a primitive double.
      +
      floatgetFloat(String fieldName) +
      Retrieves value from data record and returns the value as a primitive float.
      +
      intgetInt(String fieldName) +
      Retrieves value from data record and returns the value as a primitive int.
      +
      intgetLength() 
      <T> java.util.List<T>getList(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.List<T> object.
      +
      longgetLong(String fieldName) +
      Retrieves value from data record and returns the value as a primitive long.
      +
      <K,V> java.util.Map<K,V>getMap(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
      +
      RowRecordgetRecord(String fieldName) +
      Retrieves value from data record and returns the value as a RowRecord object.
      +
      StructTypegetSchema() 
      shortgetShort(String fieldName) +
      Retrieves value from data record and returns the value as a primitive short.
      +
      StringgetString(String fieldName) +
      Retrieves value from data record and returns the value as a String object.
      +
      java.sql.TimestampgetTimestamp(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Timestamp.
      +
      booleanisNullAt(String fieldName) 
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getLength

        +
        int getLength()
        +
      • +
      + + + +
        +
      • +

        isNullAt

        +
        boolean isNullAt(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        +
      • +
      + + + +
        +
      • +

        getInt

        +
        int getInt(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive int.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getLong

        +
        long getLong(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive long.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getByte

        +
        byte getByte(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive byte.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getShort

        +
        short getShort(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive short.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBoolean

        +
        boolean getBoolean(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive boolean.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getFloat

        +
        float getFloat(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive float.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDouble

        +
        double getDouble(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive double.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getString

        +
        String getString(String fieldName)
        +
        Retrieves value from data record and returns the value as a String object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBinary

        +
        byte[] getBinary(String fieldName)
        +
        Retrieves value from data record and returns the value as binary (byte array).
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBigDecimal

        +
        java.math.BigDecimal getBigDecimal(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.math.BigDecimal.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        java.sql.Timestamp getTimestamp(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Timestamp.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDate

        +
        java.sql.Date getDate(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Date.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getRecord

        +
        RowRecord getRecord(String fieldName)
        +
        Retrieves value from data record and returns the value as a RowRecord object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any nested field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getList

        +
        <T> java.util.List<T> getList(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.List<T> object.
        +
        +
        Type Parameters:
        +
        T - element type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any element field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getMap

        +
        <K,V> java.util.Map<K,V> getMap(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
        +
        +
        Type Parameters:
        +
        K - key type
        +
        V - value type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any key/value field, if that field is not + nullable and null data value read
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-frame.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-frame.html new file mode 100644 index 00000000000..044cd9dfddb --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-frame.html @@ -0,0 +1,21 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.data

+
+

Interfaces

+ +
+ + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-summary.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-summary.html new file mode 100644 index 00000000000..e32652d82ea --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-summary.html @@ -0,0 +1,148 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.data

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    CloseableIterator<T> +
    An Iterator that also need to implement the Closeable interface.
    +
    RowRecord +
    Represents one row of data containing a non-empty collection of fieldName - value pairs.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-tree.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-tree.html new file mode 100644 index 00000000000..16daf1856a5 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/data/package-tree.html @@ -0,0 +1,145 @@ + + + + + +io.delta.standalone.data Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.data

+Package Hierarchies: + +
+
+

Interface Hierarchy

+
    +
  • AutoCloseable +
      +
    • java.io.Closeable +
        +
      • io.delta.standalone.data.CloseableIterator<T> (also extends java.util.Iterator<E>)
      • +
      +
    • +
    +
  • +
  • java.util.Iterator<E> + +
  • +
  • io.delta.standalone.data.RowRecord
  • +
+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/package-frame.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-frame.html new file mode 100644 index 00000000000..71246a3e2e5 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-frame.html @@ -0,0 +1,21 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone

+
+

Interfaces

+ +
+ + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/package-summary.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-summary.html new file mode 100644 index 00000000000..947354b4a3a --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-summary.html @@ -0,0 +1,149 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    DeltaLog +
    DeltaLog is the representation of the transaction logs of a Delta table.
    +
    Snapshot +
    Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/package-tree.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-tree.html new file mode 100644 index 00000000000..5a1a8050147 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/package-tree.html @@ -0,0 +1,132 @@ + + + + + +io.delta.standalone Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone

+Package Hierarchies: + +
+
+

Interface Hierarchy

+ +
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html new file mode 100644 index 00000000000..63779f3deb8 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html @@ -0,0 +1,336 @@ + + + + + +ArrayType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ArrayType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ArrayType
    +extends DataType
    +
    The data type for collections of multiple values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ArrayType

        +
        public ArrayType(DataType elementType,
        +                 boolean containsNull)
        +
        +
        Parameters:
        +
        elementType - the data type of values
        +
        containsNull - indicates if values have null value
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getElementType

        +
        public DataType getElementType()
        +
      • +
      + + + +
        +
      • +

        containsNull

        +
        public boolean containsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html new file mode 100644 index 00000000000..866608eb730 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html @@ -0,0 +1,248 @@ + + + + + +BinaryType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BinaryType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BinaryType
    +extends DataType
    +
    The data type representing byte[] values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BinaryType

        +
        public BinaryType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html new file mode 100644 index 00000000000..edf7af06335 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html @@ -0,0 +1,248 @@ + + + + + +BooleanType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BooleanType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BooleanType
    +extends DataType
    +
    The data type representing boolean values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BooleanType

        +
        public BooleanType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ByteType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ByteType.html new file mode 100644 index 00000000000..5ad1ab2673e --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ByteType.html @@ -0,0 +1,286 @@ + + + + + +ByteType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ByteType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ByteType
    +extends DataType
    +
    The data type representing byte values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ByteType

        +
        public ByteType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DataType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DataType.html new file mode 100644 index 00000000000..69e6f370e1d --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DataType.html @@ -0,0 +1,337 @@ + + + + + +DataType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DataType

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.DataType
    • +
    +
  • +
+
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DataType

        +
        public DataType()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getTypeName

        +
        public String getTypeName()
        +
      • +
      + + + +
        +
      • +

        getSimpleString

        +
        public String getSimpleString()
        +
      • +
      + + + +
        +
      • +

        getCatalogString

        +
        public String getCatalogString()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DateType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DateType.html new file mode 100644 index 00000000000..2e5e0c2a1b5 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DateType.html @@ -0,0 +1,249 @@ + + + + + +DateType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DateType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DateType
    +extends DataType
    +
    A date type, supporting "0001-01-01" through "9999-12-31". + Internally, this is represented as the number of days from 1970-01-01.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DateType

        +
        public DateType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html new file mode 100644 index 00000000000..84eabab6941 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html @@ -0,0 +1,373 @@ + + + + + +DecimalType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DecimalType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DecimalType
    +extends DataType
    +
    The data type representing java.math.BigDecimal values. + A Decimal that must have fixed precision (the maximum number of digits) and scale (the number + of digits on right side of dot). + + The precision can be up to 38, scale can also be up to 38 (less or equal to precision). + + The default precision and scale is (10, 0).
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        USER_DEFAULT

        +
        public static final DecimalType USER_DEFAULT
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DecimalType

        +
        public DecimalType(int precision,
        +                   int scale)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPrecision

        +
        public int getPrecision()
        +
      • +
      + + + +
        +
      • +

        getScale

        +
        public int getScale()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html new file mode 100644 index 00000000000..437c2acae2e --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html @@ -0,0 +1,248 @@ + + + + + +DoubleType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DoubleType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DoubleType
    +extends DataType
    +
    The data type representing double values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DoubleType

        +
        public DoubleType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/FloatType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/FloatType.html new file mode 100644 index 00000000000..e35b4d7b127 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/FloatType.html @@ -0,0 +1,248 @@ + + + + + +FloatType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class FloatType

+
+
+ +
+
    +
  • +
    +
    +
    public final class FloatType
    +extends DataType
    +
    The data type representing float values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        FloatType

        +
        public FloatType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html new file mode 100644 index 00000000000..31a88fc17d4 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html @@ -0,0 +1,286 @@ + + + + + +IntegerType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class IntegerType

+
+
+ +
+
    +
  • +
    +
    +
    public final class IntegerType
    +extends DataType
    +
    The data type representing int values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        IntegerType

        +
        public IntegerType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/LongType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/LongType.html new file mode 100644 index 00000000000..75d8dcd0590 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/LongType.html @@ -0,0 +1,286 @@ + + + + + +LongType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class LongType

+
+
+ +
+
    +
  • +
    +
    +
    public final class LongType
    +extends DataType
    +
    The data type representing long values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        LongType

        +
        public LongType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/MapType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/MapType.html new file mode 100644 index 00000000000..c814ae9f046 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/MapType.html @@ -0,0 +1,352 @@ + + + + + +MapType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class MapType

+
+
+ +
+
    +
  • +
    +
    +
    public final class MapType
    +extends DataType
    +
    The data type for Maps. Keys in a map are not allowed to have null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        MapType

        +
        public MapType(DataType keyType,
        +               DataType valueType,
        +               boolean valueContainsNull)
        +
        +
        Parameters:
        +
        keyType - the data type of map keys
        +
        valueType - the data type of map values
        +
        valueContainsNull - indicates if map values have null values
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getKeyType

        +
        public DataType getKeyType()
        +
      • +
      + + + +
        +
      • +

        getValueType

        +
        public DataType getValueType()
        +
      • +
      + + + +
        +
      • +

        valueContainsNull

        +
        public boolean valueContainsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/NullType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/NullType.html new file mode 100644 index 00000000000..f8e8a8fae62 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/NullType.html @@ -0,0 +1,248 @@ + + + + + +NullType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class NullType

+
+
+ +
+
    +
  • +
    +
    +
    public final class NullType
    +extends DataType
    +
    The data type representing null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NullType

        +
        public NullType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ShortType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ShortType.html new file mode 100644 index 00000000000..aa995b02759 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/ShortType.html @@ -0,0 +1,286 @@ + + + + + +ShortType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ShortType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ShortType
    +extends DataType
    +
    The data type representing short values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ShortType

        +
        public ShortType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StringType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StringType.html new file mode 100644 index 00000000000..eab20c6cadd --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StringType.html @@ -0,0 +1,248 @@ + + + + + +StringType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StringType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StringType
    +extends DataType
    +
    The data type representing String values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StringType

        +
        public StringType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructField.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructField.html new file mode 100644 index 00000000000..38b613fe61c --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructField.html @@ -0,0 +1,362 @@ + + + + + +StructField (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructField

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.StructField
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class StructField
    +extends Object
    +
    A field inside a StructType.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType,
        +                   boolean nullable)
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        nullable - indicates if values of this field can be null values
        +
        +
      • +
      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType)
        +
        Constructor with default nullable = true.
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDataType

        +
        public DataType getDataType()
        +
      • +
      + + + +
        +
      • +

        isNullable

        +
        public boolean isNullable()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructType.html new file mode 100644 index 00000000000..861ffdd605b --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/StructType.html @@ -0,0 +1,366 @@ + + + + + +StructType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StructType
    +extends DataType
    +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    +
    See Also:
    +
    StructField
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructType

        +
        public StructType(StructField[] fields)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getFieldNames

        +
        public String[] getFieldNames()
        +
      • +
      + + + +
        +
      • +

        get

        +
        public StructField get(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - the name of the desired StructField, not null
        +
        Throws:
        +
        IllegalArgumentException - if a field with the given name does not exist
        +
        +
      • +
      + + + +
        +
      • +

        getTreeString

        +
        public String getTreeString()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html new file mode 100644 index 00000000000..49cb01e0ce7 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html @@ -0,0 +1,248 @@ + + + + + +TimestampType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class TimestampType

+
+
+ +
+
    +
  • +
    +
    +
    public final class TimestampType
    +extends DataType
    +
    The data type representing java.sql.Timestamp values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        TimestampType

        +
        public TimestampType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-frame.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-frame.html new file mode 100644 index 00000000000..e081f233f13 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-frame.html @@ -0,0 +1,37 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.types

+ + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-summary.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-summary.html new file mode 100644 index 00000000000..aead501f7f3 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-summary.html @@ -0,0 +1,245 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.types

+
+
+
    +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    ArrayType +
    The data type for collections of multiple values.
    +
    BinaryType +
    The data type representing byte[] values.
    +
    BooleanType +
    The data type representing boolean values.
    +
    ByteType +
    The data type representing byte values.
    +
    DataType +
    The base type of all io.delta.standalone data types.
    +
    DateType +
    A date type, supporting "0001-01-01" through "9999-12-31".
    +
    DecimalType +
    The data type representing java.math.BigDecimal values.
    +
    DoubleType +
    The data type representing double values.
    +
    FloatType +
    The data type representing float values.
    +
    IntegerType +
    The data type representing int values.
    +
    LongType +
    The data type representing long values.
    +
    MapType +
    The data type for Maps.
    +
    NullType +
    The data type representing null values.
    +
    ShortType +
    The data type representing short values.
    +
    StringType +
    The data type representing String values.
    +
    StructField +
    A field inside a StructType.
    +
    StructType +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    TimestampType +
    The data type representing java.sql.Timestamp values.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-tree.html b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-tree.html new file mode 100644 index 00000000000..4e52a7f4997 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/io/delta/standalone/types/package-tree.html @@ -0,0 +1,155 @@ + + + + + +io.delta.standalone.types Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.types

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/latest/delta-standalone/api/java/overview-frame.html b/docs/latest/delta-standalone/api/java/overview-frame.html new file mode 100644 index 00000000000..0187b771272 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/overview-frame.html @@ -0,0 +1,24 @@ + + + + + +Overview List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + +

 

+ + diff --git a/docs/latest/delta-standalone/api/java/overview-summary.html b/docs/latest/delta-standalone/api/java/overview-summary.html new file mode 100644 index 00000000000..69ddb947786 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/overview-summary.html @@ -0,0 +1,145 @@ + + + + + +Overview (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+ + + + + + + + + + + + + + + + + + + + + + + + +
Packages 
PackageDescription
io.delta.standalone 
io.delta.standalone.actions 
io.delta.standalone.data 
io.delta.standalone.types 
+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/latest/delta-standalone/api/java/overview-tree.html b/docs/latest/delta-standalone/api/java/overview-tree.html new file mode 100644 index 00000000000..1461dd63ed0 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/overview-tree.html @@ -0,0 +1,184 @@ + + + + + +Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

Hierarchy For All Packages

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +

Interface Hierarchy

+
    +
  • AutoCloseable +
      +
    • java.io.Closeable +
        +
      • io.delta.standalone.data.CloseableIterator<T> (also extends java.util.Iterator<E>)
      • +
      +
    • +
    +
  • +
  • io.delta.standalone.DeltaLog
  • +
  • java.util.Iterator<E> + +
  • +
  • io.delta.standalone.data.RowRecord
  • +
  • io.delta.standalone.Snapshot
  • +
+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/latest/delta-standalone/api/java/package-list b/docs/latest/delta-standalone/api/java/package-list new file mode 100644 index 00000000000..5dab2c7aadd --- /dev/null +++ b/docs/latest/delta-standalone/api/java/package-list @@ -0,0 +1,4 @@ +io.delta.standalone +io.delta.standalone.actions +io.delta.standalone.data +io.delta.standalone.types diff --git a/docs/latest/delta-standalone/api/java/script.js b/docs/latest/delta-standalone/api/java/script.js new file mode 100644 index 00000000000..b3463569314 --- /dev/null +++ b/docs/latest/delta-standalone/api/java/script.js @@ -0,0 +1,30 @@ +function show(type) +{ + count = 0; + for (var key in methods) { + var row = document.getElementById(key); + if ((methods[key] & type) != 0) { + row.style.display = ''; + row.className = (count++ % 2) ? rowColor : altColor; + } + else + row.style.display = 'none'; + } + updateTabs(type); +} + +function updateTabs(type) +{ + for (var value in tabs) { + var sNode = document.getElementById(tabs[value][0]); + var spanNode = sNode.firstChild; + if (value == type) { + sNode.className = activeTableTab; + spanNode.innerHTML = tabs[value][1]; + } + else { + sNode.className = tableTab; + spanNode.innerHTML = "" + tabs[value][1] + ""; + } + } +} diff --git a/docs/latest/delta-standalone/api/java/stylesheet.css b/docs/latest/delta-standalone/api/java/stylesheet.css new file mode 100644 index 00000000000..98055b22d6d --- /dev/null +++ b/docs/latest/delta-standalone/api/java/stylesheet.css @@ -0,0 +1,574 @@ +/* Javadoc style sheet */ +/* +Overall document style +*/ + +@import url('resources/fonts/dejavu.css'); + +body { + background-color:#ffffff; + color:#353833; + font-family:'DejaVu Sans', Arial, Helvetica, sans-serif; + font-size:14px; + margin:0; +} +a:link, a:visited { + text-decoration:none; + color:#4A6782; +} +a:hover, a:focus { + text-decoration:none; + color:#bb7a2a; +} +a:active { + text-decoration:none; + color:#4A6782; +} +a[name] { + color:#353833; +} +a[name]:hover { + text-decoration:none; + color:#353833; +} +pre { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; +} +h1 { + font-size:20px; +} +h2 { + font-size:18px; +} +h3 { + font-size:16px; + font-style:italic; +} +h4 { + font-size:13px; +} +h5 { + font-size:12px; +} +h6 { + font-size:11px; +} +ul { + list-style-type:disc; +} +code, tt { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; + margin-top:8px; + line-height:1.4em; +} +dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; +} +table tr td dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + vertical-align:top; + padding-top:4px; +} +sup { + font-size:8px; +} +/* +Document title and Copyright styles +*/ +.clear { + clear:both; + height:0px; + overflow:hidden; +} +.aboutLanguage { + float:right; + padding:0px 21px; + font-size:11px; + z-index:200; + margin-top:-9px; +} +.legalCopy { + margin-left:.5em; +} +.bar a, .bar a:link, .bar a:visited, .bar a:active { + color:#FFFFFF; + text-decoration:none; +} +.bar a:hover, .bar a:focus { + color:#bb7a2a; +} +.tab { + background-color:#0066FF; + color:#ffffff; + padding:8px; + width:5em; + font-weight:bold; +} +/* +Navigation bar styles +*/ +.bar { + background-color:#4D7A97; + color:#FFFFFF; + padding:.8em .5em .4em .8em; + height:auto;/*height:1.8em;*/ + font-size:11px; + margin:0; +} +.topNav { + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.bottomNav { + margin-top:10px; + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.subNav { + background-color:#dee3e9; + float:left; + width:100%; + overflow:hidden; + font-size:12px; +} +.subNav div { + clear:left; + float:left; + padding:0 0 5px 6px; + text-transform:uppercase; +} +ul.navList, ul.subNavList { + float:left; + margin:0 25px 0 0; + padding:0; +} +ul.navList li{ + list-style:none; + float:left; + padding: 5px 6px; + text-transform:uppercase; +} +ul.subNavList li{ + list-style:none; + float:left; +} +.topNav a:link, .topNav a:active, .topNav a:visited, .bottomNav a:link, .bottomNav a:active, .bottomNav a:visited { + color:#FFFFFF; + text-decoration:none; + text-transform:uppercase; +} +.topNav a:hover, .bottomNav a:hover { + text-decoration:none; + color:#bb7a2a; + text-transform:uppercase; +} +.navBarCell1Rev { + background-color:#F8981D; + color:#253441; + margin: auto 5px; +} +.skipNav { + position:absolute; + top:auto; + left:-9999px; + overflow:hidden; +} +/* +Page header and footer styles +*/ +.header, .footer { + clear:both; + margin:0 20px; + padding:5px 0 0 0; +} +.indexHeader { + margin:10px; + position:relative; +} +.indexHeader span{ + margin-right:15px; +} +.indexHeader h1 { + font-size:13px; +} +.title { + color:#2c4557; + margin:10px 0; +} +.subTitle { + margin:5px 0 0 0; +} +.header ul { + margin:0 0 15px 0; + padding:0; +} +.footer ul { + margin:20px 0 5px 0; +} +.header ul li, .footer ul li { + list-style:none; + font-size:13px; +} +/* +Heading styles +*/ +div.details ul.blockList ul.blockList ul.blockList li.blockList h4, div.details ul.blockList ul.blockList ul.blockListLast li.blockList h4 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList ul.blockList li.blockList h3 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList li.blockList h3 { + padding:0; + margin:15px 0; +} +ul.blockList li.blockList h2 { + padding:0px 0 20px 0; +} +/* +Page layout container styles +*/ +.contentContainer, .sourceContainer, .classUseContainer, .serializedFormContainer, .constantValuesContainer { + clear:both; + padding:10px 20px; + position:relative; +} +.indexContainer { + margin:10px; + position:relative; + font-size:12px; +} +.indexContainer h2 { + font-size:13px; + padding:0 0 3px 0; +} +.indexContainer ul { + margin:0; + padding:0; +} +.indexContainer ul li { + list-style:none; + padding-top:2px; +} +.contentContainer .description dl dt, .contentContainer .details dl dt, .serializedFormContainer dl dt { + font-size:12px; + font-weight:bold; + margin:10px 0 0 0; + color:#4E4E4E; +} +.contentContainer .description dl dd, .contentContainer .details dl dd, .serializedFormContainer dl dd { + margin:5px 0 10px 0px; + font-size:14px; + font-family:'DejaVu Sans Mono',monospace; +} +.serializedFormContainer dl.nameValue dt { + margin-left:1px; + font-size:1.1em; + display:inline; + font-weight:bold; +} +.serializedFormContainer dl.nameValue dd { + margin:0 0 0 1px; + font-size:1.1em; + display:inline; +} +/* +List styles +*/ +ul.horizontal li { + display:inline; + font-size:0.9em; +} +ul.inheritance { + margin:0; + padding:0; +} +ul.inheritance li { + display:inline; + list-style:none; +} +ul.inheritance li ul.inheritance { + margin-left:15px; + padding-left:15px; + padding-top:1px; +} +ul.blockList, ul.blockListLast { + margin:10px 0 10px 0; + padding:0; +} +ul.blockList li.blockList, ul.blockListLast li.blockList { + list-style:none; + margin-bottom:15px; + line-height:1.4; +} +ul.blockList ul.blockList li.blockList, ul.blockList ul.blockListLast li.blockList { + padding:0px 20px 5px 10px; + border:1px solid #ededed; + background-color:#f8f8f8; +} +ul.blockList ul.blockList ul.blockList li.blockList, ul.blockList ul.blockList ul.blockListLast li.blockList { + padding:0 0 5px 8px; + background-color:#ffffff; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockList { + margin-left:0; + padding-left:0; + padding-bottom:15px; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockListLast { + list-style:none; + border-bottom:none; + padding-bottom:0; +} +table tr td dl, table tr td dl dt, table tr td dl dd { + margin-top:0; + margin-bottom:1px; +} +/* +Table styles +*/ +.overviewSummary, .memberSummary, .typeSummary, .useSummary, .constantsSummary, .deprecatedSummary { + width:100%; + border-left:1px solid #EEE; + border-right:1px solid #EEE; + border-bottom:1px solid #EEE; +} +.overviewSummary, .memberSummary { + padding:0px; +} +.overviewSummary caption, .memberSummary caption, .typeSummary caption, +.useSummary caption, .constantsSummary caption, .deprecatedSummary caption { + position:relative; + text-align:left; + background-repeat:no-repeat; + color:#253441; + font-weight:bold; + clear:none; + overflow:hidden; + padding:0px; + padding-top:10px; + padding-left:1px; + margin:0px; + white-space:pre; +} +.overviewSummary caption a:link, .memberSummary caption a:link, .typeSummary caption a:link, +.useSummary caption a:link, .constantsSummary caption a:link, .deprecatedSummary caption a:link, +.overviewSummary caption a:hover, .memberSummary caption a:hover, .typeSummary caption a:hover, +.useSummary caption a:hover, .constantsSummary caption a:hover, .deprecatedSummary caption a:hover, +.overviewSummary caption a:active, .memberSummary caption a:active, .typeSummary caption a:active, +.useSummary caption a:active, .constantsSummary caption a:active, .deprecatedSummary caption a:active, +.overviewSummary caption a:visited, .memberSummary caption a:visited, .typeSummary caption a:visited, +.useSummary caption a:visited, .constantsSummary caption a:visited, .deprecatedSummary caption a:visited { + color:#FFFFFF; +} +.overviewSummary caption span, .memberSummary caption span, .typeSummary caption span, +.useSummary caption span, .constantsSummary caption span, .deprecatedSummary caption span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + padding-bottom:7px; + display:inline-block; + float:left; + background-color:#F8981D; + border: none; + height:16px; +} +.memberSummary caption span.activeTableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#F8981D; + height:16px; +} +.memberSummary caption span.tableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#4D7A97; + height:16px; +} +.memberSummary caption span.tableTab, .memberSummary caption span.activeTableTab { + padding-top:0px; + padding-left:0px; + padding-right:0px; + background-image:none; + float:none; + display:inline; +} +.overviewSummary .tabEnd, .memberSummary .tabEnd, .typeSummary .tabEnd, +.useSummary .tabEnd, .constantsSummary .tabEnd, .deprecatedSummary .tabEnd { + display:none; + width:5px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .activeTableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .tableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + background-color:#4D7A97; + float:left; + +} +.overviewSummary td, .memberSummary td, .typeSummary td, +.useSummary td, .constantsSummary td, .deprecatedSummary td { + text-align:left; + padding:0px 0px 12px 10px; +} +th.colOne, th.colFirst, th.colLast, .useSummary th, .constantsSummary th, +td.colOne, td.colFirst, td.colLast, .useSummary td, .constantsSummary td{ + vertical-align:top; + padding-right:0px; + padding-top:8px; + padding-bottom:3px; +} +th.colFirst, th.colLast, th.colOne, .constantsSummary th { + background:#dee3e9; + text-align:left; + padding:8px 3px 3px 7px; +} +td.colFirst, th.colFirst { + white-space:nowrap; + font-size:13px; +} +td.colLast, th.colLast { + font-size:13px; +} +td.colOne, th.colOne { + font-size:13px; +} +.overviewSummary td.colFirst, .overviewSummary th.colFirst, +.useSummary td.colFirst, .useSummary th.colFirst, +.overviewSummary td.colOne, .overviewSummary th.colOne, +.memberSummary td.colFirst, .memberSummary th.colFirst, +.memberSummary td.colOne, .memberSummary th.colOne, +.typeSummary td.colFirst{ + width:25%; + vertical-align:top; +} +td.colOne a:link, td.colOne a:active, td.colOne a:visited, td.colOne a:hover, td.colFirst a:link, td.colFirst a:active, td.colFirst a:visited, td.colFirst a:hover, td.colLast a:link, td.colLast a:active, td.colLast a:visited, td.colLast a:hover, .constantValuesContainer td a:link, .constantValuesContainer td a:active, .constantValuesContainer td a:visited, .constantValuesContainer td a:hover { + font-weight:bold; +} +.tableSubHeadingColor { + background-color:#EEEEFF; +} +.altColor { + background-color:#FFFFFF; +} +.rowColor { + background-color:#EEEEEF; +} +/* +Content styles +*/ +.description pre { + margin-top:0; +} +.deprecatedContent { + margin:0; + padding:10px 0; +} +.docSummary { + padding:0; +} + +ul.blockList ul.blockList ul.blockList li.blockList h3 { + font-style:normal; +} + +div.block { + font-size:14px; + font-family:'DejaVu Serif', Georgia, "Times New Roman", Times, serif; +} + +td.colLast div { + padding-top:0px; +} + + +td.colLast a { + padding-bottom:3px; +} +/* +Formatting effect styles +*/ +.sourceLineNo { + color:green; + padding:0 30px 0 0; +} +h1.hidden { + visibility:hidden; + overflow:hidden; + font-size:10px; +} +.block { + display:block; + margin:3px 10px 2px 0px; + color:#474747; +} +.deprecatedLabel, .descfrmTypeLabel, .memberNameLabel, .memberNameLink, +.overrideSpecifyLabel, .packageHierarchyLabel, .paramLabel, .returnLabel, +.seeLabel, .simpleTagLabel, .throwsLabel, .typeNameLabel, .typeNameLink { + font-weight:bold; +} +.deprecationComment, .emphasizedPhrase, .interfaceName { + font-style:italic; +} + +div.block div.block span.deprecationComment, div.block div.block span.emphasizedPhrase, +div.block div.block span.interfaceName { + font-style:normal; +} + +div.contentContainer ul.blockList li.blockList h2{ + padding-bottom:0px; +} From b76223b375cbff54f3bc31b3c1086871041e8389 Mon Sep 17 00:00:00 2001 From: Denny Lee Date: Mon, 5 Apr 2021 09:10:45 -0700 Subject: [PATCH 055/291] Create AUTHORS.md (#83) --- AUTHORS.md | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 AUTHORS.md diff --git a/AUTHORS.md b/AUTHORS.md new file mode 100644 index 00000000000..47138157691 --- /dev/null +++ b/AUTHORS.md @@ -0,0 +1,7 @@ +# This is the official list of the Delta Lake Project Authors for copyright purposes. + +# Names should be added to this file as: +# Name or Organization +# The email address is not required for organizations. + +Databricks From 1912ac5b6d4f3742d7d0d21b52e03846dd5e8683 Mon Sep 17 00:00:00 2001 From: Denny Lee Date: Tue, 6 Apr 2021 09:20:29 -0700 Subject: [PATCH 056/291] Rename AUTHORS.md to AUTHORS --- AUTHORS.md => AUTHORS | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename AUTHORS.md => AUTHORS (100%) diff --git a/AUTHORS.md b/AUTHORS similarity index 100% rename from AUTHORS.md rename to AUTHORS From 539463a99b0c9b84f03448efe6c8da3a5b4f4a28 Mon Sep 17 00:00:00 2001 From: Alex Date: Mon, 14 Jun 2021 22:31:30 -0400 Subject: [PATCH 057/291] Adding `sql-delta-import` utility. (#80) * Adding sql-delta-import * cleanup. resolving compilation and scala fmt issues * changing copyright to Scribd * adding link to README.md * using scala 2_12 only for `sql-delta-import` * Create AUTHORS.md (#83) * Rename AUTHORS.md to AUTHORS * Addressing PR feedback. Adding Scribd to AUTHORS Changing attribution to The Delta Lake Project Authors * Update AUTHORS * Addressing PR feedback * fixing formatting * fixing dependency resolution failures * changing spark-sql dependency for cross scala versions * adding test for sql-delta-import * proper project name in CircleCI * Just trying to restart circle CI build.. * only publishing sql-delta-import for scala 2.12 * adding aliases to lower/upper bounds columns in bounds query to better support data stores that require it Co-authored-by: Alex Kushnir Co-authored-by: Denny Lee --- .circleci/config.yml | 1 + AUTHORS | 1 + README.md | 5 + build.sbt | 19 +++ scalastyle-config.xml | 2 +- sql-delta-import/readme.md | 89 ++++++++++++ .../src/main/scala/DataTransforms.scala | 33 +++++ .../src/main/scala/ImportRunner.scala | 70 +++++++++ .../src/main/scala/JDBCImport.scala | 128 ++++++++++++++++ .../src/test/scala/ImportTest.scala | 137 ++++++++++++++++++ 10 files changed, 484 insertions(+), 1 deletion(-) create mode 100644 sql-delta-import/readme.md create mode 100644 sql-delta-import/src/main/scala/DataTransforms.scala create mode 100644 sql-delta-import/src/main/scala/ImportRunner.scala create mode 100644 sql-delta-import/src/main/scala/JDBCImport.scala create mode 100644 sql-delta-import/src/test/scala/ImportTest.scala diff --git a/.circleci/config.yml b/.circleci/config.yml index 3e9e7364258..51689c86462 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -12,6 +12,7 @@ jobs: build/sbt "++ 2.12.8 hive/test" build/sbt "++ 2.12.8 hiveMR/test" build/sbt "++ 2.12.8 hiveTez/test" + build/sbt "++ 2.12.8 sqlDeltaImport/test" build/sbt "++ 2.11.12 standalone/test" build/sbt "++ 2.11.12 hive/test" build/sbt "++ 2.11.12 hiveMR/test" diff --git a/AUTHORS b/AUTHORS index 47138157691..3d97a015bc2 100644 --- a/AUTHORS +++ b/AUTHORS @@ -5,3 +5,4 @@ # The email address is not required for organizations. Databricks +Scribd Inc diff --git a/README.md b/README.md index 6c5955277f9..d7bada751af 100644 --- a/README.md +++ b/README.md @@ -164,6 +164,11 @@ If the schema in the underlying Delta metadata is not consistent with the schema #### Hive has three execution engines, MapReduce, Tez and Spark. Which one does this connector support? The connector supports MapReduce and Tez. It doesn't support Spark execution engine in Hive. +## sql-delta-import + +[sql-delta-import](/sql-delta-import/readme.md) allows for importing data from a JDBC source into a Delta Lake table + + # Reporting issues We use [GitHub Issues](https://github.com/delta-io/connectors/issues) to track community reported issues. You can also [contact](#community) the community for getting answers. diff --git a/build.sbt b/build.sbt index 661ffc8b42d..d5400f2dae8 100644 --- a/build.sbt +++ b/build.sbt @@ -312,3 +312,22 @@ lazy val goldenTables = (project in file("golden-tables")) settings ( "org.apache.spark" % "spark-sql_2.12" % "3.0.0" % "test" classifier "tests" ) ) + +lazy val sqlDeltaImport = (project in file("sql-delta-import")) + .settings ( + name := "sql-delta-import", + commonSettings, + publishArtifact := scalaBinaryVersion.value == "2.12", + publishArtifact in Test := false, + libraryDependencies ++= Seq( + "org.apache.spark" %% "spark-sql" % sparkVersion % "provided", + "io.delta" % "delta-core_2.12" % "0.7.0" % "provided", + "org.rogach" %% "scallop" % "3.5.1", + "org.scalatest" %% "scalatest" % "3.1.1" % "test", + "com.h2database" % "h2" % "1.4.200" % "test", + "org.apache.spark" % "spark-catalyst_2.12" % "3.0.0" % "test", + "org.apache.spark" % "spark-core_2.12" % "3.0.0" % "test", + "org.apache.spark" % "spark-sql_2.12" % "3.0.0" % "test" + ) + ) + .settings(releaseSettings) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index bc5d4444b7c..29b569fd689 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -92,7 +92,7 @@ This file is divided into 3 sections: */ \E)?\Q/* - * Copyright (2020) The Delta Lake Project Authors. + * Copyright (202\E[0-1]\Q) The Delta Lake Project Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-delta-import/readme.md b/sql-delta-import/readme.md new file mode 100644 index 00000000000..4efcd60208f --- /dev/null +++ b/sql-delta-import/readme.md @@ -0,0 +1,89 @@ +# sql-delta-import + + Imports data from a relational database or any other JDBC source into your Delta Lake. + Import either entire table or only a subset of columns, control level of parallelism, + include any custom transformations + +Destination delta table has to exist before import. It's schema will be used to infer +desired columns and their data types + +## Basic Usage + +You can use included runner to import data without custom transformations by submitting +a spark job + +Ex: split data in source table by `id` into 10 chunks (default) and import it into +destination delta table + +```shell script +spark-submit / +--class "io.delta.connectors.spark.jdbc.ImportRunner" sql-delta-import.jar / +--jdbc-url jdbc:mysql://hostName:port/database / +--source source.table +--destination destination.table +--split-by id +``` +A good `split-by` column will be indexed and ideally will have close to uniform distribution +of data between it's `min` and `max` values + +## Control degree of import parallelism using `chunks` parameter and spark executor configuration + +```shell script +spark-submit --num-executors 15 --executor-cores 4 / +--conf spark.databricks.delta.optimizeWrite.enabled=true / +--conf spark.databricks.delta.autoCompact.enabled=true / +--class "io.delta.connectors.spark.jdbc.ImportRunner" sql-delta-import.jar / +--jdbc-url jdbc:mysql://hostName:port/database / +--source source.table +--destination destination.table +--split-by id +--chunks 500 +``` +Source table will be split by `id` column into 500 chunks but only at most 60 connections +(15 executors x 4 cores) will be used to import the data. This allows us to import large +tables without overloading underlying data store with large volume of connections. This +configuration is also useful when distribution of data by `split-by` column is not uniform +and there are "gaps" for large ranges of values. Delta's auto compaction and optimization +features are enabled via spark configuration to make sure that storage of imported data is +optimized - avoid small files and skewed file sizes. + +## Use JDBCImport in your project to specify custom transformations that will be applied during import + +```scala +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import io.delta.connectors.spark.jdbc._ + + implicit val spark: SparkSession = SparkSession + .builder() + .master("local[*]") + .getOrCreate() + + // All additional possible jdbc connector properties described here - + // https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-configuration-properties.html + + val jdbcUrl = "jdbc:mysql://hostName:port/database" + + val config = ImportConfig( + source = "table", + destination = "target_database.table", + splitBy = "id", + chunks = 10) + + // define a transform to convert all timestamp columns to strings + val timeStampsToStrings : DataFrame => DataFrame = source => { + val tsCols = source.schema.fields.filter(_.dataType == DataTypes.TimestampType).map(_.name) + tsCols.foldLeft(source)((df, colName) => + df.withColumn(colName, from_unixtime(unix_timestamp(col(colName)), "yyyy-MM-dd HH:mm:ss.S"))) +} + + // Whatever functions are passed to below transform will be applied during import + val transforms = new DataTransforms(Seq( + df => df.withColumn("id", col("id").cast(types.StringType)), // cast id column to string + timeStampsToStrings // use transform defined above for timestamp conversion + )) + + new JDBCImport(jdbcUrl = jdbcUrl, importConfig = config, dataTransform = transforms) + .run() +``` diff --git a/sql-delta-import/src/main/scala/DataTransforms.scala b/sql-delta-import/src/main/scala/DataTransforms.scala new file mode 100644 index 00000000000..404154f0eac --- /dev/null +++ b/sql-delta-import/src/main/scala/DataTransforms.scala @@ -0,0 +1,33 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.connectors.spark.jdbc + +import org.apache.spark.sql.{DataFrame, SparkSession} + +/** + * Class that applies transformation functions one by one on input DataFrame + */ +class DataTransforms(transformations: Seq[DataFrame => DataFrame]) { + + /** + * Executes functions against DataFrame + * + * @param df - input DataFrame against which functions need to be executed + * @return - modified by Seq of functions DataFrame + */ + def runTransform(df: DataFrame): DataFrame = transformations.foldLeft(df)((v, f) => f(v)) +} diff --git a/sql-delta-import/src/main/scala/ImportRunner.scala b/sql-delta-import/src/main/scala/ImportRunner.scala new file mode 100644 index 00000000000..20c6494723b --- /dev/null +++ b/sql-delta-import/src/main/scala/ImportRunner.scala @@ -0,0 +1,70 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.connectors.spark.jdbc + +import org.apache.spark.sql.SparkSession +import org.rogach.scallop.{ScallopConf, ScallopOption} + +/** + * Spark app that wraps functionality of JDBCImport and exposes configuration as command line args + */ +object ImportRunner extends App { + + override def main(args: Array[String]): Unit = { + val config = new ImportRunnerConfig(args) + + implicit val spark = SparkSession + .builder() + .appName("sql-delta-import") + .getOrCreate() + + val importConfig = ImportConfig( + config.source(), + config.destination(), + config.splitBy(), + config.chunks()) + + val transforms = new DataTransforms(Seq.empty) + + JDBCImport( + jdbcUrl = config.jdbcUrl(), + importConfig = importConfig, + dataTransforms = transforms + ).run + } +} + +class ImportRunnerConfig(arguments: Seq[String]) extends ScallopConf(arguments) { + val className = "io.delta.connectors.spark.jdbc.ImportRunner" + val jarName = "sql-delta-import.jar" + + banner("\nOptions:\n") + footer( + s"""Usage: + |spark-submit {spark options} --class $className $jarName OPTIONS + |""".stripMargin) + + override def mainOptions: Seq[String] = Seq("jdbcUrl", "source", "destination", "splitBy") + + val jdbcUrl: ScallopOption[String] = opt[String](required = true) + val source: ScallopOption[String] = opt[String](required = true) + val destination: ScallopOption[String] = opt[String](required = true) + val splitBy: ScallopOption[String] = opt[String](required = true) + val chunks: ScallopOption[Int] = opt[Int](default = Some(10)) + + verify() +} diff --git a/sql-delta-import/src/main/scala/JDBCImport.scala b/sql-delta-import/src/main/scala/JDBCImport.scala new file mode 100644 index 00000000000..41a9f9c9ccd --- /dev/null +++ b/sql-delta-import/src/main/scala/JDBCImport.scala @@ -0,0 +1,128 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.connectors.spark.jdbc + +import java.util.Properties + +import io.delta.tables.DeltaTable +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} + +import scala.collection.JavaConverters._ + +/** + * Class that contains JDBC source, read parallelism params and target table name + * + * @param source - JDBC source table + * @param destination - Delta target database.table + * @param splitBy - column by which to split source data while reading + * @param chunks - to how many chunks split jdbc source data + */ +case class ImportConfig(source: String, destination: String, splitBy: String, chunks: Int) { + val bounds_sql = s""" + (select min($splitBy) as lower_bound, max($splitBy) as upper_bound from $source) as bounds + """ +} + +/** + * Class that does reading from JDBC source, transform and writing to Delta table + * + * @param jdbcUrl - url connecting string for jdbc source + * @param importConfig - case class that contains source read parallelism params and target table + * @param jdbcParams - additional JDBC session params like isolation level, perf tuning, + * net wait params etc... + * @param dataTransform - contains function that we should apply to transform our source data + */ +class JDBCImport(jdbcUrl: String, + importConfig: ImportConfig, + jdbcParams: Map[String, String] = Map(), + dataTransform: DataTransforms) + (implicit val spark: SparkSession) { + + import spark.implicits._ + + implicit def mapToProperties(m: Map[String, String]): Properties = { + val properties = new Properties() + m.foreach(pair => properties.put(pair._1, pair._2)) + properties + } + + // list of columns to import is obtained from schema of destination delta table + private lazy val targetColumns = DeltaTable + .forName(importConfig.destination) + .toDF + .schema + .fieldNames + + private lazy val sourceDataframe = readJDBCSourceInParallel() + .select(targetColumns.map(col): _*) + + /** + * obtains lower and upper bound of source table and uses those values to read in a JDBC dataframe + * @return a dataframe read from source table + */ + private def readJDBCSourceInParallel(): DataFrame = { + + val (lower, upper) = spark + .read + .jdbc(jdbcUrl, importConfig.bounds_sql, jdbcParams) + .as[(Option[Long], Option[Long])] + .take(1) + .map { case (a, b) => (a.getOrElse(0L), b.getOrElse(0L)) } + .head + + spark.read.jdbc( + jdbcUrl, + importConfig.source, + importConfig.splitBy, + lower, + upper, + importConfig.chunks, + jdbcParams) + } + + private implicit class DataFrameExtensionOps(df: DataFrame) { + + def runTransform(): DataFrame = dataTransform.runTransform(sourceDataframe) + + def writeToDelta(deltaTableToWrite: String): Unit = df + .write + .format("delta") + .mode(SaveMode.Overwrite) + .insertInto(deltaTableToWrite) + } + + /** + * Runs transform against dataframe read from jdbc and writes it to Delta table + */ + def run(): Unit = { + sourceDataframe + .runTransform() + .writeToDelta(importConfig.destination) + } +} + +object JDBCImport { + def apply(jdbcUrl: String, + importConfig: ImportConfig, + jdbcParams: Map[String, String] = Map(), + dataTransforms: DataTransforms = new DataTransforms(Seq.empty)) + (implicit spark: SparkSession): JDBCImport = { + + new JDBCImport(jdbcUrl, importConfig, jdbcParams, dataTransforms) + } +} diff --git a/sql-delta-import/src/test/scala/ImportTest.scala b/sql-delta-import/src/test/scala/ImportTest.scala new file mode 100644 index 00000000000..4f1bf45a611 --- /dev/null +++ b/sql-delta-import/src/test/scala/ImportTest.scala @@ -0,0 +1,137 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.connectors.spark.jdbc + +import java.sql.{Connection, DriverManager} + +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DataTypes +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach} + +class ImportTest extends AnyFunSuite with BeforeAndAfterAll { + + private def initDataSource (conn: Connection) = { + conn.prepareStatement("create schema test").executeUpdate() + conn.prepareStatement( + """ + create table test.tbl( + id TINYINT, + status SMALLINT, + ts TIMESTAMP, + title VARCHAR)""" + ).executeUpdate() + conn.prepareStatement( + """ + insert into test.tbl(id, status, ts, title ) VALUES + (1, 2, parsedatetime('01-02-2021 01:02:21', 'dd-MM-yyyy hh:mm:ss'),'lorem ipsum'), + (3, 4, parsedatetime('03-04-2021 03:04:21', 'dd-MM-yyyy hh:mm:ss'),'lorem'), + (5, 6, parsedatetime('05-06-2021 05:06:21', 'dd-MM-yyyy hh:mm:ss'),'ipsum'), + (7, 8, parsedatetime('07-08-2021 07:08:21', 'dd-MM-yyyy hh:mm:ss'),'Lorem Ipsum') + """ + ).executeUpdate() + } + + implicit lazy val spark: SparkSession = SparkSession + .builder() + .master("local[*]") + .appName("spark session") + .config("spark.sql.shuffle.partitions", "10") + .config("spark.ui.enabled", "false") + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .getOrCreate() + + val url = "jdbc:h2:mem:testdb;DATABASE_TO_UPPER=FALSE" + + DriverManager.registerDriver(new org.h2.Driver()) + + val conn = DriverManager.getConnection(url) + initDataSource(conn) + + override def afterAll() { + spark.catalog.clearCache() + spark.sharedState.cacheManager.clearCache() + conn.close() + } + + val chunks = 2 + + test("import data into a delta table") { + spark.sql("DROP TABLE IF EXISTS tbl") + spark.sql(""" + CREATE TABLE tbl (id INT, status INT, title STRING) + USING DELTA + LOCATION "spark-warehouse/tbl" + """) + + + JDBCImport(url, ImportConfig("test.tbl", "tbl", "id", chunks)).run() + + // since we imported data without any optimizations number of + // read partitions should equal number of chunks used during import + assert(spark.table("tbl").rdd.getNumPartitions == chunks) + + val imported = spark.sql("select * from tbl") + .collect() + .sortBy(a => a.getAs[Int]("id")) + + assert(imported.length == 4) + assert(imported.map(a => a.getAs[Int]("id")).toSeq == Seq(1, 3, 5, 7)) + assert(imported.map(a => a.getAs[Int]("status")).toSeq == Seq(2, 4, 6, 8)) + assert(imported.map(a => a.getAs[String]("title")).toSeq == + Seq("lorem ipsum", "lorem", "ipsum", "Lorem Ipsum")) + } + + test("transform data before importing it into a delta table") { + spark.sql("DROP TABLE IF EXISTS tbl2") + spark.sql(""" + CREATE TABLE tbl2 (id INT, status INT, ts STRING, title STRING) + USING DELTA + LOCATION "spark-warehouse/tbl2" + """) + + val timeStampsToStrings : DataFrame => DataFrame = source => { + val tsCols = source.schema.fields.filter(_.dataType == DataTypes.TimestampType).map(_.name) + tsCols.foldLeft(source)((df, name) => + df.withColumn(name, from_unixtime(unix_timestamp(col(name)), "yy-MM-dd HH:mm"))) + } + + val transforms = new DataTransforms(Seq( + a => a.withColumn("title", upper(col("title"))), + timeStampsToStrings + )) + + JDBCImport( + jdbcUrl = url, + importConfig = ImportConfig("test.tbl", "tbl2", "id", 2), + dataTransforms = transforms).run() + + val imported = spark.sql("select * from tbl2") + .collect() + .sortBy(a => a.getAs[Int]("id")) + + assert(imported.length == 4) + assert(imported.map(a => a.getAs[String]("title")).toSeq == + Seq("LOREM IPSUM", "LOREM", "IPSUM", "LOREM IPSUM")) + + assert(imported.map(a => a.getAs[String]("ts")).toSeq == + Seq("21-02-01 01:02", "21-04-03 03:04", "21-06-05 05:06", "21-08-07 07:08")) + } + +} From 26495e9d72bf6d129f4820aa74c959b2fc185fc5 Mon Sep 17 00:00:00 2001 From: Gerhard Brueckl Date: Tue, 15 Jun 2021 04:43:40 +0200 Subject: [PATCH 058/291] add PowerBI connector (#87) Co-authored-by: gbrueckl --- powerbi/PowerBI_Delta.pbit | Bin 0 -> 37754 bytes powerbi/README.md | 104 ++++++++++++++++++++ powerbi/fn_ReadDeltaTable.pq | 181 +++++++++++++++++++++++++++++++++++ 3 files changed, 285 insertions(+) create mode 100644 powerbi/PowerBI_Delta.pbit create mode 100644 powerbi/README.md create mode 100644 powerbi/fn_ReadDeltaTable.pq diff --git a/powerbi/PowerBI_Delta.pbit b/powerbi/PowerBI_Delta.pbit new file mode 100644 index 0000000000000000000000000000000000000000..d8701a9e718f03610ccc834eda460eba236c4e66 GIT binary patch literal 37754 zcmaI7bySqm*FUOKDkUHwB{76ZNF&`bq;!Kw4lts0BO+Y`Lr6DBcb6hDLwCbScX!{1 z@B7~S{$j1m9|Nof=Q+>b`|Ptn`}5f!6w%OMK6>=%=_B`w9QB0oioB=)`R&motVfiO zv@IMQA@-kXvR*xUlkxCp|MwTS`H&gM6Uj#({qK!I-7b&+%HGGY7mq?xrUxmN-QBqs~xvJ{4>jh$!pyJB6=!Ofsp>%OmWVJ>;d=~2vA4g#8MEO;R805Hgz&pwl}x1(=fBPfSUgQ*>AVSHfoNq_Pl5k z>pdznI-+XvA6US-4ITcN(lsSjWFe$`x35_qC%Y6ZHVW}<+% z*xU|(Y0_s<8>5pcrJfA%MZdQ{xJM`5TF?H+r;90uWpKL5eIxR0;*TU11;+8m!n=J8 zvr~71odZa2WdC@6t{|6=@9B3vjt||kTUG@MdcV>O-kN18T+uruI3H-wucivFdw(e@ zZNmFJ)^`49|Bs{d_`5AsM8Tg&I_LPMtr}K{n%R#;&FVS|5&PpP#aQwQ?_Z@9g@hhk zzL#tkE$VI8Th}3%BG|8sRK2wnFB38DvDSS8%?<@ozeZMHqY8xV`sU$#Jb(5p8Z}Jf zn9gBGfm`bpS3KYBR@U$TB^3)96)@d`=U1&xT^EeHFGMt4sgkbnzw8$@ZcEoF-_0Ap zDQMMleF?^9K zQLuThMes<1>Y9GhLqKQplt)lCH6vCqsQtm>dY%b{hG&iq*ca z?1NTssGOTL9K7`FF`U!uwNG0kTSg)FM=wOYGp6VQVN={AWybf@60Vm61%qKU$ICQA{~geJR+xCX}&Fvw{z9R7XeCI6c zEE6@}WWDVcbbdq;+a?*7KThBBu0{B5HWpH4+#a$X;2~!bo1jWmeaC&OCwtd$#iO3? zxi!bM75;8)Um%BJQfWN;X68rsX^}q@Jnize{TCmyNrlS$TpxRmnSgBVt9lQwQ`fD+ z?SH8EZ5QLd8tb>lOO0P%KSg~sYR9YUY>^`C>_#Gu@rRdlHpHVc{|1O%j+CA;u8xE5 z?_7Tl?{P%l#kz?N3&>2y8nc+etr^NPJiSORQ+=5>6ZbhoOUK}$rKcut?Lw#hRMi#b z<5egFmb&_`yweF>1!Sq>;aQ}Or9Y_OE=K4RNnAtk+dlhX*nUsAd>w&NzXfaW=qcaOa{RyRw8rJo(Q;Bv)Q2m|?O&M^WG-T?rDPuKA{wF`O~xAr zq4cNDc+G-tzPl}DY=1b|Fx_4 z0p+iC!ph@lx1RB0$c$d0OT3SeU!<@<{@~NrM3H956}&G5+xgulvydUE#lU0YNt=<4 z#LQQUQF)>MnkMQqD)p$gvE?uqiLJx7h(<*b4ORUNy)EYK3TKHK`qwPGXCdnf;A{N| zbWtwV>BiLdWGGJCiRk-0EL1JHoF^JvsC zh*QsGsk>_JGp{jv(s8^rJE7B;+}3?&UPP4xn76XkkW`ir*H!)F=Y;1{69u+*pktoN zz3^G2pcE=MEPOf2MPlN?V7Q_e!%dnQUC zOVPq!+u}#XxJr57aUF_2M>t#gsj{SS|7LrXblz-xtXGTA6)Hf9or2BV@t>&vNSpr3yTCLm@|=dr~W8 zva2aG-~Sqpt|U4RQL1!vmyJ!g6|~J*yfj%^V9l1+VZ65IhD~*fq9>YFxz}LvJHV#24>pz7RFV?bp)1U)ahBN1p#g%N z8KHiSihG&xN~Dxi;o^_D*4Cug83aksCbWkzsKq-e6*Q0%Pjd1oe~csw#!sOn``ulF zpI*mylMfbm4u{fbtj(k*0za>3TEM>D*k|t%4vl@~&EF(>KhB;l<6O&*NVze~ofp0L z%8!&sdHGGd@6yyb?M;8P(Ef>42vx@)@!37caI_pv#xr5;4o|xu%|jz)mhYwZ{kYLVB`G!%PkN}&<;DAJ zgqE%3Mb|U^hCKU#EK+zrHAQjXqA~lq2lBxn)8)IO(ahoX~0^Dc< zk4Hs=#+Tneoh+_EMO-v05A}$cgW=qPKXDnlzg-|34yhX$?K|iufn&Zx z?$GbPfoF8`^;qu8=Wr1AT*7qCmJMEd>(OwqreH7)%M^t^The%bhMNo^~ z#8&!?Uf~V0-oD?`>#Uh!Y!##ZI6IY+qE!U?hIHtSo-Oab;)P(`0Hf+>q;jiG4KG# zPHx1F{^pXHyEKW6ZCoeGr{SM}EIHNE8MY_UIcbT?16G8LL2sqsf3gr&Mhkg6?I+^$ z4@aN3FhbZobZsVi0;e(n<>d?QaRr6>TZRhKh$)jGXy>5*3f~2+MXYjB7oOH7cCo4%4N#KOo`UT*|FM zuds`_W0>~W34W$0*&9c-XTg6h!1@Ne1J*Z2CM1`d-Vr0F@}F4fDuL?`NSSSrR9t}0 z*@=2Ph}asK)eG`u9x}ROarNwajJ>~5b$cpVgrU=t(>@kkr9m!!YcpsgP-Oa?ms}oJd+Fb{O1HK+=E5D4oyE>3RMx55X4xw!jIw7vuh92;eo z&6w&94*&JxnG{t_B>Nk~8h0ymt5Uj~U!gv;SXb+*H!Aw|Yr}Q-=hCx-yxB~>K$cQZ z;3m%D2Xgr4?n50k7dlZ)WdnboYD6=Qh~I)s@8JyJd7u&!1sFvgyjOHdQk{M&BvM?D zN2${-Sz_|j+vi?Sihsd$pfDFYT9e=dHdB)T)lpv;^Uwzr8D5f25 z5}C#s@LNA@mIc>LW^b>ZjA;_g%t+c zBGR)=ExgkUBO^09$hdJM3zbkI{?J}eqT6!+l*sk0XpVc$mK(M;R%K~bO_pNL8L-{` z{#vNdGJ)Cpux$U)(4?nw4lbcXI=}bOVDbX^SMl8h#&o(6#W?eBrCsbv|8{Ktt$fQV z7gn2`y;JIK*frPZxUtiXS{?z3zlVFu=DWM(UgaO!f)Lo5A_qT+I@YS5p0tX=KZYq0 zO1>0I5#z7Tz!1K>+B$DX-enu`_1KEBxHks~&pBOGHd;>`6Hj7v|NJ8>k!tL+C%BRG zVqdq>dGZWJecTd7htlNcq(n~!SeQ;AGKhGYk?T|_DRCKkfIGO}* zIaga6witJcsI<4NH@BTH(JJAT=PRQS#$br+*)O zpe{0628(9Nx2oCHVCdc-Yra*8yDmwRDpj9hg7#fbE#A4?GTmn?{A%>6?857HyeV_# zn#G9Gu=*Bl|GY)dpq2j>u3WkMc|}?lsv|Xfw?2}%NN}x_S-ug`+BL4Nf_Z?JZpvWx zIG(xyMKorniMU$8<4;FJwzN;jFi4nJKOVMaGlTFfaF`*ew#Inwsp>9YwJ|ZA#fU_x zXqVa(Q~67!UK5&ak%N*_BSN?5E%YtEYbTnr!uY>_(XDTZUg7mfqoM zp6(;2)K|8?IG1iXqdvZF2 zNsr$lX5voCkj3QNt+5Y#G|Kbo-bd?ibgJ@1-qVZfJ6tHxpWv9?qZd7&G03*pW+3TW zZ^fdL+raZ9arxTPKO%p8m-IbDe(kBaBoG!gTVbc3L zbm@5;(*`?eZS8NF!GZK}HvSkTl$*&{d>JFb5JtTodUD^{Kv)t8;QSU}dWH(}{narQV=2$Q z=>Y2Q@O~-Jos($J47a?2Qj91W#E4mR>BQiGyRc#e`o63Z^*+Ee9`3+*)31!p4Gwzf zhJ^kE)h1%xL|HN37e#(G5*M2*K(TD(vU6hOkc}wSK>};Z3)r(kU`e&o$X)lO6pF&A$K8dgcZGZQf^3< zo)kA_dzw@l?O;xCKYp4(e;RSl&yx`_CsLiU^J11S=VvWFinTL#6!E$;1Av|*R zip=a=8pM0)CxFQp2JOt6`+lf7nZUvs3}AhPj%gh|2zE~*SRcB#)P)t=78$KC?SqBc z`Nib(puIS^eRiAXNq{vVfHk<30c*&-{_e*0YJ(qs{H0l1adKeD|RqkR0N6TFIEJEi+qe_vNUb6Q(_>=9YsCp;vi z7=ea767FOQ`Ky<0?tmx2Wb}E3((>~}F5EsQSLp%$BKV=;R!>jcYmj4Qq)D+^)e-UR zG)sijo*9oC^>%ED6-llg!8Vs1efSx}sQV+HT-M+3JmmHhk%P^W^bsdm)3e$ptCe-G ze7SRy`(KeX;=G6|U(U3_IZ_rBXpX7tx=q{Vi-TVkedU=y zkW#=c%@o^Gn=7ZqKz7R}z<7DL^3$=dW+=ZeUL_42&jn=%>;z?pg6~dNG-NeA5iv}U zFU7^__D#k#7LpSu#kLBx|K@?^9DAM?^>A+>q?`wxhp;x42#|CBtCn^=)|b`oCK{&+ zC*Co5V8CGXsW-#1Kcw1e7cZvx=u;jdXHj#w=XGT@m{;^eMDAW2%O)GAf}t7Dfj^{h ztPGO13Bo_d6zG%v_mkM+=cZS`t+@nImfl{Q5k?G3X~|-({0L{*M4pE}$-E)jTAruz z|8TD&&#H4}5UcG`IKJ?4svw>BA%^9AjeqrsfF&BAo@q*%3YUdVJXDpSjANIKkN1#W ziX(8Sn?&t8NuF`5G3~0wzeVPxp6w_aHxm^tDo>4c0C^8NvKk9A)9dx*XT!=e^fv0Q z0=8A7}W=(VPJOKDH)AV_IBNq|Y6VWAMf5)5o~wWxe{R2?9`l z`c%z^Xs0WL3Bf{gj?@ScKHzRtE$wDC!=)uSKh)(6maE)&==b?Y+iGIXntK4j28m#U zwB9r5oac(Ewhpw)%>w;sNH3a+k!9*B=b49V0)xJB^WaH^s87eFUB;dA_&V9ofM!W! za6E;3KzyRx=1QD7^XOzTZUOM&y<1O4QXuE+fpB1oyAWq6>+|w)95VA>j@~-$i}ct8 zMH;(U>3w91f6U4_BWL%W(pcm}&kKSJA?H)P_;@xhvOzSYM>Bm0Oj>i41OKAR6Eq-3%HSY|XSQm7&g8CPOYMyqG+CCBLrfjAq zI8i+4;+gu*PDc{+SYM~oDv+FH zAn~$9i$p4%Ab>N_3f`d|JTGx{i>G{ZfzH;@QDHQMCT2J8Zty{x8Y+;f^&KbrEk74} zOU6F3@zzpX<{@V_^Om|<@j33(vr^e&krgje9YC3 z6YLpdw;znEKfTQoN9v(-`x`fPMb&2Fn92;N8CHT2czm&J0~>$pY+vDIMl1W%y2*(~ zf0XBdIDr1ZJ_0uU7?bNI_`N~+=kZciFE;mc1gh@+eOY1OU)w#RC}Z@&@sx`i!P0%6 zA7fU5ajnnyI9yv5Pnd;Rw*@?=qSj`tn7Pe$V{$9nexZ;xw>;X<8%my`YU<~ZpYtv1 zmKxWk;yEGxSo%#r`n>3CvUqKSM$F5)0%)r+>=Y=Z{}rRPaNoZ0xTg%S{mV4XV&T_j zj)K!3`8OcLVr3P!*>gGZf3l^<#p3My*?rj=|AUEPojP~tQolVDGAs(hd_yNK7;qM;3~72`mHh#T4KYz zb4*qAV8#|pyFpJLG_kN#b zk`*v9biJ^Vg5Lq^kA=?sNLNOm#)<{!l4ib5AOMS@9$e-%Z0r zU@Hrv=Vertnx>$S+&@?xrY~SrJk||0`NiZ@lr|fw15r({yL#WvjUW9Oe3rfNN6ch! zo~VQB{L4T`-;PI(!wc5KR70VjpK4o){BoA3Z*JL@OYsIA*OjLiXG_I(sD1I*)o9O5 zuw^r-&weJzS)BY^1i!Z(m_ufQ?=7kBH2Hs>gqZQhprrSrXQ zUzbMy>Gn2S%#g)~3rf)FmdbM1SFS6HPUl`Z_1rULyYi3L|7eJu>L-6GMUp0eSH%4NoMZucLjfIZIWUo*)mO&N0QcL$*g@n5_szrW@ z{mpDKBx`x6kLt4cK8?^{A63O0oI^-vB!ZV^<0*yk{s351PaEm{uNB0;8$(Im8yQu) zyZiQ-ZW&~DW4BqY$f$VfS2v_rW3m726NV_dVD}t=sztezc&z)cJ<4o{ zB3<=`jLG^Rz(VHi(cGFT+ioRa%T@5&`%j}xg%5B6DZ9?Bhv~szKsCV=RE~c7oA+w; zNsC9=>4(GS6wH_ewWUSvA#yMueKKo9w4FTP;Ru)yNcGb|olKuxOYgg7N1a145&?8g zort2GPD1Pe`pDh0pR$&`Voi}!4_|eb--nWWC1(H6lM+yGd`2PztFR>R(sc!1;eLr? zA?-NOm(aKQm?^e&+7nR)AO_R z_!t**C`mzf>xO~D~W(9W{kcgxmU=54wAd9ocC>b zh4^5-D1q|j3~xV=RNv=zM$pb9R`)x4Y;kI(Dw#ckilGyKj=D?ot7(PXGH%?Rp&|s2 zd6HAZWjjyxI1G9-!H(bqiq&bLs86Re6aTskxG~A;q5D5C+^Opoil;2J10DE++c#;S z%bnR0?pMojh?~R&Cqpt^zrP@l>3i}Ci*6t0Id5X?oi zpEHAlG|Ca0AZZ;^FdZ~nUZCWm*rEKCyUP>=pbAeB;gN8Rya!bAmo-8}#$|oN(-Ge^ z?pe0g#QRzv9MD|0>!RV$xq+pCS{J$)yUJlyjBU)mYN>!dMPDocJi&2XP>u&Q!SUZ{ zLh3(gLJwf>#OcmW#zq&C$)^4rPvqmPR^hU~*Dx1`rwZ_2TPHpHX;tdcO-O!%zFM5A zh5u#mxwP~M?IwM1KlO;Irt%p+nCUhMs5xmjQ`C?8V^oMs0PiRNm-ls_-#5i95oUMt z%3_F3`Y8yRNs) zHS#Q{dANeoc;qHl8Qi3JP8qDtZR*W4?@T0mQw!`npI2$?;J`3GqBGJ)z+_`qWywu7 z+t^cI#;~rAsv&fSSWjlXq*|?E@7O_AqR*DAf2bMCc_6qL^&YrR5)(Q}HGNZmCf_G| z^=sAtg)B&*!S)lBoEMS=mH6RPyqJGTsyXF^&R`Hbh*v8;{0Osg_R+3bKolv&zdD+g z-`6@jqMq2pd3pr=P>w>YX-_|o$`VlR8Wv=SPK;p`z@;8q)G#yI)*q9Z{pHtBuJWnx z3{S^vFH=qF`DAouvrg7O{>0!4->i@0eQM>GRwEll{`;!3QwwcKCH*T4eTrtorPO<< z>;~M)Zj(M0sMVW#QMH?ut_da{mj!klt30*r)sK(}AZu|6?&P29T5siKJ}{2oEVty2 z-r^tJcIdh=27c?XxAVJ*uMw2$ZZ7^UM>t;Dd9KlM2mdB7j&wK<`u3_J{Z{G-d{27` z^O6^ZR*P4yXvhd{e;)%2J+LB?2Mxka6VhcIT=U;Q(JLr!EH8its~c^SJKudOi&jEn z@7Om;O7}ei+2DWW1bON2a!2*QXc7AXwZe)_q{HXF47iB_taTAULg^4b*vREQJ_8f< z9={omPd1J9IQhknNU#0`#6d|j*YQoi1pZlsM&Ts)Y`0yon>VC|%4wEZUtez6?$Bs3 zr2;)O+@IE|RMe;J#c3ODChP-%H6&>>fz+D`q~0G7SmVu|HKPB8f)?%rU6lMl7csg- z(NUXS^-i#dNp%Qvsk7C{DBZsq@~kcn&jt9Xy+oI}U#c2Y^3vP$G(^9ZQZrp=K|wI7 zn3XmBpz6iVPMoeShCo!__Yq zaS>-zKsBRf{mGx2HoR5P6;rVrAw3ngKSAEAr{Ps6!5SL2r_fGg>J-6Yy%-#?(k>s* z^(-5=0M2nU?3P^lrQQ~71mkLG2b@~!hM8!>Vsyh4g~+llshpDZ%9ripwjnu*&@OZ} zr~E5Pd#Bkeeys$~TO!+#;&Sd+?zoDvlc<%COBS;mUK?OC$1xBj1L^gchx6sAbREcD zLjyTOHx-Rw+{}N0^?Tex-A@NL^vaa-V7?fFAT6FcdG{v`_*nv_g@p2{yaNl&7TDB< zWJ~YY{1ym&f)J4#t>J{oOel@PlyI$|`3vLhK~@_C$P3%3J769GkVKVUOEU|lT=@ya zmpLi`{!o3uAJorcz>emG{T^U0u&uu(GDlS8fhe+hAd2)w#eM%jqDTuMiiDmURT`GO z@DqKK5dP1z=27O;Cf45!M=fil$Y-pmvM*5T=#BAX^XL!qZ{66!05HO(N<21 zNK4ZAGWY6!oArwZLjLDRJbTHC4(Ne_#~|!P)m`&A9-eSiJ(+un2ar^i8LwK(5{s&sN@{aF?f?6`NoFfe^?!xh!EFk6N7`d`DO55VXkY;`^qTX-g>`*Ykxt)G+*L^ z*?@`bWZx~MqwaqBFq>_XRPYq)M;yzJN6fg+7SewGrW_tv~| z6(Zs9p<%q{XVR5RLhp67g$bQVeH;B&(@Ejt{CNGNCT}+$a3fM(Bz}$PG*VRO_QQgf zd;CRd41^GATogN}NMsg3{Yi!T_he3#fYH7fEx{0)bFT4Yicdap0*UHx4T2h$rzcJ6 z4$z8e-cI+@{C2smR+st4D_!sRg9@Ja`W$z+<)r^n2%3PX?rGKNOfpCj`OiH5aPPFT zxBMfUxHTv3_wt13;%;gE&wzV|G%J-r*1@Ke;?3@y^26q z!_{2%Ex<>7scG%8xs^nEZoL>>eQT@pTz{ps&tkLJ*@kYofy>KZl~aFJkGFT6b+B@C zWU%Vw`?Ka{z5`~eU%nzI!L7*o_fM9$=OPrFZMkoP%L;dLg(S|pWN`WV05~%EuH+{r z>7c1$%1UH#wN5Rh?8MCb*2$&!4K@Hpw(EveYF76{Yyx<@ctbcZ<_(AKMI1w^p<~74 z``az?5}qdd*gUrTEZNC`D6INY50X%G$=k`NTHaIGqEzjp73Z?mzoDxGJw6g-$1MvV zp5Z<{Z{G>ym7P@_n|taTLgg4>Rw=hn|MR1f<8}_wC>nfcgu&&j(I_p@jc8059L8_p zE;GA^eo&iv2!<1%FyEK~f3og_Pe+0Do=t&j z)RrU#O#gv5j%UecO2?`g)0%Cjs%QKdFY54dgK}?HeEkc`Ps5{pf;GIx@`wB!)zas@ zEv4bBL=0GU3_JhGb4WmS3(AKxTNdQv0TPiBrmBZKpH&B8<&L^;5>p z4404K8u6UG%$#XFn*CVZiXMJ%!9VwUN|u#_`yy2NH@oANU$h?AimWAV6;Mr7nJXsu zjYQ5NS$FFsM&skR$hAmwR4pICie8$;THR3V4P(5ZO%hdNn>JekFM?{u13kUmDC1mt z=J!mcrV^p-b}u{F@ggz=YWeh2B1{LtQs4k8gXsWGES<_+Wn+lk-b&nq#f=L}Rh=cy za)<#S2U&e?Ho_LXta3}DQt$ecY=q1LZR^W?2|KD#z1Bz1oJ)4Dc?x`TXF?doh3lkZ zo(ZEBEqTHJu7sVB^bkM!Rc|jP2{GxsV6xIl-Ac^LHC%CkSF#g|Lmvf`Q$d1EBxZ-~ zPjL(AtuYO`WTGa6~WRt5v0n)L{W{QA@w?NR|OY5ujWwSn`>Q}1m7 zw~zzXshuFp5&KY+-n)OiwU7TQjo)(<1-xtd6%=QdO6(4yGj$n4{csL^F9~oH^t}-P zM=_iW|5BOT--o!XBj5(r%|B5F4#Sq65lqDWp<;bKvO{au{O+}xj7_WzljDUC82wJO zCU)Qn--4$fRDxZ^%^$kR#uvmU>yj6%dkcuJG?-cv(o9$2wU;@{gB%XPy`K`p2K5N6 z0l5MnD9rwmpr(Z}3^F*^L-d9hiFb$fzWEX`{e|j=#jP~*ChKlq^JA1ClzrTGy1G-w zMSu9R-KB81bAan{iFJCVOvbro^hP{`X#Z~x4dF@37CAR7twf0RSK48oQ9v4UEtMsJ z72zLGdOsT+pHOR%0lBZTNira%SEYN9{RKcPXQ=`~wjl!;(FDgu7H zmSlDWvu&8;O5);*UQB70W*vhkAJO#xs6x>(V_qev`itQ|3r3fILiafO+flCalcTl> zd2wi+con*)w%PXg4zTFM!5MUPlW&zf3$=X_{LcMR&ls!ceFEnvv$bHqXgg%nN2{kY z2Db4Z<5F#8TJFP6R9g0X@i@7Krd$zk&Eo)aM+@;5Xq>;D91D{Pf+uAgL@<+sDL)ZOC_Jzab@fdy5{_-PfLvUE#%VQ9?4ezFPk6D-&$11@-8%-0Yt=a zvP7IEdRA~BIxA9L>8QABW9DZ!_-ohr3eS6k_crB}Jb}~TBNr!XM`vLD$>5;(RZ?4R ze(_g3KUQNjU4y(X%qq!~WlXPX+IiS#^_2-|V`4_c^<1y0y5J_G!A$t)^R|dr^9wH6 zJc{Du&UNgFG{Xm-9w`_-764ir=@ca>Qq}Uxn%*qZ!*#;DjlXTS=!0drOG+n^EC#yf z((fivTNiN7frR(O6(jXKHFEYGM~eKH0TYSKBgl8tBYFI~LJs--y4TPl*0|Ww%tjjf zn1|C%@9>OmlW{0<(W|#ntJX!2p36V2_9;EN%>MBkwLN%-y}Wtk(>RQ!Rk;JwSxk^Q z)`FkRNh|ZE2ioqN6&SliaBVgYf_f#&5Hc#Xv#9>GF)F4H{4!% z^Zw1zjCG%nWv$OMQ1NW3bJ1+j8IhT2q21RH2^d&wOW6p<`Cq7|zWsXNGV#6E0NXja zg<4=F1oaA^rNRW!x`$?6WHxSWtIL(s64*7s$#E8iQE%>FHP6bIgQG5c{>FcpwmTQj z1tj1eYq`Cga3BG18XA;(01>6s^AJ&7)v^s*gx8k$5!BUcd6s%(v<+1`#IBa7H9;Vd z6cr2H-PXdZq*_);qBzS<2ck{(YadLN6Uo!yoGfTP`<^{@4A{839B}hF12MFK;x#{+4VT~%Y&&@SlZMOO%0($faOClHpq*4L?JNv>l;bT}3y>k#OI3Z?+~ofk z88Y|ZWXOqCU{R7=nM}PTmOMa*K0q%3 zoG|cSk2MdqaF6qcVwGvnv7IN=>9R)@N*{TN?tvk<2var+8a#06o|I=gCWX12?s5;? zOzJcqLJyFASVdO>w`mpN%&6Z$dWiacHv8{oEr`Fr09&2WRu__B5s_nWoG-j9w{<&F z+q?aw%2mzMJxjK7y|AoJ?LKU0siEJqyj1DXfR&Zo`ahoch^eV0FzABUv7SpscY#pIf$bEV&mgu z>z!(iyWap zx1mu7`LE&I^9{XQM1b7<*d^8|$F+s!g^|1PUxv6@4EI)P)aAGHv$*VqBzm#yu4|^6 z71!$n>li0^ZW6CXSRVcI)TCu!(_fD%t&FP2s3LC3ep+RfX%=r#-+^r(TTDweR)alS zOS-{^T*ddno8K4R_+x`l?_N6YVAb{LZVK)87*?@393yO$XKp8>ou{`z7mLEI+_PmB zBId`a$iJ zQE-bk`d5ZOC7qBt>%HxC>iYBio{H>isNpTY^TIwu={4r^lCrruoY2LSL~!+e!X(_= z{ecacFaq>Y!sqfxPm*L*lF4?w`x19`Wgf|7&EDt+i<-hy)P_d5aiNIA!CRV|bfLUoYCJ4t}(w_wgRv;jYr{JC^5H8qQ(=$VZx-^_eo1Vaw*z zUq_sc9pW0U-y>5^vU5inf3oe4PNQdc9H@@>q9lXHN!pnZn5U2J@APZRTC-HIO^2Fq z54KzOB(_cUu~QtPbonDnut)NUhZUNNi7P~xF13+f{=-5ig{L7Nz(fb+DDiKdnn!?} zbSTyeCX1N|uhS#j6xiTXj}&9ER8wB1C9GP6SfvX+Rt!EeziDN;RZ{xspd^;0f75)obi+wQI=00TlS@h_L%wXXIYRR-Zg;% zfx7{<6;m1u12km+Ao^iW#NA-<{nR(d>N0cuL7_iPm%c*3;9gh7ythXD!*l>H3s^b>~jlsudbJBoHYLk-8!U>o*NS=FW2%ihfkdTVF z_Pp66I)4yu&1EmUx-b&DVZ7t3IAT@qD2v1*p{^r#LC#3H!uRXy9Wq46qxYlcr1l_r zE!h*VXWS}%<%M#nMpRrVqFo*~56Mr-o4{~~yF3(zR8*d4++CO=pz98yRohA7!<`|| z>#h4|)fU6!9qpssd>IVEa;&0~!bHwacm+5xG}GszpEU79m=r&^z8K7qx#!dO0Qw%@ zK;k(AW@!+j(7NBuM`SN2|HuBvFU?MwZS}hc{_2%7m*ju5d^5J?|`E9RN+XcmlNsT5@`EpIJ;-&3ET52R1=Z*XVnbAx z`m{oA+QU++lU5`ezI!xGFMlR;z!p z%cprgFeT~2{7GISDF9QF|G<1f0U>Rwa`VE(FX*dY?kAUs`Ga z9z!kUyu6iC5Wq6}=M;NH%!6~Qje?1NU%Oo9%EACt9zZ$#gt=76Sz>}7AGoZM%Zk!U z3M_U9E2=culZJ-*rZ5}lzN~284YQ1y4^H`<#dMOiQgG&v*whsmzp|rLV;+0_I3wLg zqU&K6?*EV?r;Pv}2^~dze<}GK8OO)`oes1N42^jXUZr%Zb7Y7sOwt69tn5RFLv80ZShVXVp~Q$J z%8p>O@=@GVvlGqUlA)}bPjt?kra?ai&W3i1RI$ymrs&>UJ}C+fQ&E0R8#VNa@SYYR zP7t&YDkO+YYmOM{CP*2ekb&14MbbnmIqnSzppXfR1~P#0U+<9Gnb%e~)FfV+hU;t{uZ*hQE8rgke^&%bF3S z?@=&E=)~JF&JMl`>E7VA{BzHg?3N##fqw{h1^G3H{juWBS&WyP1Fl2@v=zH(%;==ND`x68hur+vM3J=uFxBnp~Rvr0r?&{ z!uv&kfL5IJL!T|T7^bGUJ`!^z-5Y29BGuIm2dZ;~r|M6Dka57-e&c-DFe6DGlUn~4 z(4wAyHGS{!TwvAfeL!MRb}n3_GS^#B#XWZ*-kVAOO+S$Wk0RqkE=*#Gy*epiuYSDQ zsW6M>arsDOi!E}YK>KP!!?HkskX1b~yR(C+$iC9Le|1RY1#R=)}`L@TpEE4dN$W#SE+`xs%E*c z{iKt)u-^3WgbBQUF5|DM)c~d%e_+0y6JV)P(mK_ayF)1*cdQan?37{uy(VngF5_I+ zGDBM1M7P6`|_YP0~|A0*XF%aJd*P9%)HOaXXOkYg|u}LoypVN3L zpH4wTwLd;n?tNJ|I#4Yxi@Cgo+a~jIl!ql5{@Qa2HV^B2~N84{Md{Psa}TG>f7FYK6@Sn3s_}b zSC8X*VpdJ0ylJyH5s@^{czf@bDGS7@%}mXAla)mUxnFsB^-h;by8UaH`szzWMoGI^ zrv%U11NZ^Sj^f8F-9l$dWk~DW#8zFYgCe_MDwyk-jJP!R>q(YMfh{mk_wxG5!D&S2 z1$hUAKNj&FvU5%AjuF~jszn5qpw1Mf*80va^fAUeZSrwTrJ_EeT^Q0b*1BzM-{b+VfL27F#Jjv3)u-k$RQi`0e;R0fXK<3)a0$sBH8kT5ER47KW>4;YLp@AbZZu^_~avQf&v# zxtKAeFgt8%+eQ+sYhB&pZa!2}xRBP)?F&B(qKF=5;QNp7L!~-H;#W`MJg7R4TXj0% zaOfMDiPbdJ8C^9wy=DDy!G}8Q(sJA1lzm$z__vFoHN%7)nmK9n9B>Pl6 z%&SOyjxrW)s3%mdjSSzHq3>!fp(;Aut=q6qg(P`yBj;ykA0pTXE(Ci)O|McZrLKqI zKk$V<*~-Yy+2Cr&b}C;a3dnmDAD>HeyHINAurceGQ(y`z8si4NE3+$Hh}??Ml3uEH5|2@oz*7Q7@hds5pR|p@H9LR zASB+LNgYhD#XxJF?7|Jb>NcRFUuvW-vn8iA+w7Xoz;Hy{>?-g=TA(xRG3iB+En4y5 zJk;NvXYb^E^TVlB?pZN_0yb&IC0@VTe^8hCL+mxLF?kN?FM7zABadR-?Jb)aM;G7g z`ZBCq7xOaR&iS8^*km1YnHt z5q{@zyd@KBj>t35#y&WHXMULfy~L=mVUG9R*PlH`#s1u@43;>tXFn=j${p@G%dQRR zm&BYt?rtOyj01S4O*^1SN=W$nt$F5tL=tc{UMPWw8e_y1IsrkMYB81jASGcNCRh91~Lu6E}{Wc%l}zF zTg0HYi9YIc$H0xWGfQ)vDy-na!mX`V=BoX>erKs?d@9`hL4 z@QeDcqFo3TlW%)MaN@c=@iV+%Djay$Dp_>v06q-{AYO5SU-^y`z%69-D=%=c*ZbU$ z+OtTNe{(9ZQyLdB2pCXpX8yce_%6}#3cAhgD3WQy+GCVn^X`sHoaE6Hudi%8uZ5C= zs<=iLUrP@HIxp&aRRJhfpM3FT{qUTR4(m|7wfcZye#s?82S@I`ks~FKr->nnGY?9h zCA%5jR8C+(&5)!4rB&-~;-P9Z`ooPE$!{WHU1lNzkx zm{^jmT}S{1uCGz|#Kl~tc93^V`-C-T0gD2I#r<~XaO#SJ&V;l@*ku#q!c@}WD}r+{@8r-o;PGeLcyN45o@yR zek>0YLRJB-PY9fe=>}+Oz*!lA&AJY3R$&#eZes&r7H_=4-g^cT|5AJPiO4^cHYicP z%w*5`v_tTkUuCO5pBrW3E@@MpMcoQF!`B)2ti}+pD^2Y(9h*)HHuG$<6?SLt!7?RhWG1(w(evL zzTp|0OW084EPSG+JVCSYtfYE;^ z2P~*=0(A|&W9aV~=KZgBR(W02!SbTLvCobL9`=@&jgm>ZAO3wKEmLH%^Je?!zL0c6 zF-cfUE^6oV)Bqc>f!zlLK;}Q9U^e#xZ+A|Sbl2XPNYc^`%r}+UeZ$l(Xtq(WU&2)! zZTx$zg0h(Nt-WeHqvI)GIf@2<9UH#I3T0_c_^2oes3RqTwtZSU@G<1CYc~EpC7S65 zFiS>%h1?JW%=9OdzC2={TAEEXqKy1PI$G(UY>Ky-jn7imWM0=-Kp0u4meKd%B&ol@ zrnG5Q%E@A!x*o`qJ$oo1^y*CZpFYuOUJM}Z0w|^bgS1QW-_owZf23V;P5(%{+F!l5 zV(o6Q;v>h!)D9DCUj5+L-s)ZLnNf(ND$^i;JOrO}WB*EhNN~=^cO&jfr8^uJ-y%FV z_4`)%dRZh*)HYRj2UgO?6r>dew4|mE2ZwQDcz|cetjui8QEQ%gFz8WDgTSh-)h{RO z_g8p-#EqF|g6?MxJ)lVa)Mg)y_Wq;5QWmeX>h{lVQIZZ&tO1}UJ9yu#x=>3uY3K9$ zz;j>?gd&${2m0o&0ls9Fv~~8DaJ}BOXt79BiWQ+K^YV>3eTmPNPXlcJEz98QyF0~JzcW0hTeJHEA_|U!CJo4RPcR5(rt2CPVSFpycQ&e z!CIo<(1i0XoA1)9CXy?(26;UgDMMLG;_WjegSDs*^4wa?A*E zt^)4DD2XOtR*`++lp`ZvRgoQo!S5Eno2UdZCmA=Ifg~l7sxm;p_pg?~@&j+%FCZa_ z)~z4gS4M63xI4?AXcc?;_+|dAI-|U=Jo+q?^rgI@%Sg1s9DYcHFpNX!IO}O_)p&4o zf1Ai%l&b59vJ^L9*m8d^78D)|wqVV69~V(qc$tRhZszahVnc#S=LkU&t8&4=Ppw{NxUS;N4 zk?p&bY0~q2RrHzG?W=9J07;qI0G!vGe9Rz5rE1#|c>Xt$i?!ZInRDz{E>8&aBnyMy9}}u$~n*b{UxY@ zWbBSOO7AyQokzs9{eEE!YNpXM3ER;#yq2g@6M14%vME{l|m z)U1+fFX+hQv=|v`z4+?xl??sEvF^yJz1Q1pl=dd)<~=d``axRuVx55^ve3i&o?G%3W~r-n!9Dv zn)-?wPt7{!`Whug{j5~b-TDPJ(RZZFxVx9fd}>@bNT@gqP0C=(Ij0X?U_{kdPu9v8 zaj4&3H95|CM4|1<9rJO*ahCe_)O5-#Zb6{|%9m3h&l%cb{D}ujNGZ3(Jjvv2jAWyNTDr{-e9oqXAUWLWnj`46 zSBIf;=SJA6_DkAC{xC}^*0M#u{v$T*C|-wB4c$W)UhhT+^(Ac;-t#4}WG3s9RhiAU zHh*kmi`e(Hbko3I{M6<4L%lrxa8!KG44wf5`a3<=)%>-~Bs_g)dcDK&YlEv(Epg|Y zDMjp(MxFiQjQxSaINoc5HyBdfTm7~ke?6Xp?-u61K02YhTtd3ivY2Ec3_$z58{)`u z-=oeCF09JE&vG15mP!9$r-P&Rwd#kM-AqHozB`fB`<=%@e8}DiO@R9G`-l2L1JsYj z|49A(R)Q#Z;v`RX2IeW^Jw|^lk30>q>3^8n zf(Y!K6}N3sbNFPl*K94^#HX0mY|On^-@JfTM82Gu<$&K(%)|T4&gs}v&nn}j65v635>i6jGS>Z^3l!6 z{RVGMjBi9XQoz+&drUhT=532dT@vsfdNUOtoBKN@UTBH~ZotC~YQHhMT^8|&n=fY+ zbmB(mxaLjR0wL@156+h5S#MY^A8rj-plyH1j=t$@gS>*W^{IYY z2hOyUp8o=SPBoklJ6VnoKOjq3dTzs%owD+k2s~1!@Lh1+XX)puQ|x7S>PyI*ArkTD zn$GJ?Zf`q0+yLTXKn8%q;y1*DL|aKY0>~{%il}T1c>gTRyLOHNV)w>_u^<+kUg{{- z;QloFJIDL^`Fkwm!}wixQa=y3!s#ujbE=$Z73J?nU|{ob_D?{6QCX;EJnqns&oGon zJDKB-Aw9z?+ga!#1g!rVKSrRJERf>yRRg4q@R6qC7OQ-=+m;! zE(gRKr$hE+5oI-DXJtO)dH~r7)4RdX(C9&1XCNPo>?R*8gw^d@PHL|+Bk!^x-(Eyg z&dk2-To3cPimR%`>REub^&vTr>b3zv*{-@#I{aD!E3$)@TYs7%R1l0!J47d;eH2P@!VB z23OZaMGA#Nt~&n8UHEZKKcArk5ilqpYyTR1DX%1v3(1AwQ@Ytz+y5{& zfWxH{U~K*s-}hgP4FGyvMBk;}fS!;W(4!-GJB~1c;GO-&nCRD7hWg^|;oB9j(9lU) z$EE(zjK?fJ` z2U2tL#V_4s0$i~D)U_{FXk4eYvWm%0edE;?IV^DT>N_arz#W1@@`(ZQtq(|L4&BCb z05E3Xl*%P^4aaO9R89|mQ;us~{voKSX&ynZh=y@zFUN zibSZGCd6e*lU~6cdO}6^7Mr8^YG0585WZ!`i+1({hlp=E1iD;kQ=MA>ef1fC9VYk zK%HSoMcCjy>XA*`quX#l;x3r0#1Aph#qpE@2=Dq>S7~COETI83nLWOqbqq>yS+rIM zIyb`1PS`Ff6Sq8cmI|%#5Fchs-!xvD^Tt*f>0`ZlnH9{iOWUne18KIF_o~nM{8rRP zVf2ec+pSjHb`=r_x5TANJhSZr{^6Ce=35v-B|g?Py7L4RMLqx_@&QLnYrM&*Od3ev zt*qS8Q|8c$0+^hPU;9n#md>AU@J~|Cys3Kx6}1y+{=3^m@O|sQG~9nK?EcM>sPal- zf^Zti?3L{b`v}B&ksm%h|Cy=|29Cn>5Pvsf--XsgCGmiA;IvA)K@@EHb=RTb9g?Ug z71|V^2=i)`SqBvZ#Kd^T1pT@Rrv(o7CO|)j50t#ZN!9ELqm<1r86~`*Kr7%k09x51m6_yj7~YIU3t+!^O9l_uq*PxeDXeQY>dnu@W!Q_?`6;5p11u>N ztvW`_cuwM|!!1jUyvo2zg%LmJP>NX2T~XhL-IcLkku#w497?nbxRoCIW^QcRk@*B_Bb+(9-&?JW&ZSwB)CXG%kocvZ{QD z_z)iJKbuPG#6g-B0d6K}4j0{Kp**DX8_I8d1Y2fP(odlK1H{q=V09_se+dHy!@DE?e}OR+3TPef02F7B4iHz;TKP^Y zq2s5+uFAajb?;n>7V==HU@xs-jc1v7T3h96^=UO-p8mH@_nVbffYhWy9mGACQqJ= zgWBXA`4jIu-b>u3M#VG_MMr;LkyOyp8G-*{O}uzaxDf4nSq|MIe=16~%ye|cFt01L$G zAqVQl%TlRtB6cGo^S=3{E!}uo=5M?#IxTlg>=--PzBzqS9m&H(P|Z|`I~s%L>Nda6 zyoG|@9|cKi(Uj&2DTm0epmI6O zv~}96a*!jVmT7?0PmRuC;NO8vU09~iVZ>^XTl~ctK)%_!1W?DBr2RV)3?mI=vI_N{ zHH}0*UVe@C4?gD(yeXf5=jRGQ40z@H%$3?zDrX+~9+SP95<4)WwHh=obx|0v(f{^7 z#?go1BCC`|2$EQbNr89Va5QT-98HNR@3jW$njT)Ku>11!s&=6vT(Iuv3&$^5iHM_* z5a=P|xS}?i)^)y+MBelnF?%3X+j zq;0mlX2p;d(7oaGO`oN)qy1k%DC!>|r1sx|(DV%uvaI)L@9Mpr_0l6=1yo8EJA z?l1b!Pl->H90EzMieoRF3qRy3CxcXzr=&wAt7Y!YyA4<_w%H`P@tAmkA2xka>31M9 z>;$|!-+DjnhvD_gAbJAyql_cXB`88>^`ZM}CK27XVe)tLAV&p%eK z3k^-1?HIeKj=3Rk8!0!I7`_-by-T$BK)T$|ynrvC5|9>;ZXgm9OVk2Fs1?ULd`rsw z_fQ4D8PlVPJxgw5=PXO6QOI=i!hIv}$d*HI`{+UbHLcE|wzU4FAfi1Jy^281q6<1d zXx!hiiMofJ_8oVXoCRbdBcBaf`t#O_o%F6Ljhs$O4NjIuQ36V*pun0V#BWxqR;=HJ zJZ7P2rAy?w?BPau=tp|1JgPLa#kv}-C@#^e}`J4dXy$H*aE zZF@uMRAX|iJEZZ@N|5umvmrjB865QNTZ^l;lxDYS(0Bt<0z)*%o#!Auxhk9ybK=WCSCxJr7@MV{q zM=e@cPX2RLa7I&qf4o$vE-ecKM=W97i2F5drW0K8K>E`z2YC?Rr7Bvc?f}woQhb5P zVXe}5L4Ij>b$z!l(f$JukSrYK>o`iO^s~k%2GSrJPvAMcsdA=$>E98AOWNl|PZ&?K z0QqwTe$c?{NK8^Aglon)a%g4{Wgvk?D2ae}$u-D*Cxsd>#v-IPa0PXoyP$ziZqq@^ zm)zj}P7qZ@c}(lJ)6qfTu{0h3nq^VSfNMb4Tg)7D;I;ZR8&>Uk_tr~x-pep*!=#y@ zM5_P-=k|!09P3^WP<>m=NiLQ2F(|n)G=;6g!}M*O#)SFq)_Y#*1JAY(PV3;b_QcDk z;<-s@BgVn z@80cF39Ws>Q0Wm?FG}6>nQ3yPq&Lq>TA*^vDql`a*bwl)k1}WgRo8I0aExjAGA!D} zrj5Az&GtK?fKF%cv6#E>~P7%vSYU`z?}32R1{jlRO(;9(uA>*-Re{w^0N$DMPAR zV%X}RxoDsG#d9=?3A|6K17*)1C(Y@mdDI6K6~2e`^nmE%I+Re&x-(;mGp9-VvH7Tb zC$`6deHe|j_{9xUBW*tu6Qij%MLQqPu_b!-~nx>?+do};%mKKoO`=d6F5*ff;k+FOyV+pJFnV9aw#F$aCrLzX|v1i4zCGJ?v z=I;}cDh-U@I2!rny+9ZsDhjI`r1_u1GPE4uy_!% z4$emQ9EDk`6#Ahr48N8@mWmwXNUWcnq;+tdTaJz7oeo;mNM4O%&WUS=1B*XwD)|OS z4e+(mJnjt2@GY^!m+Q^-A1IujYyPp)2EAY${TpVvvXIoc>0i1)-|XRjTG}JTAF#(3 zurXES)RU=#2F0c&RCK){KH1BN<_O?-=@FVG+qcpF$F}36AClypHnyn5W z<-V;JTAn725~J*DUS^B&u=JrRt4X^>$Ed2cY^2tj42!S=__OPgQe{6w+obI-b_F%c zRh}Oe){7o_xnyylB;P#?Z3kQ5JYNPppUmyK8&J!85Pft<<7Lg6|52hwOAn}(XD$?$ z;-3al1$WQ6_YOTjs-*qnG4*5bq%kc4WdapFzj&`f%U`g#;8SYcCbd_0$$6<1H^zIC z+&A;>nZ_9y|ER8`&mUX;uA;Xg@>`Q5Ttv?Jv^Lv~Wzu+f%XqP@zoBHCc3T7LC0B<8 z^RVre1?=ZmOC}VF=(0pP?v3d>;CeJ$a9fJa1ItB22k|e}y~p0e-Qk&*q!S-}L=RFY z1VR_4jw->I3el9{0e_Ggt@wg2oVJJi7pp*d3`=MoM~n4_wD)mY)$C7XvCZLW!R6M+ z{^A--77MaYvvjxJbe13#=Cdq+VyXV1MYp6GWspl3C;Ne`6B#4@v522F0=J4l*DG_u z@kjy)`QB^l@>Sg7OH@1I{806Y5%GY0_knaMy({%r=Q3Nt&Mm|PkpFBCAFb?1TEbeA zj`c1>CcpK^8sh!mK%GRH$D--a1bqn&)smA#UsCiQPNIJMZ}_Mq@T`kwIw(Ik?NUbN zOLUUFHv9Prsul;K>74L)vB~)3vjn-4HNR(?wx(+gib8=x!Nt~VqmzwGyBy(AmpG1X z!ELtzUu89Nm3~g&>PBbVbG3RcIyl$HwTSW+HTP+KKD)2lv0qO2np7bNM%pS<)B%?2 z^2#d+|MJeniULg#cgQ}3nm-t({{)dc~l9o%9eBfVZq&-3Rx#hTsXk>>uv z$6R{Vn#?B{PVP-(RHh~8f`bsq(ypn0z=M}r<3Zro{lqBW*j%|UY z*n69LZ6OB6#;Srb=scyh`t;Ro+_UqqBZE{Apr{N1XMsmdd>S0#9lBBrW)p*<4tkwP zn~V0iIr}br53fLK>xwGZ!rA-|NJZ4FDK;RVF2Fd3pY zX6`o_$WVR#;j$z^%B47VSfE?Ut483^TdnR+p||C(?3nTD994y^_8WfwgKMga*W4Ic zsT!0o!)#Eg7_LN&VnPEtwn{~8;1YH z#>S7yT8UQsFlHSdSzqz2^tEYENRn1rXSXij>)wYneh{_uE@>@%{`ORS7AsN$DEB3i zZEr&LhMcyWkL1j35?;{yYUx<(jGfY09g=9!IMuKm^2!0GcKwB0}x_0KDpD&5*Z~4^;c>6H6 zD$LLIAA2)nds&wRGOVuFwiXqrTp|4@QZKHtxFANnYrI+izec2X{ohwj%EM@T8n39K>J ze#zo;LJzfD*C=IN6^f~naEpL>*&(0`#HXWXPED=aYgHezXJ54Cd%YB!Ws6ZLADh2q zItCvvkoOnQO@`L1R69)&w@&juwMhQLdz7x%lUtses6nP8HG_$Ddb;1{1*0)d*IgINy|?N5E@o_rSL^*P+`yo*h}ACc_n0Nk^>vB-^$^o>f#QrO3-VT=maWU7`S;*WJm&HOcS6?AwLuxiTCV;-j-#!qTes zJkqgCd+|vV`%sfU%OA>zSr2iM3+eGz($V{UQ0F9`lVYRgymc|&-Gtoo-1w`t`Bbn} z{o8?M>EB2e4o*gC2RL-&B%=D+d|uSx$HD9`Dvwt@2Fh5OnqCA; zLne4=&q=-7+{oPhs1vjynTsYR4f@RID_=Oad>sFLjX7btV2e|8M*G=?cxT@mgrbi@ zL8E}+&{0cvuPi@mdvwg%I$SMEI!Az*y>q73`Z6E}8%2l6SXig=Rirw{;EW zdi@1TJMtIyxu%NCzQkIvJvu(Xh>9}s|dJQ{wSagt^0IPKmxI3=~N_fJB1pz zhrf&133~4jL#IaZWwNykn@+LS*pr$HR}fp2wRK4Abo)6P4p;wO^YnQB=nwW?U)X%?QX@>?_B6K^whgpPHuSoqQ&MzRt*yp)T z0WufaSEM1SSAN4D-lcmvNu$P|roFnw+Zo-f1rFM+zf4$3moU#Ad)!2e8x7xFE$;yn z<9%0k!r1KHQ{1DqY179ogmS0K;nEnC6z=5obPw&>`=M#fbb-^QJ!$^cLRt+B-4el~ z>TQ)})%l@H;Uz82uS;yD%*sWY)33-uTuNeWjtB`l9xTXFrK@AnQ`l5}+$k?>I?E7)K%a?E zuoYdJ6?(b^e;Cokbw9C~+`=#_bl@-Q0ij|o9pK<)|w<|E3Xbp-g#Xg z_3tyfh}ifHjUA?nQKTeTVZ&2$Wg21yhA0=5l3I&RQMkwM>g||$$&B@}3Rw6r39R)2 zVf{g-RETQ5&r$g`we=8644Guu4Bjj8qqZpvKC24;S+HLbx921`Trx%!olU~>x6|06 z*ypR-Sg`yA{&wLRXO~9K1O5z zU5Uog6|2NmM05Y?v{M7d^|JCbAw7qCm<4Y0bas$r?QwB#VkNImy~c+H0-85o4Z( z$HidkI97*|G~4^w9VgdJkxkqGoQ}MJl*H~7E16)vvA^;(kL2?xJl@00z})ZGiln98 zpQE4O250fcXxzMJ8qBg%<{9tOB8}H0tX&?Sa>VAEZbo=M)P)-4mn}cYYk}020QT>F z4O;2;;mIAca>+eqdOt=-6-iXb-$|p~U}VRt_WqwyK7wYuDazpu!=y3(la2$ya>=6y zCXAROFTb6keosD)9RI+&lGrrOy*;TZ!zCzou0h~3;s#wb_60(%p~52+zu|*+6~2bN zIbLe888A-?9RGwO^E-t%&$D3ROE%LxcKT2Ju6nc%p$KH4ak6oJW1aoIq$*uNRu=Sf zCz_Fd;N^L{hYv&I$o9UWI!MEqh4-~E-`Lrcc}M3w*Ww%q=2-f@$)VEb!XdnWp_*@* zijRvULDQ*2Fk0#F)RgNBSV_wjE;lQ4U^(uj0Le_=O-w9gKo51b7!)72wJx}yxW+<^ z+dgOuG;3~f^oemuy4d=Pp65P7u$%^_>uSw|eX@!8XxjJdxmKR7JfrBkpeaJCi<>w} zSMl=m1;5y2g3gv_i{UH`2N<62cM(YatfoWB@qwuixZ|8qn)2>J+(~OjgWVZQWCZ&3 zN2>#whi`o-=On?59E06T=%nLzg@syM)pt2w=@(ZY9D~KdHQ|_zv#vgebeO?nN9k(0 zk5%Mir+@A=5}wFdA$AHDl7G?4Uoxw6TBi||+g;bYNnq_M5v>&xd`}I7ybr>7#Td?~C)dngI3LNTT*~j*Exg4}pD+)M zs$S$~%(4pw8Px19DX&kK;Dja)2`EkPiXV>vU&#aC2|O;h8dFbOQmb#ox!p@CF{hj! zJ8axs!ytMzV-ca4s36O@hWbvtos4AH&T6A4fvCv&~E|L#Lsd`y?S$=bH-R|t;P-`xyRCEq(DN9 z)vz;;_RUi}@Fwb*w6HoLBd*h*VWKN8Ovs4z;q}6YF*NCrmXQmQLOd?3@5Yuj67<#Y-Cp-#W!a|cOsj8DlZty>`K;zDosBV8BLG=Bbk z=bq&SbZ>d;6ZTq&O>N%veW`q$7qX#ex{+f?Eufl}+oe)+k|4SwQ4^8_o6O8c#Y!3G z-n))sF+TpQrB>t$UxI*B-c9|=pLnxRSe zl`?i(GR|hEFa0T^!a?>uPWWRO+Ug`hB*yk+T7%sB^rf48|EFcV>n{u*c;%*+VMO%j ziBtTTN%Wt~fDj@Q$av|`!I2yo+2ClJtnTE~zy~&6G;8y{XTsW2LDk@q={g{0;f0IS z3tGn)cho-6BZsQT(vYAN9OoGJI4+*)A_=|?8ATdcP*Tg;ZI2+TxB3>Ts}|P61NIXP z?_UsF6)WThS$GA6v1~Ww_~QsGn$T*EA=yR9A?|)zv=r&!$_IvHAL0dz(VdDl%V}Km z?{?}LUJM=kQ8FFvC$yEP_NE~7uU0H|nMMF+FWa38`C`r`r%q?q`L2dN@X zYP1CkG1CHdUx}uUYQ%>=q^|I2S4fHvSt-p1q{19*0*4u^UH?P}c}E377IEjjow@EE zptVJ6B%B(-CLim?35IGyrw@TVzrX@2gG$4}`C8;=S#6ZRZTG7~nO zs9LR{UZ!0jxu$W-Q!n3ZNl4g0Wan6|HTPBS36F&0Zjc+Py`8VY;Uk3$M4oy=2X}iD zQFd=q*7w=@> zE)?!fXQ?`)y6Vb47}bpR9PwCw)opg)aGrl~V5lqSZbur~EbCz=r%%kY2Ge%;9L#pm zr&A5+u9(KASSLjm1I|>GLcJyiM@t~~Hi&2n_S7bh83D>L^-TVO?;alem+^}6O66_A zPnho(vKJpx#V>h;!fZe!VAt({usA*6UNtADt3c`AZ@%WOSlwSm6uvLz3XBGQUzVu+^rd4!aC52;f(P9D zFBFp51aV}2E$MfwYMi%B8BHykVGj~H_ZcdBn-{qoe>ENGy{tY(>5zz~eavlK>Wu45 zi`>`D;pth|-R)^VuS=AYY5tZ&qivld6Zb7YCPs}tSwR=jG7r_)n_i<63A8ob&d}+h3CmbHbT6$Qe#&R4vs~6*{erww zs3RB^p&~up6<-)+Y7vs^)k<;|ac<)9#FdwOp6kH17*F>^sIeupW_Gjbsexi~SY-tZ z@U<{DO#4NWbd8K+}T2ecp>mew{=xx>*6MjfG?V+7ZD2s!pZC1 z45Ke*+;&@#z7)*tQn`VADwD3SFIlwNSDmU)DI=D}TIDb~CYJY1cA)Dexx#Ujdln_@ z)&o~9p--Lg3FJr|l7cUx#tvtH4PHPU*hjM(fhg4F?cU7?=pOr9Q%}5p)8MKI^U|2AM|_+eCY^2cl~sq#@& zlcz0MC7MTQBsVZ{66p1S38Hz(a!S3&HZQ!cfZZAB;1#ziVKsZfgIu`q8ji8{|5njA z4gTDuH5M*PVsxb0{0l0&KD-V$^6e;P(v;S7fy~isA8HZ4i`lc7rJOf4n%o?odxvDV zlw(A-Xq$-1OjQrh8XmDgEJu5@W1@zV8e9V!4$9nh&J;|^-+YgSLAfhw_f^sa|=$bdACBay|~YV%QIP4%)0(%|FVzEoHk)viPt1Fd$hAD%yCaEbE>t) zukC6(s2wtP>z7e)3JK!xXGO*=$+BBnAMSQy6)MdKcK@(d&p4l9cl9`PpCkczdnZm5C=i z^|KpDZ659~g{3<^_4%#^^QYo`%=0YMfsB|_AIsf5rPhuhe3R38Kqr1rLhmG|6=fvQ z{*lI%vGH;;#3jxtuu~y;NfXWy*7wlztSi3c9d6~B_o_?)_j_YYUq!^XH9lcE0DqDlGxsX%ajuV=VlbBcrZ1o$y6%xiX^zB4NYArL} zZz@8Fk913*EZ~>a*YSsN7x9kh)+>pC;&RV8yWTFpmJdYOXAX0{I62e5>2Ru9g1R=NM)2$5p(^NbeRws4Op z-%r{2(XoiG*NRrjY+7~nZj;;0@8#S@c*Mv%C~f&PJ=}=nf1a`q&Tdw#`?qdYQ~ztq zw5&h4xOwpZzo%@^V8MAo{_&CqXLeWzRTZmPQA8^-Z%GmR6vM(vs`3<36Q-SK67e@RCCt@ofnRorjz@`&uY~zg`Ao8LAX0 zhWv)%zih+SNAT(Mn0*3d1GMx>Qt`G#pkum&f0Pe&r5EheJdncGYN<80>nZRa4GVF4vcN#yIEc(q5xK{p~ENmg+I_wo);2^=?sIP9%-=L z+i3%`=3*gwkDRw{O@cpB@)#`&G5m7R61m+lD0DVA`hBG7t9W4aw;_z#t2IJHwglW= zbAtLJL&eqEr}-0E%cW_2lbDg$<4zoUs>KH4%yc)wh~Eg>kQb(A%9X4odq@2?+wlp+)?u! zU-K>Biq;BRU{d<>5wv6eE1Kv#b)1{eNNr?|W9o!pa~7=M2-gKCS9IW-$aiZyxn<88Dx#Jr57RaTyV0qer#fenR%FFN z?`9)Iuna7!I125nJ&yJtluhub_JlQu{G6M&iNeCK{08kQ%gN2vh&7+~TDiO~?G;k5 z4eEEK->Pd{qjK~Tr6Dhi7~6Dc!k%tS&e~(fbSv)e>bbQ;R%Y7YA*djCGw5C3xyp-< zl)e;d=RVWFTBF}!n4|4C1n>e*4WuW?tT_>;4N6)LYK)%VUJ z!>wOT7R_^Y>28~uThT4SEm<2CaLuQq4(rH$6<6>h(#QaNu#7aDe6R>rFCE-OH41;( z+R@9e<4fQ7Mb`eN-T_cgHin zZFD-669>&ov}7#7(rUt27|ON-S*F}aVcNa~Ew9?HW?gjPA$y2KP>SiIRaL!1eOK4S z7Lwq7aaRJqW93CBocZInU=>4B1ze2|UA4Uav&H%Qrajv0i+7?cvki3x5f_@Gs%<0;e$|cz_YzBxH<``uN#FjTkLn*vhx5(A zwafPQ&GnA%mbSHrhn+LT{r`4h^Ng1v`26bemt~g&IxQG;+tVi=Q~B*>DXd)846iTH z0U)`@q`~b%E(kx+=+{>q8}=JI|E*Iq4~J_1$5{z^qE3OB~o>GP*Wz@xq`Fe zX0E2=2F!n5umJi2*D{*eWRyG6Hw9LIdJ2nrAu%uw6;6s>IFC(^nAoxJnA^5G()EC% zpFeL^+B1DrjO;l3qqc=AB)PS%`*!ImYl8^d>`LB#9+_dGOR|N`d0$;BOTsYnZfBIA zd0QfJFa!(-`&Hf*-Zoxba8sm$mx6aMq8HW6wo)Z(A%=^i95qxl{LQ9cOd*000OswMy! zY_|B)KD#r=F7Co3P7=))8#5rw&fRKhPUegV#Ou&W4g3gff_U4+4361=#| z`upk+R_54KT@Dhn*a3i30sz1{M(qE!1GYFA&KG_Gi46=5fg`aeY<*xMNW{N&0~i)* z=YtGjoHBG$2ie+#v{ljAK+Lb2sK+rN@VzGQ_VM^YLK1I=s1ofi92H|d#KtM5*J$@7}rxOdk$liy6Y=!SgheZN^(tvSNs+*tWc4IJcw zuTr928un|~`b<-{@2!pv`O#IAA>Lvq(GsE>1ZL2yrRLX7hm$qGZ#m81JDHZXU)qFw zOA3@Fp4RfU+cfyjk1!R}uKx2h`ApPMow8i~cus$U@1b09>}Sg~W5Mp~$(#)mO0CY{ zK+Z%4kteGN(O2i|BDr0C?du8Z49WtI$b5cxRmMP$L_5ksdzEf7C0qMSO`24{>^2Qn zLqYxP-9ag@hrQU}kCC&^`?vbept_CbIc0e)Zkd^y`V?g@MYRqk1q(z&+f>bXJ%Kqz z#PTXFGuF|8I|M_Nw6KfPxWh3CDWkXRQ@z@*+fSUU%5IYl9FEyvZ&Z=(P&Ty=X!eGz zVjR^Wre_k}v{f*4(cJX~6_1q<5b;TcRrQCZ3lc=lf+AR|m|q@`$U%;ZYcJE?%oX{% zRoBshR3g-B;IN?_uAzkg(qwgX`k;obE*V_YFi}Fz9k`25tW`|YIO5w7RhcVDZR)X? zS}QWTqB9jV@Z_2N%{<*tc7r9bK_zP|{lttalVIuNx*ucpQR^Lpmyne9v-vUWscK%) zk`k|gmx5I;P7kk5c{h1cy03YV3dqHX%S#?WY{Tp$p%1>+39}Xw+A{Kz1$EQ6L+bUk zwiNu$Vstm@?`e;69#El_FjWeA9B9gGj5OiEyMe*>l}d35?4dUOqr>$XryZ`f7M&M1=QC-0!`h@>{A6~6XEF3$lgJ%uJONf9{u$<<|90P- z^pN<=^B(vgDaZLh2guIQ;&8sKhmoztmv08JZO`VxUF%YFgSpT>dCm@eZSd#|diT0- z&S=>a_M3oer?NkU8=LR5U8!@K;!LXie17e;@~g^BVjzv>)aYjK4%EjbYgq8HYL4)Y z6>)b^x#YZ@*d3@yXYvH&Pbf0d^K_V294u|LT~8J0jOLoYpHqik+e!9V)hVyxtJ#3> zzlY)ngKr2@Fc+Vr`RW&in}qXXv*`_Dr~mMp5Sy%|!oHvV9%NCrt`Av$Wb2H72%cUa zYF}DOge`>PN4ea`qG!{F8}TfaB0Iomm1ccEw4ND-bP z?i%W4v)Fk))~cz`y<^K>t@pvv*=9R2&B)5uqU#o0!4X)DM-HL+Y z>#^V#wr!Ng=3GV}K6@=T7PJQYK7l8UJ5x99=3<7&=_63u#=&^=e#_eNO+MyN;gmsP zoe@K7O&CSsFS&(>hWO$l4SX?(|0TO{gr{fYpa%Uc@sn{!p;8jtHom0In>{t4wvOf` z_WI+bhD&f~(QQSlo-|NSphGGHC>sY6iW^?}7H~K!T%N0v`lveKtEJi(d$O@}7?3a# zJX&e372&-moe|XvOm@hKvR~7|qzG50$1bT7wh^IYtGDph=R|jqc`bCzj!*!{!*d*Z z;E(f&dG(4u^TLx~2v0r0rxr%qh-XV3Z@4vTSM4}#xI%LpADzSYxod2h>9~Dgr`9An zI3$}NBPyZ3+Wv+te!+2_487l$@m~@CKu(nhz|VApG)BKLV%QMNKm^j?7S2#V*j|`juTJbmndjxZQNskl z-?IFlS^8d-*(dHs7x939T;pDf*=6mfW*J82Z_l-tV)i<_saq+4-Ogt($ZRxrgBMc& p$?yNQ9eW?fJd}4o% Queries -> Transform Data +2. Once you are in the Power Query Editor use Home -> New Source -> Blank query +3. Go to Home -> Query -> Advanced Editor +4. Paste the code of the custom function: [fn_ReadDeltaTable.pq](fn_ReadDeltaTable.pq) and name the query `fn_ReadDeltaTable` +5. Connect to your storage - e.g. create a PQ query with the following code and call it `Blob_Content` +``` +let + Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), + #"Filtered Rows" = Table.SelectRows(Source, each Text.StartsWith([Name], "powerbi_delta/FactInternetSales_part.delta/")), +in + #"Filtered Rows" +``` +6. Open your query that contains the function and select `Blob_Content` in the parameter `DeltaTableFolderContent` +7. Click `Invoke` +7. A new PQ query will be created for you showing the contents of the Delta Lake table + +# Parameters +The function supports two parameters of which the second is optional: +1. DeltaTableFolderContent +2. DeltaTableOptions + + +## Parameter DeltaTableFolderContent +A table that contains a file/folder listing of your Delta Lake table. Power BI supports a wide set of storage services which you can use for this. There are however some mandatory things this file/folder listing has to cotain: +- a sub-folder `_delta_log` (which holds the Delta Log files and also ensures that the parent folder is the root of the Delta Lake table) +- mandatory columns `Name`, `Folder Path`, `Content`, `Extension` +- a column called `file_name` +These are all returned by default for common Storage connectors like Azure Data Lake Storage Gen2 or Azure Blob Storaage + +## Parameter DeltaTableOptions +An optional record that be specified to control the following options: +- `Version` - a numeric value that defines historic specific version of the Delta Lake table you want to read. This is similar to specifying `VERSION AS OF` when querying the Delta Lake table via SQL. Default is the most recent/current version. +- `PartitionFilterFunction` - a fuction that is used to filter out partitions before actually reading the files. The function has to take 1 parameter of type `record` and must return a `logical` type (true/false). The record that is passed in can then be used to specify the partition filter. For each file in the delta table the metadata is checked against this function. If it is not matched, it is discarded from the final list of files that make up the Delta Lake table. +Assuming your Delta Lake table is partitioned by Year and Month and you want to filter for `Year=2021` and `Month="Jan"` your function may look like this: +``` +(PartitionValues as record) as logical => + Record.Field(PartitionValues, "Year") = 2021 and Record.Field(PartitionValues, "Month") = "Jan" +``` + +If you are lazy you can also use this shorter version without explicit type definitions: +``` +(x) => Record.Field(x, "Year") = 2021 and Record.Field(x, "Month") = "Jan" + +``` + +It supports all possible variations that are supported by Power Query/M so you can also build complex partition filters. +- additional options may be added in the future! + +# Known limitations +- Time Travel + - currently only supports `VERSION AS OF` + - `TIMESTAMP AS OF` not yet supported + +# Examples +The examples below can be used *as-is* in Power BI desktop. If you are prompted for authentication, just select `Anonymous` for your authentication method. +> Note: In the examples the root folder of the Delta Lake table ends with `.delta`. This is not mandatory and can be any path. + +## Using Delta Lake Time Travel +To use Delta Lake Time Travel you need to specify the `Version`-option as part of the second argument. The following example reads the Version 123 of a Delta Lake table from an Azure Blob Storage. +``` +let + Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), + #"Filtered Rows" = Table.SelectRows(Source, each Text.StartsWith([Name], "powerbi_delta/FactInternetSales_part.delta/")), + DeltaTable = fn_ReadDeltaTable(#"Filtered Rows", [Version=1]) +in + DeltaTable +``` + +## Using Delta Lake Partition Elimination +Partition Elimination is a crucial feature when working with large amounts of data. Without it, you would need to read the whole table and discard a majority of +``` +let + Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), + #"Filtered Rows" = Table.SelectRows(Source, each Text.StartsWith([Name], "powerbi_delta/FactInternetSales_part.delta/")), + DeltaTable = fn_ReadDeltaTable(#"Filtered Rows", [PartitionFilterFunction = (x) => Record.Field(x, "SalesTerritoryKey") >= 5]) +in + DeltaTable +``` + + + +# FAQ +**Q:** The Power Query UI does not show the second parameter. How can I use it? + +**A:** To use the second parameter of the function you need to use the advanced editor. + +-------------------- +**Q:** How can I use [Delta Lake Time Travel](https://databricks.com/blog/2019/02/04/introducing-delta-time-travel-for-large-scale-data-lakes.html)? + +**A:** The function supports an optional second parameter to supply generic parameters. To query specific version of the Delta Lake table, you can provide a record with the field `Version` and the value of the version you want to query. diff --git a/powerbi/fn_ReadDeltaTable.pq b/powerbi/fn_ReadDeltaTable.pq new file mode 100644 index 00000000000..373febdeb0c --- /dev/null +++ b/powerbi/fn_ReadDeltaTable.pq @@ -0,0 +1,181 @@ +( + DeltaTableFolderContent as table, + optional DeltaTableOptions as record +) as table => + +let + + DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), + PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", (x) => true), + + DeltaTableFolderContent_wFullPath = + let + Source = DeltaTableFolderContent, + #"Added Full_Path" = Table.AddColumn(DeltaTableFolderContent, "Full_Path", each Text.Replace([Folder Path] & [Name], "=", "%3D"), Text.Type), + #"Added File_Name" = Table.AddColumn(#"Added Full_Path", "File_Name", each if Text.Length([Extension]) > 0 then List.Last(Text.Split([Full_Path], Delimiter)) else null, type text), + Buffered = Table.Buffer(#"Added File_Name") + in + Buffered, + + #"Delimiter" = + let + Delimiter = if Text.Contains(DeltaTableFolderContent{0}[Folder Path], "//") then "/" else "\" + in + Delimiter, + + #"TableSchema" = + let + ExpressionText = "type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", + BufferedExpression = List.Buffer({ExpressionText}){0}, + TableSchema = Expression.Evaluate(BufferedExpression, #shared) + in + TableSchema, + + #"_delta_log Folder" = + let + Source = DeltaTableFolderContent_wFullPath, + #"Filtered Rows" = Table.SelectRows(Source, each Text.Contains([Full_Path], Delimiter & "_delta_log" & Delimiter)), + #"Added Version" = Table.AddColumn(#"Filtered Rows", "Version", each try Int64.From(Text.BeforeDelimiter([File_Name], ".")) otherwise -1, Int64.Type), + #"Filtered RequestedVersion" = if DeltaTableVersion = null then #"Added Version" else Table.SelectRows(#"Added Version", each [Version] <= DeltaTableVersion), + buffered = Table.Buffer(#"Filtered RequestedVersion") + in + buffered, + + #"DeltaTablePath" = + let + DeltaTablePath = Text.Combine(List.RemoveLastN(Text.Split(#"_delta_log Folder"{0}[Full_Path], Delimiter), 2), Delimiter) & Delimiter + in + DeltaTablePath, + + #"_last_checkpoint" = + let + #"_delta_log" = #"_delta_log Folder", + #"Filtered Rows" = Table.SelectRows(_delta_log, each Text.EndsWith([Name], "_last_checkpoint")), + #"Added Custom" = Table.AddColumn(#"Filtered Rows", "JsonContent", each Json.Document([Content])), + JsonContent = #"Added Custom"{0}[JsonContent], + CheckEmpty = if Table.RowCount(#"Filtered Rows") = 0 then [Size=-1, version=-1] else JsonContent, + LatestCheckPointWithParts = if Record.HasFields(CheckEmpty, "parts") then CheckEmpty else Record.AddField(CheckEmpty, "parts", 1), + + #"Filtered Rows Version" = Table.SelectRows(#"_delta_log", each Text.EndsWith([Name], ".checkpoint.parquet")), + MaxVersion = try Table.Group(#"Filtered Rows Version", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion] otherwise -1, + #"Filtered Rows MaxVersion" = Table.SelectRows(#"Filtered Rows Version", each [Version] = MaxVersion), + CheckpointFromVersion = [version=try MaxVersion otherwise -1, size=-1, parts = Table.RowCount(#"Filtered Rows MaxVersion")], + + LastCheckpoint = Table.Buffer(Table.FromRecords({if DeltaTableVersion = null then LatestCheckPointWithParts else CheckpointFromVersion})){0} + in + LastCheckpoint, + + #"Checkpoint Files" = + let + LastCheckpointFile = {1..Record.Field(_last_checkpoint, "parts")}, + #"Converted to Table" = Table.FromList(LastCheckpointFile, Splitter.SplitByNothing(), {"part"}, null, ExtraValues.Error), + #"Add Version" = Table.AddColumn(#"Converted to Table", "version", each Record.Field(_last_checkpoint, "version")), + #"Add SingleFile" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint.parquet", Text.Type), + #"Add MultipleFiles" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint." & Text.PadStart(Text.From([part]), 10, "0") & "." & Text.PadStart(Text.From(Record.Field(_last_checkpoint, "parts")), 10, "0") & ".parquet", Text.Type), + AllFiles = Table.SelectColumns(if Record.Field(_last_checkpoint, "parts") = 1 then #"Add SingleFile" else #"Add MultipleFiles", "file_name"), + AllFiles_BufferedList = List.Buffer(Table.ToList(AllFiles)), + Content = Table.SelectRows(#"_delta_log Folder", each List.Count(List.Select(AllFiles_BufferedList, (inner) => Text.EndsWith([Name], inner))) > 0) + in + Content, + + #"Logs Checkpoint" = + let + Source = #"Checkpoint Files", + #"Parsed Logs" = Table.AddColumn(Source, "Custom", each Parquet.Document([Content])), + #"Expanded Logs" = Table.ExpandTableColumn(#"Parsed Logs", "Custom", {"add", "remove", "metaData", "commitInfo", "protocol"}, {"add", "remove", "metaData", "commitInfo", "protocol"}), + #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) + in + #"Removed Other Columns", + + #"Latest Log Files" = + let + Source = #"_delta_log Folder", + #"Filtered Rows" = Table.SelectRows(Source, each ([Extension] = ".json")), + #"Filtered Rows1" = Table.SelectRows(#"Filtered Rows", each [Version] > Record.Field(_last_checkpoint, "version")) + in + #"Filtered Rows1", + + #"Logs JSON" = + let + Source = #"Latest Log Files", + #"Added Custom" = Table.AddColumn(Source, "JsonContent", each Lines.FromBinary([Content])), + #"Expanded JsonContent" = Table.ExpandListColumn(#"Added Custom", "JsonContent"), + #"Parsed Logs" = Table.TransformColumns(#"Expanded JsonContent",{{"JsonContent", Json.Document}}), + #"Expanded Logs" = Table.ExpandRecordColumn(#"Parsed Logs", "JsonContent", {"add", "remove", "metaData", "commitInfo", "protocol"}), + #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) + in + #"Removed Other Columns", + + #"Logs ALL" = + let + Source = Table.Combine({#"Logs Checkpoint", #"Logs JSON"}), + #"Added timestamp" = Table.AddColumn(Source, "log_timestamp", each if [add] <> null then Record.Field([add], "modificationTime") else + if [remove] <> null then Record.Field([remove], "deletionTimestamp") else + if [commitInfo] <> null then Record.Field([commitInfo], "timestamp") else + if [metaData] <> null then Record.Field([metaData], "createdTime") else null, Int64.Type), + #"Added datetime" = Table.AddColumn(#"Added timestamp", "log_datetime", each try #datetime(1970,1,1,0,0,0)+#duration(0,0,0,[log_timestamp]/1000) otherwise null, DateTime.Type) + in + #"Added datetime", + + #"metadata_columns" = + let + Source = #"Logs ALL", + #"Filtered Rows1" = Table.SelectRows(Source, each ([metaData] <> null)), + MaxVersion = Table.Group(#"Filtered Rows1", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion], + #"Filtered Rows2" = Table.SelectRows(#"Filtered Rows1", each [Version] = MaxVersion), + #"Kept First Rows" = Table.FirstN(#"Filtered Rows2",1), + #"Removed Other Columns" = Table.SelectColumns(#"Kept First Rows",{"metaData"}), + #"Expanded metaData" = Table.ExpandRecordColumn(#"Removed Other Columns", "metaData", {"schemaString", "partitionColumns"}, {"schemaString", "partitionColumns"}), + #"Filtered Rows" = Table.SelectRows(#"Expanded metaData", each ([schemaString] <> null)), + JSON = Table.TransformColumns(#"Filtered Rows",{{"schemaString", Json.Document}}), + #"Expanded schemaString" = Table.ExpandRecordColumn(JSON, "schemaString", {"fields"}, {"fields"}), + #"Expanded fields" = Table.ExpandListColumn(#"Expanded schemaString", "fields"), + #"Expanded fields1" = Table.ExpandRecordColumn(#"Expanded fields", "fields", {"name", "type", "nullable", "metadata"}, {"name", "type", "nullable", "metadata"}), + #"Added Custom" = Table.Buffer(Table.AddColumn(#"Expanded fields1", "isPartitionedBy", each List.Contains([partitionColumns], [name]), Logical.Type)), + #"Added Custom1" = Table.AddColumn(#"Added Custom", "PBI_DataType", + each if [type] = "long" then [PBI_DataType=Int64.Type, PBI_Text="Int64.Type", PBI_Transformation=Int64.From] + else if [type] = "integer" then [PBI_DataType=Int32.Type, PBI_Text="Int32.Type", PBI_Transformation=Int32.From] + else if [type] = "short" then [PBI_DataType=Int16.Type, PBI_Text="Int16.Type", PBI_Transformation=Int16.From] + else if [type] = "byte" then [PBI_DataType=Int8.Type, PBI_Text="Int8.Type", PBI_Transformation=Int8.From] + else if [type] = "float" then [PBI_DataType=Single.Type, PBI_Text="Single.Type", PBI_Transformation=Single.From] + else if [type] = "double" then [PBI_DataType=Double.Type, PBI_Text="Double.Type", PBI_Transformation=Double.From] + else if [type] = "string" then [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From] + else if [type] = "timestamp" then [PBI_DataType=DateTime.Type, PBI_Text="DateTime.Type", PBI_Transformation=DateTime.From] + else if [type] = "boolean" then [PBI_DataType=Logical.Type, PBI_Text="Logical.Type", PBI_Transformation=Logical.From] + else [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From]), + #"Expanded PBI_DataType" = Table.ExpandRecordColumn(#"Added Custom1", "PBI_DataType", {"PBI_DataType", "PBI_Text", "PBI_Transformation"}, {"PBI_DataType", "PBI_Text", "PBI_Transformation"}), + #"Added Custom2" = Table.AddColumn(#"Expanded PBI_DataType", "ChangeDataType", each {[name], [PBI_DataType]}, type list), + #"Added Custom3" = Table.AddColumn(#"Added Custom2", "TableDataType", each [name] & "=" & (if [nullable] then "nullable " else "") & Text.From([PBI_Text]), type text), + #"Added Custom4" = Table.AddColumn(#"Added Custom3", "ColumnTransformation", each {[name], [PBI_Transformation]}, type list), + #"Buffered Fields" = Table.Buffer(#"Added Custom4") + in + #"Buffered Fields", + + #"Data" = + let + Source = #"Logs ALL", + #"Added Counter" = Table.AddColumn(Source, "Counter", each if [remove] <> null then -1 else if [add] <> null then 1 else null, Int8.Type), + #"Added file_name" = Table.AddColumn(#"Added Counter", "file_name", each if [add] <> null then Record.Field([add], "path") else if [remove] <> null then Record.Field([remove], "path") else null, Text.Type), + #"Filtered Rows" = Table.SelectRows(#"Added file_name", each ([file_name] <> null)), + #"Added partitionValuesTable" = Table.AddColumn(#"Filtered Rows", "partitionValuesTable", each if [add] <> null then if Value.Is(Record.Field([add], "partitionValues"), Record.Type) then Record.ToTable(Record.Field([add], "partitionValues")) else Table.RenameColumns(Record.Field([add], "partitionValues"), {"Key", "Name"}) else null, type nullable table), + #"Added partitionValuesJSON" = Table.AddColumn(#"Added partitionValuesTable", "partitionValuesJSON", each Text.FromBinary(Json.FromValue([partitionValuesTable]))), + #"Grouped Rows1" = Table.Group(#"Added partitionValuesJSON", {"file_name"}, {{"partitionValuesJSON", each List.Max([partitionValuesJSON]), type nullable text}, {"isRelevant", each List.Sum([Counter]), type nullable text}}), + #"Relevant Files" = Table.SelectRows(#"Grouped Rows1", each ([isRelevant] > 0)), + #"Added partitionValuesTable2" = Table.AddColumn(#"Relevant Files", "partitionValuesTable", each try Table.FromRecords(Json.Document([partitionValuesJSON])) otherwise null), + #"Added partitionValuesRecord" = Table.AddColumn(#"Added partitionValuesTable2", "partitionValuesRecord", each Record.TransformFields(Record.FromTable([partitionValuesTable]), Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[ColumnTransformation]), Expression.Evaluate("type [" & Text.Combine(Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[TableDataType], ", ") & "]", #shared)), + #"Filtered Rows1" = Table.SelectRows(#"Added partitionValuesRecord", each PartitionFilterFunction([partitionValuesRecord])), + #"Expanded partitionValuesRecord" = Table.ExpandRecordColumn(#"Filtered Rows1", "partitionValuesRecord", Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[name]), + #"Added Full_Path" = Table.AddColumn(#"Expanded partitionValuesRecord", "Full_Path", each Text.Replace(DeltaTablePath & Text.Replace([file_name], "=", "%3D"), "/", Delimiter), Text.Type), + #"Removed Columns3" = Table.RemoveColumns(#"Added Full_Path",{"file_name", "partitionValuesJSON", "isRelevant", "partitionValuesTable"}), + #"Buffered RelevantFiles" = Table.Buffer(#"Removed Columns3"), + #"Merged Queries" = Table.NestedJoin(#"Buffered RelevantFiles", {"Full_Path"}, DeltaTableFolderContent_wFullPath, {"Full_Path"}, "DeltaTable Folder", JoinKind.Inner), + #"Removed Columns" = Table.RemoveColumns(#"Merged Queries",{"Full_Path"}), + #"Expanded DeltaTable Folder" = Table.ExpandTableColumn(#"Removed Columns", "DeltaTable Folder", {"Content"}, {"Content"}), + #"Added Custom1" = Table.AddColumn(#"Expanded DeltaTable Folder", "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", #shared)), + #"Removed Columns1" = Table.RemoveColumns(#"Added Custom1",{"Content"}), + #"Expanded Data" = Table.ExpandTableColumn(#"Removed Columns1", "Data", Table.SelectRows(metadata_columns, each not [isPartitionedBy])[name]), + #"Reordered Columns" = Table.ReorderColumns(#"Expanded Data", metadata_columns[name]) + in + #"Reordered Columns" + +in #"Data" \ No newline at end of file From 04d3c5ffbbfd4133ee019fc32a6e42afea6d2db3 Mon Sep 17 00:00:00 2001 From: Yuhong Chen Date: Mon, 14 Jun 2021 20:27:59 -0700 Subject: [PATCH 059/291] Pass down hadoopConf to ParquetReader (#93) --- .../delta/standalone/internal/SnapshotImpl.scala | 16 ++++++++++++++-- .../data/CloseableParquetDataIterator.scala | 11 +++++------ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index aeb72c55d50..d0160c7dedb 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -17,6 +17,7 @@ package io.delta.standalone.internal import java.net.URI +import java.util.TimeZone import scala.collection.JavaConverters._ @@ -50,6 +51,15 @@ private[internal] class SnapshotImpl( import SnapshotImpl._ + /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ + private val readTimeZone = { + if (hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID) == null) { + TimeZone.getDefault + } else { + TimeZone.getTimeZone(hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID)) + } + } + /////////////////////////////////////////////////////////////////////////// // Public API Methods /////////////////////////////////////////////////////////////////////////// @@ -67,7 +77,8 @@ private[internal] class SnapshotImpl( .map(FileNames.absolutePath(deltaLog.dataPath, _).toString), getMetadata.getSchema, // the time zone ID if it exists, else null - hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID)) + readTimeZone, + hadoopConf) /////////////////////////////////////////////////////////////////////////// // Internal-Only Methods @@ -84,7 +95,8 @@ private[internal] class SnapshotImpl( JsonUtils.mapper.readValue[SingleAction](line) } } else if (path.endsWith("parquet")) { - ParquetReader.read[SingleAction](path).toSeq + ParquetReader.read[SingleAction](path, ParquetReader.Options( + timeZone = readTimeZone, hadoopConf = hadoopConf)).toSeq } else Seq.empty[SingleAction] }.toList } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala index a7922024824..bd9cafa82e9 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala @@ -20,6 +20,7 @@ import java.util.TimeZone import com.github.mjakubowski84.parquet4s._ import com.github.mjakubowski84.parquet4s.ParquetReader.Options +import org.apache.hadoop.conf.Configuration import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} import io.delta.standalone.types.StructType @@ -36,11 +37,8 @@ import io.delta.standalone.types.StructType private[internal] case class CloseableParquetDataIterator( dataFilePaths: Seq[String], schema: StructType, - timeZoneId: String) extends CloseableIterator[RowParquetRecordJ] { - - /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ - private val readTimeZone = - if (null == timeZoneId) TimeZone.getDefault else TimeZone.getTimeZone(timeZoneId) + readTimeZone: TimeZone, + hadoopConf: Configuration) extends CloseableIterator[RowParquetRecordJ] { /** Iterator over the `dataFilePaths`. */ private val dataFilePathsIter = dataFilePaths.iterator @@ -116,6 +114,7 @@ private[internal] case class CloseableParquetDataIterator( * @return the iterable for the next data file in `dataFilePathsIter`, not null */ private def readNextFile: ParquetIterable[RowParquetRecord] = { - ParquetReader.read[RowParquetRecord](dataFilePathsIter.next(), Options(readTimeZone)) + ParquetReader.read[RowParquetRecord]( + dataFilePathsIter.next(), Options(timeZone = readTimeZone, hadoopConf = hadoopConf)) } } From f71f646575d7b8d22b69bd9d05961a24a1c7eb85 Mon Sep 17 00:00:00 2001 From: Gerhard Brueckl Date: Thu, 8 Jul 2021 18:18:12 +0200 Subject: [PATCH 060/291] Fix issue with data sources that do not support streaming of binary files (#94) * add PowerBI connector * add parameter to buffer the binary file before reading it to mitigate streaming errors * update global readme to include reference to the Power BI connector * fix issue with online refresh Co-authored-by: gbrueckl --- README.md | 6 ++-- powerbi/PowerBI_Delta.pbit | Bin 37754 -> 42623 bytes powerbi/README.md | 7 +++-- powerbi/fn_ReadDeltaTable.pq | 53 +++++++++++++++++++++++++++++++---- 4 files changed, 56 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index d7bada751af..353399aed3a 100644 --- a/README.md +++ b/README.md @@ -2,11 +2,11 @@ [![CircleCI](https://circleci.com/gh/delta-io/connectors/tree/master.svg?style=svg)](https://circleci.com/gh/delta-io/connectors/tree/master) -We are building connectors to bring [Delta Lake](https://delta.io) to popular big-data engines outside [Apache Spark](https://spark.apache.org) (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)). +We are building connectors to bring [Delta Lake](https://delta.io) to popular big-data engines outside [Apache Spark](https://spark.apache.org) (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)) and also to common reporting tools like [Microsoft Power BI](https://powerbi.microsoft.com/). # Introduction -This is the repository for Delta Lake Connectors. It includes a library for querying Delta Lake metadata and connectors to popular big-data engines (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)). Please refer to the main [Delta Lake](https://github.com/delta-io/delta) repository if you want to learn more about the Delta Lake project. +This is the repository for Delta Lake Connectors. It includes a library for querying Delta Lake metadata and connectors to popular big-data engines (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)) and to common reporting tools like [Microsoft Power BI](https://powerbi.microsoft.com/). Please refer to the main [Delta Lake](https://github.com/delta-io/delta) repository if you want to learn more about the Delta Lake project. # Building @@ -168,6 +168,8 @@ The connector supports MapReduce and Tez. It doesn't support Spark execution eng [sql-delta-import](/sql-delta-import/readme.md) allows for importing data from a JDBC source into a Delta Lake table +## Power BI connector +The connector for [Microsoft Power BI](https://powerbi.microsoft.com/) is basically just a custom Power Query function that allows you to read a Delta Lake table from any file-based [data source supported by Microsoft Power BI](https://docs.microsoft.com/en-us/power-bi/connect-data/desktop-data-sources). Details can be found in the dedicated [README.md](/powerbi/README.md). # Reporting issues diff --git a/powerbi/PowerBI_Delta.pbit b/powerbi/PowerBI_Delta.pbit index d8701a9e718f03610ccc834eda460eba236c4e66..63d9fc88c86031debcf94a53db26eac1db75f4c8 100644 GIT binary patch delta 38472 zcma&NWmuF^+cs(|0wSn%r$|Xl_s}3A9RfoRA>G_6N=VMoEiiOSFP(@=Xqgs?_BrY!GEiK_ulhcw{AVU)h9Ix$A{bne=2mO zZWDmNn%6=&K9Y!Aw?xr+F}b&21KTaWPIHin-QG5W53PaGKc~z0bJJQYxPCHzbdd9I z>5lPYx4g66@fw?K6UFY!C=|N*zV*D|Si3^eBgWpOsW64$UN47$K6~jaof6X$<;%^> zwuS4Jw1SMmW<#g7mO~xPXu`jyysPx0wuhKS1OHtw<74loi1EBkELl-D72uQ=;CEcM zqB_1;(O6hCpkHvEhsi4%IPW_bbxHVoYBK*P@YUYWCY?W~zJi#=z*DquK`B=CvWc62 zqrk=go%97p#68HU=^%a~u2Iv@uk0#4bL{)Bzq_$%9Ibo&qUOX4%n4IZ?4re9D0b`^ z@{i2-`m1YOM}=3KllG=d6hJdZ6~DHt=t*9%=+BhRcI^EbfyJD@^$++gg4Z_)Bsq=j zxfP@z>AEL)V)pz!kN15;nj-h12hTO5STLinPyD7m{d17#)Fqx}0kXer)eHU3Ycd8;XO<*gau-O zvQ=jP?Uu$}@qM+Qqh-yeV>P(r4^3AJ%A6O!HVW78N>QEVNBXCKXh|*F^4apw4hX~B zYX%C5oIdoo6ppu?Ql)P)%X`B{$p`myQ4}ssaOn@mM}^mn3NmdK2H(oGh1!Z3Je6Fy znq3(Rl-;#4>?2qfptVJ?^6}~=oyWYP`vk4j!71G3>Qm!9tB^R?+~-Dhqaq*E?uDId z!NxLda`W)B(aE-s5XL?Isg)SJBBuJOv7Np9yA=Ly@*C5ms)GsC83BtA!*c{T@Ecb% zJPZQroS36}*S%bk9Yv}IQbn!rMvODt<_gyJw#OJ}w%ps2fr8dHX~i~K$kkV@psxCA z(c-~atB(?mCTXEH^~$IUy0t7o7-R2^WwQFljpWxpNQKU|D8PbO3@!{<{Nw;Fzh#;~DARXNQrIF#LRzq2k~iR67#Ui>jPSS>T!ZfQhmEx`i{>GnRD&~gGt-s9v z=e6m?cTdcuhw|!Pw(j35)0y!Fr2p|0a4phj?@0>C1FFrY>c14m5lyIR`p1CDsp3tR{3c_kNopVKfgQ?#^3c~*ycJ7yVh zoAz!_^a`)2V(RouTAe)R99jE$lzDaTXCKj7A@d;@qFJ}Ki>Ar7_n#a$@iB}#$XfUE z=I)H1#Sg82`2xK#DuS0;TU|DgX{`_8O{@ViN8g0j<}(}ZO2l_)v}DEUqdZCF%UZ5A zbR}lPa}2Dpk8Jn23N{j_`1@YVwe5b2%S-gdHgwAld`i7LEB4UDv88RPg{y+-fZf^` zGvaCeGj9X7*RPi#S26B`x)OB07OiVDvCQ!!B-<-eI4r9`444D{e=ds$LPi#&3O#|g zhfyd&Yb&uGh6OKuno(W1{yI}H)4ECrHB;Qa_HP1)zvJ{{E(R#e>hAa0*f>O=MCm0( zme!83ord*eg3m{4)!!_#C2?`vWl*&AF!ivSHsvbH>lDZ*?lBhzJv+k7oMlq54g#p6 zBlLK3x>3BPEJvXbcrk)5lrMYg4EHL0Hi){jd8GJ8 z2Z!aFLSdvrPPls*lBXg%&GFg|c|Vycy>iBYf-zuwEXI^Z^Z`aFc-x3E-Gv{P>|s@y zma~W*f_MM@L-s?rDo{`rKnQ6x^XKFhJ4YNjZt81Y3~1qwsI_uORF4`pH39Dz>5l7F z#7b>QkkqHM`7L9-F5nJ1={)3y6iC%&oE(P?FV48?vT=pq^oOw}vvkeVL4VcvK3fR? zbvQJeCd3^eFJ`EKG58&f!?}w^EZo=b_!tj5I|-Z?SZjZ$MpDmsDTufqKbqvj9f>@5 z7&Dnk63+4zlo9&lR;?A!3!E{>*7~=?_4a*vb$jx6h!$f@3`hMK(rsE1KPcP$WtUFE z*OOm|JGr=q3p$I;tTA-^|BdQ6Sq=NgxCo7~UQJ$`nZ<{xiI=^1iBuh^DAA90nsZ30 zbx~dHrLdiH!{zH01a}fZrKCPnniXH)$glJyr8vUdCfbZGGd`pS?8W^?8A-;;sqJA< zx>RyQHP3co~ITr%VJS2 zzvPuvYH?d!AS#tR_~D$2zxlSB{|Y%eufZd)hUJ3pHdSD*)E$?$h#6(c(tj@moGw{Q zn`)#MRx{qtoSdx_jTo4jCeuzrDq1K;6gI>m)Z+|3g%i9fHeKIis84X<;V`zN0lJXf zu!*=inDX}E*ry!`pA}lDWW~cR9aAXlikoHw42sDC>1NsknH)kVUd($h71Jxwyrniz zf>)B*+)rpQTutH2-{w**V>fz`zOm-BBITP?DRwhstjei=L`t)&;@-A=XFhLo7Mkz6 zkJozd)ll2>@V*y$1ri+8fa{Mn&&Eqp_h+mS{xm%sxZ~m=T)2z61B*q-&+%3_hujaP~+rBRz%zz zGir+Ys36&$X&L=-&MEtZXz1m--h>-2rCr@{mariORPE{}pMZe3@<^xFU)#+nI-tgF zSO2r+TH?dP81VHhq+{{X3Ws zFfb!=19kUlGy%cgeR>L7_3EP{?tvlSm7|{uhekEDw|8hx|Kq|P9>$E7A;}H7wE&8m z@i6<1aGJ$T2f;sl^#awm6j?zhYR>HD9qYMaklA+p_Q@ew)YJLEaqHVf>XR{|Fy5!b zL!)QNJXB6@w&Rxwt7ssV0N_tjKZU(!%o1Ip2+eQss8f^;&vQ=tVl@0(L3+1N!&ea% zhqu^|c>BVk1YeWo@(Wn=C42^Fa)wp7xOwKK1rzdLXTp7hMQ1UE4kx8=e)0U`G9IVN zeV;(KE&huaciWdi+)>=1A&}kH1wvDmxk~!IQ~Of%EEyW!)+EqOolC8mTK}Jec(be|q}P1r5cf_)Q8-2H7?BEx1y?Vp)(p z`@&sBqrAg!%uV5>l+&A5cveXaQPa0LIN>77UXP-+F{C*PufXtEWZJ^>Pw+r|WAl*~ zQS7{Nl{WC%FZV?3BAF?!Fj<>|Q5|v>WC9XTAu050#%c(c5~L5PU36oq#O)n;9iywQG@$e3EP z$7R!8j36uHO^y-bUlJT?cur;OrB#N=0ROTc&a0o;Z8~jJTRe+c zg^GD;VXL0eq0ze`vusm3zeDu!)D1`3GfvcK`0k7%b*VyI|0=hO>|Mv6I|x&lu;nI# z3ykgcFkIAy!RBh4uq>$@EUuH(e3-P|YN==tG2bhX7ede#-Q0yM`{|K-UwRXk(4{D;C^zqp(&|sO^Gl%t8QErBNryc5@ymy6&in(`8Tw6Fa|AYtXdr4S4Q>)A^!7 zQ2EtxaZ)L~=aEGMuEP*}!C0PdO(wnqcV@Nm_r$27?1hz$G`pNVD$9hz<34&~G(od8 zd!rYuUy{9&_gif?`?%oUr^+M}RO%jO89zSr$l;;&(kccD4rBCAqO=%+__dj|4rz0t zClQ&(9m|n(vnuSZ3r`;u`$>{g;P%_i~bPP4ic< zYQhXXQ?Ezs7O2+x2tr%!W-VJ}3C=r>eBnI25177cpXeToG}M}QC{?sO)%xm>iaR?u zG<7>#$p3554`M6zt(BTU)`2OsxkFj$3ZMHn(f6cAF?C=j#vXw39@{>_cl%;NHIyjM zEbd~wszaPsU8a;odOl7nP#_xrT+=X}F!E>utnCX5PTiTX=;awn=hCBEkI~c}qZg`s z?fSB~V#b?@8PHOBwaQF8u;bPyltbs1Rq!0~0^TB?q}hnF8}Wm*V~4K_`H}moB%a3Pnjb&xuS~tcI3Bsbj@~L zH&Nh!Y`FbJ{r5`J;>^2<`AQ7kjQC~=`XoMnqE*RTU#zT8(YYJqe7+znxJ?_p(qO~b z7gdF^*%%-bmtmcx*OWkWBZ@bbEO^K-z$QKYoxah(9_ zYHB)P4X{nH%tksbAVqhqF(nB?|8BQ%?`@mTB{=6GOXcg5&^YAbo=yxfr^(-e1=x9c zlSezBSW;;PG79oeulHMC@F+ggD0aT;nEIUi_up3G;;7LH&3vVXiErNOKJ@`AuRUR0 zuJ@#{b+|ci<6V1N_^t8dD?ecy?)3h$MG1StBj75011lUVcKtqmYvE|+dpLzntV9vR z&$9%0V;Za2stS5l=eT1s_Y#5K%I?S}8TwGob>=#Ee7ogLY8=bH|84J`Si9g3H^x0; zd}kHAGrG4${%~PwdJjKhLgGQQ&ft*DM%ZL<(AcuN~H6Zsj&d zE03bWjPdR7)Sf+UyCiBtQA2RqChNXM0zW^+`$uxSeGoJbL2P)6~|VnG-)pQWwRnD5zi9H1Ztx><*826WupH zA?RR;)^=`{qy(k;$zkTK>40Wr-=DJR+Y{Fjt{zmFERI!)SO*^Z zWZ7giK0NO4QKXi5goO`9eR2K0Kk&(6pCR))r0I&`h@!$wwT|n|>Qb)92C&Lw|D$z+ zhi0=pXRJyQ>Ss#6fsmnHf4;FbcD* zN9{FOv0#&AmW4p@Q|S7mt=u+|Nl?}pt2qaXk#G()jerE3HxMMca%WIfUf{MEOV7(4 zqCpV;9g6bG^(wpd-H9iyXko`2M3s`Vsjm|yF%%*6Y^|6FtZw(06Zk&y4ib&w6cqGY zdo5!9T8FvCP4mVC+@J|qK@+6pB)X7Kv-T!EU%At{L0albdiO|Wi>Uhs^#xcduUImN zXPtkB0y+AhpjL^8j1b5P_E3sP92@j^kZ6ooQcVtlr^syV=opR1!E48 z*PU(lw@i6MONYG|jw6Y2i-pUUJSTeJ;S%8kp}F(2GHj}P!lBIiJ&Yj!36ix?KIk{J z{k8>$hV?->u4p^ekF6uAm<2kIq?}C}nJJ6-^ntlKnL)z%Brl&jw&7Ww_o;NtABbNj zrOFH_CQt|kdk?s6uBL#t2C*>3K?Dc7&MDZ*6RI}h9OnM3*ss&WNtq$-oK$tJ8xXeK zN|Iw`>u6MYij(b+cB8(f%BuA@Y(dx0{(xtYP?6zM>ST6Y_s_JMXo{v|=IJV$RuV#& z9UB%yW&n-FA^RGRc=e~9``d;Ekl%))>UWlR3OV)U02gBiIEsAF(o;2(vQk_XKB0J7wx` z)M11ChG|Vx<*fjsE;F3$`ck(Z9%lV|N#aFXA`N6)$1@4wE8f9cp)j5Wi2gj9duE^l z6WlMUriA=f`o{&i{}Cm%J@^!!$u{s|_pz81JHr{D$H)WQz-p-(S_ekuq-rx63=;o5 z`2%mYh8);{+s23({f;yy77x}nOJqHLail7EQHv>Z7&S41*}TV%Mn68Pgp`qzXC-}d z$Sp4oxr73t*%o?lA7B({?4#AK7|DrL4MT&P8o%Vk*dTILxHv&?m>AVE0yY%NLaIBl zOzL-BK7Fh`w-k#{5H83wz)xXL6MB;qHIYG^lGu^WuF~8?M1*-iu=U9s`%?Z?t4Rs=t7aG2u5il z@C-yZXXpqTQ=~uPU$#;}7OuJV5U~W-|Datmk>c!{NhmQUcIq2yXa3us{EMVFIL+eD zLi8gnnU}>XN2XTjS7h)+9J{Fu^cWyAp>v8BMErp)e9zNqb~}Jn#-b?%kZKF%WtRdi zhYXix3S^&RceH|Regogt8{3IHF|l&bwWes+{^>mH)V6(ZA@f`s*L6jiNg(n`D`C&m z7TZ#vpQOXP;g_@;NOXj(6fUIMjYtkG&CkJQv&zdh1uTD}%9zqB-Oz-bz=w|eQ2!FQ zJyRGgX+zi6^3PfdEHn1!1AjIj15@Gad@745$8wmvFgiz!$EaT?x>{vC~U zJ`kss78D|Ha9VbI6JXQNzsv>+X-u^DLr(mwz1tf!0Q=)h6gb@J?vmaSG6*ZTnUl^# zx7(~fb$KN8g;Gj8GuXG24Fr|+M0rl}2gpNl6aTU(zh;Iq zf(5ubBPo#z0B-j>oB#}|&x-K}OOMw4U;|-;oMnA4X@SM%i?Vk~vy*cy)MAGwg+lvZ zg-iFwg2%dp;#!Y)UwZ~99C6HYZZY{xajngS#gg&7WJOlA6)vJ{F6nh1Hc{%^4;l6Z9!B%h{%FKdKHZn>ML0T@NL$m%s;GS+_l&o*}QJ@h`{$_ZcA?qnLj zVl^%DoR@>JnFxx+)U@8lcQihnIsBAUnKf5`lYRYL1h zmG5p+!{HXAoVU7F;C;p`4_ArISZW^evdnjpa1pbRd7J9Af?!HqSJXRubzJCYilXcF z5POV|@>JFVLSB!?wvHfC^C<`ZH2jKV5K2;xv1#R#L-Wk)%dypDcDP1@)NJ?PH z*Cx@M-m11tC!UzsB-kSGa^-=+&LBqN!<8k{(OPN!jTWNeo!EYjZKvz;bwrY`H8af! zLoxnb*7%>(Sp}NbABCSzd>)-2u6tQrg=R~owH`pNa|9PG#@5Nn^<7#{D&&lvd;y^c ztk~9v+5dzLY;ODSwXX$q${)x?w78c#-{eKi+4X<2C7m?ViTr zQJY%?$Ki)}t7hN<3Cy}x%)Yw%P#_rdp|cuWUmG8xCN55`D=B$6* z7KY~a?1czV!1IRxq2$h=WY4KWQxQZH6sqUZ**Xiy(z8z5dIY7G1c`uIkZ3`n&{H_~-vgn?0f?p*& z5|``Rv85XBvdbm{oc?jxYv$qNIUSDSXHOFtfwIyJK*)32Xx{Aw5Ks^6bmfH@{@d8} z^UUabVQi%oH&R)7)%QNs zbosc`OcXq+hXBc0x@F}P946lx)Lmqi>=!5v7{I0^hDi`W;G>yMgwZ$6Ck1%fLc!K2 zctBLTk#7Fxol|f>UpWDl3j&_HNtO3SgzEUAwp+h#o4Vw!!<$d2yht`)){)s*J;056 z8}aX0L6ou}NE>X*HC2DdNf!?1#MptCc%gHhg91Nzhoe-eZZ7zj4(GB}RGAw@qhPNO zjB4}#YGzX5ec12z<5lavGzICU*AoX-t^OIPHf7NvFzn(rIJ0+(Q zc|=chlFU@X53DAuG&KC;KR-?FW!t(CU5T-#&xx@HKRX@#Y!ol}*;7s{EatWcLwsr=;{JEsTz$H-a6OwwlW1%I+n4sGSCdH( zCN|zd$$3;8=ZwtAyT+p(<|a5%0w744zQJj5fctyB2iU_h|JU4u+{BBSR8UcmP=}t0 z65C36A{qMjYslIyG!AWbB@fwK|KriS(~o+KomE7K)_#A#f24)}EKD~YZO^j`smvrX zXm&%Wf<^LX1N03Fs+)N}*{`NQu$l~hf5RHl2vLElS#|u`>Ft#U!*DHmKrCf2dZJye z4mYM>Kg25ZW$Kfx5751r`)VnLhdzJR42ztOIccP25c-nalB-k)YK?PqOH{t+)X|<< zIWeA~ofdBFBvQ+`=r*SMPHZREQtbpMXNTzgi}&TXox2GT@l)=`{3!pP3EtUj`}?Z8 z^cH+IAvIQC#5Ni48qfgezX#rDlQAqqw0c#JZ>K3s2Asf1&-4EfV+XI&m96NiVl9ZL z<9dT4a)Sj9(63q`?6ap!Uj4c^Mr|97>2BRP?MR`L@JppN{6CW=m znX=0GjDSN_5?Rx??H{G5rhDze&t!+$KiFH5Lh9xBx`goZ2Y8r5B^m-KmnybO zG_2J-aQAIda-MaE1+<{D@Vn|tt$2aW8#c5NhxOt=G!*e9>5IAdDX)%2WbGqlyYK0zR?l0R zHIM0}FH~qvS%;X6>b6+q1O$P*JRN+M;GtGYH(fm%&j7o{NP$pI{{MslQ_6ub2!@l5 zZb;C{i|>&N8Gc`<1v9zGmO~V|<}(sew09eOHVT?Ffjo z@n)>0%CAQtr|`N}3Yk^?i%9+Ic_k`k1@pRCqa;#)44~VpeAcmytBPuUZ9ci-&GxqY z-p-WNT1C?S86t6o$*rDx2DnGVrTMUcLEmxrZTFI`g>(z;Nv>O^^1O%#>Vm>ImvDM7gO=xl$wJPJRJKC3-HtMmp zbFjHhq;XTis^`_WDo|sJ|HDR(N3Ck@F8MC%x%kP5FLMdViS|mTWm*`m{iY8!*Izm( zG{007XUu?t%ZoFnS2sm__0+z7!3xB9fe}U&#fJ6Z?a-2Ag9VN$3T0OBXhnW#5N(=} zykVF=n=rmo&R;0wf+EBS?V1|JsLkk2I8&V0O~?hq}($`lLK`YVBeo`FS;>!OHU6Tp56HQZ!mt?BiGMgb7CaG5_45f=Y9|@ zV(?*9L~+D4Yz)rqa6U#yenzvAy;}~c5VqXgeF@)l9oK|a7Y%g~Nl23#yYG?`M}7+Z zPGVYe->(Al&oMMHMzoO)C-95ZFNG^Qv*_0fq``5*I6ym9L>()xd;kX+Kg_x}@$-Tt z`mAQ86L1e<|2FX|jHAQa?!9PMm&v3}j!#}E9sBcTdH%!dvREiBb)Y!ov&jzG+(8sE zGz^TKcH)!fyn{$zC?3xPvXyd87K{4qOM!OM&Z8WVJc_*`k1YR79u>32klm0+far^U zCe3t75S4Hi72eYXQHiId{?A~$@P84Zd7eI|3|S0~d&}fo^;A=e9cvW_j52@k_*)ss zr2jVD5cZym@RJs2?Pc+v6)Ecr+rCy4$5Cd=c$>kV^N$-26Vo!+5@w_PcOY6`G%GE zlAe&J=@wZptFOK3?PfvaiQbO6LEf6pjBI zrBwePlmd8{;+WQhFvZPwXNZ0D#iZ0MElkr0Oz`KPPo^6bKFGnWf)}jH--L1*nH~ow zp~+sq7P{+$g0nk|J$rV{8#r$Fao#{@17`8|?qtrvhj!8O`UDu~ynHCG#RHU8kJqI%x&H=eUWziDJ7B``Td{_f0&XhB zW6#sSxH>&3C2Z6$;9afPtM2+M*C+F!@$%P(2@Uq$i?6;qD78Q zLJXB);AN@+EWPO3sjpasS3jBB?9XrF60WvI=A%_m!g%qFT=Oliae#V@dz3~nY{Lfa zJ}pn--m?BDAGwi29lo+|6L+8Q2SLjD3z}X6G3-PG_62I0KmC}&XY!p|$tw;C$A#w1 z;CKTU(>d{E#(8MLuF_!vMvH4ag~4Arov9VqJXsc~oSb0Pjdm*T^6!%p`v-C3ez5;cR>d*2kG` zQ~@kb8e%IrW$MmfO^!>vhhROS_N*?)i(}pyN=BztkRK^1Qf8)NLF_*|`D%LdhfyRuonW>?v?8@{~|DBi4^XF?K?TRj`?GxBG`1he@wCUi^& zgN_&25!kIG+!3fX{2lG5E$0N2p@@cqeV{dxrsV^T;|~1d`_br&83kC;prEs%Bd9(p zb|zjrdKyRDoOQ}Mfyc|AZj0~)p8QJs>1vGaczh;~{cCPY=ezO-U?CYPlxel4SafrZ zj9sO4c~T-$yQ;Poc`0BMZ-N=kn;Ck5Ot_)0|BP((c1v-(k9#aqn~6{5gt1cz_g?gj z>aR`yj(buppk+}*MtkBD@iG-ItW^!Hu`<>mdrMlz5WpA4nrH(B4scn!ip2-2);mQq z33h=OitJ#Y0X*v!B;}V4Bx&g82wrp;a1YeQ6l}F_0BG)f-lv#8^^tg7E>t14>~+-m z@RQ10?3n~n?@i&`9JnKoZJA^ocZa@36k}#4q;=?uwqpeJJ0s`NuTEya>+2NAMA-!% zDZTd8{nO&hEBc&?dZeb4J4Ak>GcT@kh6zrgs|a&!q}gl0gW zvP4l@7(EtiR-il2@qhF9j954b2uN^3(VHByyHF3`Lebkg3|03dQy0b_g#M3C3wL94 zlA>kBkO>Fg*@9i`ve~iJM-GIe+{damRZuYXy?5eYqo#;fc{H!1R6AOZ=tIrOl2lN< z&vq(cSnVt!hL@BhLrhS2@CueWu zI?TvCMcEz;MEvDt&WqMMQA!DT@3E>T@%kK;tN5=@rp(YJ#N{Yw( z%T;yA_lxIRTvAXxF_9|o&d!73*LNh?(_~{f91Kk6oEQvD?CT^)1FXmS>I4+h|5rD- z*e2eF_Q@@o!XpzYT9KicnPtzq985z1Dh9`N?qm%nai*<;yC6Wxr|dz+{GEP;y1r@&J`u}47=!AfTuWqUA1Pl< z(h}%qUG|Q#`>H3?1DH41 ze;WR+I{t@~N)-&pSgjXDTD2^3+Z1imt)={xZY5GAVo!s&2VQ2ZO<4m0A$rkd0Z9Jy zp_#ll8R^=q5zq#>pwAl!h5TD#ZwAAeqXJ=L%y;#7>0ojSf3pEM;qkuNWFdNgJq9rm zhxoC`y$EP)Cd#>~3(`+*A05WVNHWcge#59&$d=w+Ks}Muc5I9H3l{N?GaF~(UMx~t) zfSG+%df3t|g-12q%6IiHixo}1D`rz9Gj4*b$A@ge zb}bC^dPK9*HkqlHISAsAYB zFag{_zmA#7?;@YvR3rE2lNmF#un~E%8XY-@a&M>sm*R^XYJV(*-EcCGFy)@lOXgdP z4V(uTs_rmZSM)zWi6-=^*iOzrM-7!n*g##&`=82YSK$td(Gb$Zs!W$(vQR}kpadK& zQ%nZ-g&1d3o%;Rv_1EIa@K^T?Q?F}Yxt=GVcVt^HDDw-pzU@9v!$x0Cy&bCy zp}EQj)|Qaq(WuW<|=3Mb`D3hvr6|Bfs`xn2h(oDpIW5^xH=?@ zYQrlEuE++y2Kbsy!M6mpAwV_y5LEwVGZIv?uayXo6 z^bjQbT4efXV&-ib+ddH#CH*^f-?x1P(+4K4#vf8N1qKv&TZn@wBbK7|9zlmzC$r z5_R#^6DoFxx2uM(l1h>%^lE92BO*oRBt}GVpSZxnk9TeK+<>tQ?7w#6gxx_{N!c@p zr9kgS-%%it$MHMPRdM9KbyEO>@yO+i;?>b+mjXwPf1zNo7aESx%LnJTeQ=XRm? z_>({VB3rS;F2&mb!I1liO^ucMT zFjHERqHP(^OsGT<06El7+gG_vm$Y-~e^kU_scRRutr5_YBeTWh)>P(Ve0g`R0FZ~I z((7KUrkxgaxG$O+HHU`VYyR@duA*^Cot=y^_Rjw0^MRu_wYl4C_Y>mujX)cJ8!=nS zU#3>zv$195?E0TrPkHL4w0#B+K^}kk!KO0pV@8Vtf?DR25RjRstyCK4803HKql*If0@mSm>V( z<{)Z2lIu~TmRl3eCDt1pKZRxpY6I7g_Z6>9eSG#7YbKA~9NU?ifBP9Rr1D9n1HPIB z1uUS<>1~v;p}GlR2L2m7=8ESaiviml)%DGn*;O}VC+Roz!MYF+dZR%L;+#;QXFs33 zMoraLC7`k3>QxaMXKj_$;9gw!ufD%RW|F)y#jovGoY-+eq*351wTn9M4pq&`)b}($ zi+)iN_eK?3wUQ;wgk0NcFO+=&O?F&Y}=>paf_Z17S3a;S3|q>1@cVMbrJ{ zI_&ew=^0ul7j^JjnR~Xo-lNH!7IoG%C&UA)%c^w!S+OgDu3Nj)B#kMb~M@+=;4lN5q2(o$Ca zyxo^J!>j*L(TyMR1u^j;m_F9U;AS{L93*y_Gn)C5{lrp7!tdwVfrC~n#Y>|Co#NRr z8wXz5Jrya@na0siomP*QlNi$Fo82bfBX8S_jbF|j)vczjp2v7%%9mc78lDh%=W#U4 z%tiny_Oi46SHy2b$>N6RW`^!CBEd+*QCz)B@tN`Em1&ZfyP5XE3GJRC=N8tI5%ag` zCnPc;4T~x^-eP~qreJw%E1J^qr%r6Ql?)L*PEEuYKcc5NHf7a{l3+K7gYv-TQ+B{F3@9?yp_trk@ z8i6)A+0)MqKB@q0Nw(k4&qj2}EElx3I6NabB$ZWWIRm?k)tgN8yUD~=EfZZ>U>{oS zrfD1`z=Rz2>n5g=G;BYyZLYf4-p5U#&5{oGFp=o};cu?)WAcVQ13;Fm`W~k+ud^@$ zD0%2Y_C6}D0UV5Wf_7`rC1H{tD6k@9v9q({IYTlW>}jzGj-7Ump1PseM~40~SAUb& zqYgueCGRN00a^&u?M8Kv3UB9KkM0BQ_qSl~aLl_JZ*7 zW(wYIfbCz9suY#6U$XQ$=2upwbkJ>yJcv3uT|{*R#D*7qO^yK1$R|xM^dN0V&ldTO zc>F=Z{|p5fdx!^?xMbZAZ>tK%r+-BgmuoL_E|pCLswZzRRWRR%e*|Sg zcj&?2N~G$|_?&{|O%c9Rn3Yo_g+bm_u>$!w#_7f|WCCfDqZBVd^oQxW%U*Q1fpJa$ zaw03=`{~8pvMbXco(_0X0#tj8f$s7iz$Dqg#8j0Qy@RZ>VW#eR10U1{Pbi4;*)?5v zw8ci1N6#1Oj2cm}!OR}ZmcNk@w=ZUND zsmYXqUQ(Z&jW%9cF7>D03{BO_0X(}98EM6s6p=58x|YM9RPJlWcxI%kIb8*1;1kc{ z4$XJjc7dztn|GBerFwZpSjDv^ETNN$Z9_$CD6||RUo%I^QOLofnefCTBuVlVQ>d;5 z#nGdC)fMc{6y>PONtZC>LUc_R94&*QTLqrFpv*j4k)|Y@465J_CBt_pF(f~47HRI@ zLmyF^=b4t88OJywTS->-z3Q>8%22M>M#V^}%fKUN=DY0_V(wpvpCTa1b2F;s)qbFA z?+tkQEz>$v2C8`!cWB7M>;hxaWHhm6sPCq5eiS*{1ttiK=0=g8BUQU-Ehq!UJ&MCMR7~vx zQx$Jk*6MQYtMY^lHu5CNUH#d_(-}5m{DZT)5VqD)s6^{N@g_`d4ON`uYNzO(HLs){ zG9V)QC7#VYSbpJF2fYvwzG^0TfOEE*2+o%(D5}xrQ6jWT z$EGXy?&0KkONUzZGG-!cq^iDXW{8gm3DCBD8S(gJA}$@}6P4{*7}x3rwOX%axam&v4# zK`Y53HzUdoS^QNp&O}?hcS}^AArFQ3W+(tsk+_8xmq>fdWy`uB>#bNpxodq|lEZ-c zu;=pMV7#5fCNDVB_!2lAMksA}KeR;ftwTikTC}VQ{i{JH1%1P$xcrAn0g049Ady1c zulnv>`8uxo18mh=kF8(Vmo)PCrN#j)POhMr}Q;+~9(de1Lp1L;p_9 zf3VR1LZN{^V-*I6KqT5Vv%2@dXhhISGyQ~j{8~0JnoPWHS2Gqjqfq?+Ve75KqH5c= z@9Q#9q@+Q*MM^+GU74FW}R!DbNr6|*!Oj&>gP7rbue|q)L>|;QazvamzkJ-gZu|H2= zVjg995xF{ypS&BpQaW8_jYFmxG1lncJGG^pm{z9mzIPONmg9BLdj5s#wGjjUP;2?G%Yk|nK-N~Uw?{x701{SX| z!JTW8X5ETgv;foc(ezgBgvSSDqpp+N2PuzCseU6@L|XGkPT)DwxfcaKKJSE~LAmvY=Q@Yj$y=G9&=FlP>% z_MSp)APmH-+pGTk@Hxt1G=A3$cfs1lxW{HER}FVQ5C*#D1}l5G(u>ghB{{w9ev|J5 zA{z4s(Ph<)C9C-$Sn%NN+KzVu9RIg?n=KW?0;E-*N5 z3G4adax!Y=*Fsit-Td0>(zDeE2R;K$zt7mjfwVW-T2h<$PY8-!l>;w9pPFAJM9&tY z)5;YEB$x{KQMN|cXaEuQ>ZhJ)i>{Ldo6`P~-R3@z6Oyl=t{G_6znB)U*Zc;3B!Zxi zxUN6yIysO*WJCE2qUc79~gQmz2wN_lpIMVM9LM((4Rkob+i}1x1PV zR3x8w+UXpz$@{p|0NrLR^jywFCyh1aP*HQ8@(-xVFg|sN=y!7OGydgL^`a7}85eqF zOqT_hI33>tpQY!GqqY268W$)Lb%_`s9Xs0~AlWz3Fnh|!KIiXorcc1tUcGXUBG;B4s=nlA*qhORBvHH@AM5) zwjA(X=Bf9HuY7AB<6N&pE3D9irOXUNRxkJ`Gq8h8&mi)Dz=jnEHpw1=#$7!e4<8Nz zx1z;SI!^zXkIF3m-77vKqyG`#mH4xs;&1oBfC3D;`jm2`w9$IqvP-1M9I;WZovL26 zs8Kig(YL`*E@#E~xCE?herKiP3XXO0t_{hm_P4QfvqV$4E2C_C$035mb<{_9dhr3R ztB0Bf)1zZTsAbUSR2E(dVc6#lfWB9{>}Q{da!6JspIA&JLX3U*vC40jgDCIF^av#g zSUve{Yq*#d6CHA5aC(a*_N?0F6MPG7vyMu-c_8>J$ko2T^eQh+s5$Q;X5)DuwFug- z>X4K}8!{@lJN}F`M$pHxi@83`^aedCX^!@>0JxZkAeq@oe90{UT5$415UAL7$iBJ$ z@fJIr9X$dM7u~W<~*>oIS7o_5mM(9k6j_dm5fk){nMtkgt>SkjCRU+**aaF`#Nl9_q1CuLQ) zrT1igJDtGmja3D}Gxu@`MBrJFq&>6D&o`c1V{`A?*EVUtQF~@Ry8w4Q#baPze{t_( zMqxwC*ESX3{W-%V8+KKea863*) zX-il*LfDP%Syup=rgL?T7$2-?0i_r0%w(@T>7xI^;l>@^m#VmNAslY#Xew>qywd*m z=D4ow@e!BSTlLW^M$(%7XUjW z*>J-AJPV^=4HFqjsBJ6QNdUnqv>OLbkW~*bI^$at4xBK!sWCJ-HRcm7_ipoB<(aBOMIBlf6tb{#v z>U#wJST7DXHClim8(dt0YsfUj3+GdbQ8cD|rpnmq0k)QtJ8%OtF8-f-lb-pphgGgLaUlY0mB5{tm!D=>MDPzdRv31wJREyYN;_M2JW6;#R~+V83?lhzql z8D76Aa-v)K12da{c6Cv!otAI1sn`&u^#677SBL)iQNzIf7r$qt=|V$1sm(y|5ZHcy z;MN+kL1_c7yi*EyZh3Y3T_pu@abib3rHi}Y>TnV16}~xg5rlID2|l(7WUTloFbmwj z!`|rWDf41sP$SI@hIgG*$kVAZVSwJ7ehn`7_fxA|-;2SH1O8*M8=taq%yjswT{ zC&u8Wt+9n`Y4`??ug+H6_=ZMrQaxiEHIrolT`NU6*Gi<39~m8+PbvpV>e(LiU zbtJ8fA{A2&zi-D2_@N9x0btpGAS1Fu@XR|64+;8ri+sbu}_I#O}@3=hsM==%E?q(>XtU&ZtSV?9khXQv56En?$Ag=IuF{k4?~M#RK)&u|7K+(NNCoVjgp=)s zuDsm=W>MA$pXUqjJu=DzRL7H1DnfTsj$g0{3-sVKGg`XpF=;|y>{;#Im1CSupz{e7 zeA?jds#H_e%d!$>IWim>FW0xJ?i3~VA1&(P;2&Jxz*k(}fRfzh#9tDleI{*vg7T{T zkmhY*#dOnwVW)xB*8K+^L1oU|%hPg>-zUnC*dyxZl^z|Bg1z$JJqe!BJ~*_DwiR? z>~v(X$sKO)S`k$!G#Xl}Onr{inCbx%M2$Wbxe)r!mRBeUHs^yitrJdCq#G6-UZic_ z?z5t~NX!*Bdg&Gu#}(}-kW5$jBy_@Pc?Zq}moFiS(+~@+kyK!BfIYS*MikQSun@fJ zBRgYY!%p4VsnEVO(o?Aq+KUZY>fv6A%lIZ+S_n=as-oIB47MC+VISqk07$farB#g5 z7`W1``I^8xaI!%k+!J5$Fr`a3caKU=s-PwN9_wu z|7*s7xp1`*Hgyh}S(!X^`22t+i5;D0n3hy{`x-F3V&Z#{gvOiMrEB_&(O<{JjTK%$2I=|^A9ij*e!lM(vs8}Ns&Kymx0}u z$821Wha@gM6i-ataw`qKdZG*i2LPHGB%1{RS3?PIeiKyIbM1Lx$(N3MD8kdumuraC z7bn1c#0fCJ{*7Fs`Ni5!S(?J|g|2XlT6^0k=8f_CH{{PTP71Di8qUvt#*eyEyd~J= zs0FngytOVwtUj`Ja-h>FymlxQec75ptO4i6JVC7S#VI^PRc30paaaSxVU5Xuum<=q zu7MbwJ-A$(i`ZFAR_aDitkY?pp3wX7coRhutrkVs%d9st^2`qY5Ot`;3{li?t^Hyu ze;-=?E=JX2x|nuKee39NVNE2&SFdVKLS!v~&nm`3bx83oi_7|YR0!V~I_>XQ7F>cf zOatQ!DM-3Fm)k_=wjKcc^BW=UTH)-(Z}~2(0HJ`n>~OR2@s)gpyx10|{>cgHvqXz@ z7x*gx$gheMm@smystafPxvCj8u3X}s(FY)e(!5*c?!^=#S__YmFA{xU830ekzsqMP zj4>7FMob+GJoim+X%+NER?R0py6vHCoEXCu<#8)jeGA_`-~y=nVV6JbPvN&UJV`mG z6GT2ND#)ujfSeh_VkS3uGJlkA^`(irEz+4@2rE3`HhA~UV7yNr8X8}tK z&Q;@-Rc)`uLTgi^*OYfu_&zyte}fI(Pt%q@F-|?968=~vGvQk*CHl!fwjyEe?FnM}%!tc@jVHnlBqk zXRMaGzU#0Tk33onfL8stj&u!FXF4^Vw6xis6T?hrTE|zaVAbxCb+>2nn?g=rF?GB1 zo)Y>7Pcc?Gv#duzQ$^l|o{Z6EJQ~9Ud?<+(+opZx7bfMoypUuHH zPc&#VK#fwt0=jB=dHA+ zkNU@pZP6->XpAIHDv2QZZe?_0;jEOT53<~`^`*aEQ{4Lg8{~`iajkq5t5h56C3WQ$ zy|`D-$nAB`8x4=vwV*`6z#i2h4y;qA9ueDyZP#Q}-$Z>dtGjdn8NS7lJJL!8GqmJ>BP^vNre{NM=KQy zs`QAk+)GYHW5T!uwb=dL&m;aiEjBn@S3bHx(WZ$zE$^6Q8}3t*UmM93|w)o5w@c6+bIs zbeYk`(R136MfaQ+My&uRCd$hbbf0c6tFvH0YmXOr;6?Djprr&nu%u(GSC3>Pjyoj! zb{M+w!2BFr}t~YpAycWwMkJ3 zYp#2cqigX`za}qz`wuh#yRH@lwe{~Z6ik*@nL_|)I29Fw{^pRFVQ8Y}c z+3KZ$UM{A920q05Ul8?@)&`I_GB;YcPW%vRlS!lqCVKTIoG#ID&M~}ECfP2u1yIJ`#Zb#s%ExLw{r~|zAKojvG*IK&=QlnpR>&;BXTvU@%y&?q5JEk zB$*T>tE}x81|{s4BKy-K7C`dDb;39DR%!9|87GkewTB5Bi{PW+gqyc?!p&RmTj?se zImi)2Md5S}_W$d<*=3~ED_mkZ)K(odD4;IVwP)ZU-{f(NO?&;0)!To=O+xPEzFSsl zKp2u6Zqtf~lZIwlS?KC#l|T~)e2nt~ybaiy=Ja$Bu@C38_a&mL(EUlj`<}RI@XV)EE?_&mfkC^_SaO9 zU`eg&XB*_omRI4*wo=8uk2~ahMtJ3kj*@O#yI6)~+>LsHE9q`=mq=F2OtB-t_CW74 zoxFky5AVT>CQDdTeKm1A^W+xrRh^CPVTj?2u90Hj@3HCS@#mc6|9JmX{M|?WyII79 zSH+DwwfLPJMP)Xh>WQ3ub^xlx)veBzMWziBT&#)`$G>HKyzXcYcnuGg!7u1v(Lb`j z|CUhruhx&|Tfr8F@M@<3l3|vrHd-Z{2Pyr!44b~bB*TIglA7N6bF;uzJt04Mi7LaMN9Bb7^z8dO*RDs0XuMkfPzkE7e_2ba7GKc#d3cqC`1e;<#^qSmtry%4q}uF)THbEY!$~R9Yfmt81y^h zaEc8)GGO&DPh~zqfpM1_k8qk(a3C7wshod#DvoXRjVu?)Qw<Z$_Sgx)Qn*>KU8NK=?mTMMEw!z&R`9H#{qQQm#&#%QLgH?{ zU8Cwf2iKT|qTA-183Jm*hDGn594qs)CqfU|>Z44-c>_~$S8!#ir~kz7RkWh5SBX=H zaHGA#s&&&bVd8iDruf4&r)<0iY2R8^f%ws{e2*%QQA+(GtP(E`P$K@DL3!LD4Ljq) ziRe-VBgm>!HM>?~l{QymK~B@`;p+Zg%fOQs2aX~$!?bY=tjqcc`>*vz3AEmjDluJd z{bfAA}v;)jif_Apu@NRvJdlpCO|h=sZZH*CB6=sSYQ&;SqvwefDO3j&% z*f%4zHfK0BcGr?`6LNzO4Cy;h<6oG3Xng5Ovo9|vs` z+6y4F<>Q!5V%$q`u!<;#b2?~@zu!X{-K6Y;8rHobCz7=}|I?s)V7dfa5MhEqC+2ht zi)VUF?52>{xlzYs?-e?lEQdgT_zW~~YB+?3d+VI+{&jgmZ@?Qx3Ht)H7*!7|3cZ16 zi!wc>TkTync$rzEld`Aq47GI}V-d?pRR@ckIuh=0pL)`ET@(lgm|@OeBM%pk3vmHv zz5g3vwy>%e0ll%LIU;M=Ph~mddRNt;1q|!uBfc;Ec~HCqhK^%ZvwQv1hmtez^%$}p z1ZqZEsQfQiWe%%y_Ms-fCXeH^};yJYAjTlv)d&MVl-J`-#Z*H)0DL4s!>a z&hY?)>+n(ezW7IjlRmHm{e@p!0#8r-nk~F*>4-e|pN?z?T>~T%|A71A~`U`14*Wu^`82mX-x}&;9l$cHUu{5ex}09On&OT5Wzc_+{vJD^R^zZ20OByDkNdktd(Kk zwS^XjU}+u|RP6=z?m@KD@Ot(j+R$<^$oxyQmA`4MK(1S|_3EQa=a#0s(YS+H;vORPiK@sve~kKIqj}vk+XuyfMOWYz6P;ZSFmN-Ubn~h*N##hkgDMt z3!K!21*IEyslBbBhtXjOdw9j8u}ziKfu%=orawbqZ!Xsb1U31oi@7GZ=Bm^=KD(JN z&vpi|uB$_vZQE(mP5>m4o4@2wB(d=R;6r2^PlxJQj%+}vYQ!Je05Ctn!cJ8@Cn22= z91I{)#(BiY<01ClQoDaPg0R&NF!a4_w~`SdR zqb{qm!PAI$NQIa_8a#VWlU0JUIS5&Kk;K|!=vj#TN1DCHP?2nVW{KFno1-O_4Q4gG z6SB}wyYT3@W)D-N7Z;$rj<~8BZmP%$3T)Cc>spDBQ%Pxhqk5=rD_0~ZEMb-(E3#AB zB}PLn51diick_At>pXh4mG3}hJMe;{u`brUF5hK)4 zh8?yX5b@!g%n-Bf29qnE_@{3!_w8ea9hF;*hVtYDB+Qaxg?4nhI4YPy$B{M8anwg6 zV-izjZ@KfXsF)Vzrot-Ouw`+tCQO`lCL8E%eBK~*QG3}_{}Qy7%V$s7^nHyHQljM+ z;zvYV!`b?x)C zVS(pqdi$xCLGNs1pq}ODIyW*re6}amqU2rdwvlczc*>u$a3A`aqcXVFkL8sxq?UB_ z(w>UTVHkV}QjJfgDTr>Qx<99BP^<3CE<(3XO`i5m2}sU4Q=>*7NNHJC*-%H*`jSL( zN8Z(XB=Jb@G|+^Z({ksoBwFA4!4+R9Ym;PX260ADW22FI(zx6gd2kUz*uKEh)i=B@ zf=x$h48AF@`MP$#;R9$^k2A*Oh3$Vzshhr7Z&}cmkd10;PzGGa2fS8@|GF zgl|{BexR3N#@|&Jk+CuDRjU5SB7R0DXD0gnU)%SW+m&ft@~cyxDp9eAS<#NCZAFdc zNXPT_iuDs*RHu!{^Va{6qBfN^FkgbUcFfo8U$25(5ukJ9{A5Hn)iZLd+7Z>qA_=>1 zSrirh?$+&X|5^Qw9Ms!2}QtjNbOrg#L~mZN+)awS)I$&KMd?>UZwAwABOhL{Y!8rozb9IXojh}w zDiVRzb(B=xaXEGw{JG8gHV}+xyPvH9##ntaz>qvdjlD# z92g-1yhf-RutGs=Q5d!U_aH%5(JWh!?+zYNr_}q$T1!EsMdT7(Q-oNb4YQtZ^U~nw zu3bJmi>V#&=R2Nq2!iw}IKzS01%JBN-oiPzbG^J9&x7B_AnP4*aY=p8C|zl=BJWK) z%9?mV8?@+CljV*E5laN}r4yv()Qfd;i30$UX0AvPk+5kir%2SJx7utXBSQ}bQYT)# zE8}z0{sg8(20uDMZ zt&{=B*R%xQFFc~P3ht>JBIVQC&mCQrb>FR!U8sR0cI^FMU1m#a2$w6s+uq1Cza$iV z#Fd|&hAL$EuY^{YT#RjOmz-i#B?#_DjEE11;q*b>Bj#ENdznEa~{6^Fi=aD zxzO1{M?g=?^u)*a@>Mhh{1QI*Q(_@XjJ-lRm4)K_zhY*~{nhSPKQhgP(_# zxGirESe97(J~3(B*>Fl_&~M6F>tU1Zut4R$3k52q6-cgE7&a_#zv_Mj?9S_?aaEZ` zF@x7$=J;L6gQeA&<2)9p^sn zh>jW*m*2v7*S+F;{kQr57i~ z`QI{jo%?+)q*}S-Dr@rmgJ)ow`{6C0Hc(?E0F!Qb1t_M2H4J{P;9=+Ahia{9fF>fl z7ps=kpowT;tBen4BAQosX`RBEh_rei%2&wyx6?r7Fgod8xrTW+2Ct)AE_c|vd?Q)_ zw#8RERxhI#c`u*f*E|24mj_gRxCd{%GFmzq zhFBTA2z{JX2=F|eAyibd$STd7K&e@TDA~9p*t9LW7gY6s#a}kd zL;l7O4Wv*RWRQITogv0R6}2Cza&gacaQF_7`VF-x%w;Wbrb)RsRB-UcWe5G6Y_M#* zMKmS)bNiN?i`JHRL03Uyfkms{O<2co@e*_aRU*`FYGhQtM8ZLznu<_dLJ(==;GgTn zs6&2}QSDbAJ^(D4i5jeLKnFwW!HFv2mI4{G>;R^x=#|;T<}aRx2%E>K*=GrC`3;;% z3!WWnkh^XlZKFv{cIu_+Dq1v-Kas3)W7LO<`|3V)omEPZeFJFo=aR+*hvEXA^@jbO zjCCp=FMgf%v^g}Cx;40%aRM7fRAMjIy>qWa%^4>)0G=7^LZqVxzYqEcLqKoG1D6<2|nv;rV)A{GyB z4%EVr-M(Q{S^9P>y%5_+t8f&bAJ*>up5psgBZ^+7fC|>!LVyi(3)b(tj#|MSdz3P_ z-Tm_*2J_8BfjM#Ho|S>9GJ10xDf2053EbBBFB*R}XnM1GmCj0)3*+$;gmd-;kD;kpo!9bokRT4m0u_1q(O( zs=bsuvm5|*6MD%U(}?jL^J??BT<}YLd{zbf6d+*P#ZXoG@~33NHTE1gg>{KHrYI<5WQsE8Z13RwqKy|w^jz3G>z%$yi*pIM@o8&}H$_xoLl$!z z4R!+944Trrk3K+={pGUcQp2*v3mJnM6LyBqwwGg0x9hbjSK zwTQ0axq$vo0;B%*OrOfJ#qB1kbs<&viMktJJ?&2RiLmqO?fc*O0}w3f2QpVp@G4;B z2l>tQ8=F7CYiaD=BXqM-{B>jP&&Q7pv8SYQr=%ooPKOVtZB3#@(}NU$Cm>fUem8Y| z74Vp@gf}?>uPjHeg@z`vIoM#x1wmBHD=e%a&Z~U=g3v*&HBZp2o3c1v&@O_*A))_~ zU=2sDlfhLh^X$%O>7SHu6Phu56-l)RXWB3 zH`GIhYs}b;XE)S_)Qi5p(nLNcSqT`ii`texh@uIf^Zp>i&?O|@oxvl={uSzMKprx$ zxJNoU%>M}sj;HD~R6#c6p4REUp`F3P@ z#SR_UsWj%%Jfbh3KnW%Mk-yx@EB%paT7m!WvUSjp0&$;k=sw58Z)bPdt9~>e*Ry^& zpM;Si`7WeJ&7)4{fr8ot=`E?=?55d<>7$5_*0wy3LW4^UA+nCuT_{Y-#;CcHvUqa*C<51%HK_p*t)4<{6uVwmUqg2sZfTz55QY za83xD!}ZiPNf*f#F-NhIb?*~1>Z0j{#NFb@{?G4mpH3X$S2(JvyBEwrr#TaRg7Wf+ zGg5M%TQpIJ?CXB>-6YlQeBO3;jI1$u{^%iqUWupKsqU<{1fcv`*Nj>0f?cS42l$2# zfEv;C>U?4@cH2#>o`|J()CR5r;oMpCQbMMEG<{;UZ0$CW6k)m}GG2*FGmQ1E$X-=E z!NTdkbje`$Zks#XGjZ#j$maKtHF`!;lT$AR zfiZ_5M7Ssxu-(aDshmRIhQ5#jZW}-J^V}%*B=N12bHWHs-#gS&zubCZZiL5oJU6uP zJK5nxs?0BBhcc#)Pt2||!&NEK=2dAm_r>no?%>;2149O4ucy8g7yKJR2v2v+KOe1D z9_-C6?4RBoej!_4H;jJ;fEIW%*9#J?AvZ)xA_NvU!+vA*K9hfDjcK5o!>$QZE*tG(VNi)n<&I&8sZ-M4{VN|V+w z^BWp#KnZ^#j@Qd&w!Uqmf;x4>mk41zB^&xp9)b z;tPC=ejOYwHx1d`TDVjw4I@({lctVU|3Rhv5$d zptqu=yD?X(aW^u?e)y|x)gScM7tG{b%nDaPNW(nLb47ix9qaFXJupkQxhQ$VJu+Uo zr(M&1Pw2sAe+$=zTl?ULvp&z#`73_Rk&{Kzm0H_C>@gm7teLwAv+s@2iN^TDFX7Ty zj#&NWGRm6ila-qgb$5AhUT?pcsc*F&K;v$L^B$vz_gZ#ob5`lMHT${mYh&13=zre3 z#}{_+m&0DZRJ>)DGP;)Ayj*2|c5csh^rBED>wqi zc-%bcOcRAa9w`h6PGDM)6x}*X^hA_uvrY_&8rpN2dE2f-O9H`~tMJH0&Tp`KU%=gW)eRQsR zn6bJaYUL;l>eLkO3R|ynPQus&vA>(fk?v>nZ#C;3B6to1NzBW_&Tp@cliW2Cf$EDk zTaTA;qZ;N`2Sx428Ec|kqM!+9HUU)VMx)K}M z!w5?$VmrQzMs9Q?A-oPZZi;CV$I0h?6KdB1RkRGXv&ZU=>}na)!a5qy*Lpx>{p-0f z_O1Ae1p%+=PNo5|PIjpx_1~Y0Td2(IG4NhArb6Wj>Rv6JaOka;o{#?BLnYDKTxJ;2 z{Jv&_O00rbF(zu-HCAX#-5d7ztWqvAN{k4=h;N0fLs(m zQdPp>56z~HCYayb>p%M1MAD~H>?063QenaSWouvRK9-gE*s(!!d?S9%Pq_cia0;-7 zK7BdjF>6*$Wa|R&{?T3KIPa5OlX7)u^r(GVxk0VLv;wI;-T@H0enIjS(-{WD7E#RA zs9XRaV$g3^&GjebOG5)nD?K4Ee6??!a>|VeOj`b)y<*uXP&%`y<*9UsmwjECA0bNY z(Ush{8lELR<=-!_T==;Wtm>cqyNUFVZZ^IB-?NhNkGwUmA|7&E!4pK=aa0*&+IpHC z_DyY??6mg>QS%GWugSl_w*^H_r-#Aa-2=Z^Ra(3rvG^W*~nBlWg0)Mm1@rYqV!>=0S( z)3i5+NijWQbx`$z=7uS;9cK=W z4=)-O4F4vBYI<^+{V;$2vuWl7@$8pw9T0pVj05fQL5XGh6%|}HTBCO%rd-tK?i=mjrpmktD`5a zqDMJHZRtmUAR1TZ*q6HjvCm=?97^SezE;4zsNG_UJdO{j2%{R!*Y{{x6xZmb$(y)u zi-ZkM&Z_Q6Gak8O$_IxO*FV%4Fjs=jG%bCp+>*`p^l4it&nz6&vqwyWD;O^v_m?A$E30KyqE`=VwWU z>qF$l^uiy%oh#~+i~Cp4H~bK+w*Dnq$m_M1gBI}hDjwM>DvnyAxwd9K6~FglTX-%j ziZ1)U9Fj5mzRgU1!M69>+Bf1V$tFb)*ud8hzIR@7Z=pV{u>OIQoW+#(mW+y7rHP1_ zrG$n!BO<`m)Oc{a)4@Z&|5a6P7G=ho$f}|rU(_Z~zin9ya_P_OyIjU#63n^Kr@d(1 z>$B*}Ju&OWm+rA?x0N%!t$cCmDD_n2ookw_5+n2dG{ zuMs{vh4i^#c21U`(m6I!dJ2mu7~s{euQg;HD4o>54`oHdrl8Kq*FlC^*+G)ez>Ke=qc%$X#hz_?9Hlh?ywd!brx zH8(OtL_&D`SlLUhGPOB4()AMz(C_LZUtHpS3mw-l%eqQy{Zc-Ble^Y-G>8I_&2bf3 zwN9{tpN^X`_`7s-XDY@E_tQ2wtZu@ncZy>vrYZOG$m*K#cXpbI)3*w`z@kt_)sA!o6HiHh3<&0@>%@KPR@GO$loT^3yF04^ya) zdjuvl%ZSN_vralAPqePDX{FOvV-EK!gV_#wFiiM9gCBizcFr$#e*3fy>zpj)_Ryq* z2%bDo{V@##Nfq;Lj9XZqUG0g*Oj%<$T7-V{2sdVD!_&;1{r4ghlm3`_`jm#-_&3v* z*QgFkSX%f6HpcGMD31YNv#XuE)dz<$t#by*p(jlyQ>O_db4ep-iIf)c2q2+4Ra&aBXvJ}|p{i5Chr4W|F?7oYHk?z&{Q~$xJyKka ze_(peCFE;mTlSk~;Ga>`6_%wy2VZHQbP`)%NKebcS-=-a zuX7jrg(b-~Vo&f-mw#-CM+{Qfk8569*qVIrexm=)%G9Wr+7p8e-%pdC3*BB1V~$t6{_9E54%vrJLvERil0q{D1obS_UT>i|EV^b+f4y38y=?%X+E$ zbkog+XAzgP!?B{5ww=%f6LPC?`-hedrJWTk@lqzXw)c2YHk*e8s!!55NVV_R_hWN9(xt_WxzQOLCrn>4)uA-zr2rj>~JCEcc(@3f7 zuCSrD-r9Ny5p=bgCQKpwceqARyOtwWl-ozH)^Yu8U4clo% z=A6|x68I(9Sd^f%peAPGFTfO2i|h&BVg!4q{GGt~3AX?gZTrWTxcRvX@2vn^|376S z+In;;<8vDaHZ@9g2FsT3+-pBM{`?$3-XkcnxL|peCovaKi)|-;4iw(p?3y{j=;{H| z#o@b`OdtIp(=#7#PR0;sy+AdZkE*)WyNv`WMf`~2u!t($MQv6awBIy;3sF6=R_5*Yy#N>+Lnv_fcsOj;V)XWS+LU&U0Bhw$PzdbXk)N@FxJji>cQ023L}*sg70m+*4B zLoV?|^OaiDs_>>NO9ov@RkXE*s=NRYDssg9+4x=dglC6yUXxs7DXhP~@63m|kMc|- zGqMCDqWQ96jq!Qb?_3_N=1XGATN<2sDqZh$O)Q9xhS-}M z@>8Uvw>P(bqYP*9tZio&)dgxfU9Mh>?H% z38CI`D?iS>n*#(j$(Imwz}4Q_&)4{q%28(*z zg=0%=6lc-7i_N@&gVET&?v0!+;7ou*dhY3*>sP*dske=@{BO66Y? zFAooon4HoKjpnsa?xp2M^P_i$>mThZOKW2MBX~c#o8XNKyNg)NO)jFW0Z%?L(G^}3 zR+Bh-qlLj5>zeW~quK${J-1W7v!-67x|+lw9><8%ZKCM{b85Cx^-zt_Cbwu*#Qbs9A{$;yee|sPV5#hu2p$9hW|bgW3;JVpCp%OJa?tvr&ev-IsBvS zy0D~x-`G;$fTb;W?$(y0i`NI}c*!-Ukk5njzImvNt|Do!pP7<{sautPTIZO3LX`8T zFU#o-Ohr@ZtSkOdHk;If^{B$t3IjDwl9G_mN)|qv1NH2f>tkbLsX^=U>FKsq2ukx~!&L$M9ZeKnwJCo8y0)bb>-)|S57hKfSE`f`qY;*c!) zRpa9$;?8l2udv7N znZ^Q{vpXWzQ%?(=>WfV&4A2poHCyXRviS}I%vFgC7k2YdVE2-T(VFdXv92wkvfPs=0=Ld{d6A%}*{aff zj0W^-!mp)ffFfj$hQIMwaq~ejWc=z^aSQ9Jf1px7>2viO1I(meOSg6vy98K;Locis zAP<;AT4J=jtwby;9yKz)jlaM2et_5l9yVXEqvp_9oUn`8+-!PSe$RKU&tavlOXm*xZfYSoJcd^!=m#bNx;$(o!wr^kZ*=y(pMiX^}72+Asi zwU_Dzjh_*1uB%#ow@_!2vdSDRUh4FO5WjHnvC!FkO&cA5?KtB}DeCd7VWv6~35N71 zUhrAdT0rEqPzrOf?YeVbvYvx_0$0%GP+j0M+w8f}x zCn4VMg?tr?PlGbjfdSpaEN zF1qsxBz%Ls(#f~<>wVZ(rSIZ1at9wwzL3oP)*qp_Q$gop$y5|K&JATE{p6EUcjoZr zx8;tEb+%WGsS;i`nS67ZEBPsufc<~EdhVd6o2G*TqFBI&7*G%pktRVz2oTx}3IYn! zdkqkZ)Bs8Nf!Ga%t|CQx6~WL0L~4+#fP~(W7J3OS$%lS3@B7TNGdFj)w{3RspSzpA z_4Z$Z=tD3@iwan1+m~RcoYGA72^q^&JF#{CD_>mzQG@d4>SeBTFV5?9&LOfrS*RbA zMrOMV51=dM{!7*+Z9x51OH65CL!8&$zRPw&iwkBMZKGd;9jRL<b@;89bg1%n3)B5+7iKJD#~2S+4Mt&sgKK4xAAT_OSDkCa)Nf|?*{|%P z9GMq^jT_qzFC&90YHG~hDT?Ag(FjjddeLm);uauADMo+xk)>X3!GyAy*$c0{`|eg%_Bcp(a)yl9q{F|F|x8RH4xf}lq({UB`$x!vVI zi^jI=p7;J>I*GXv|9z$2ST;#>3$L6hATgz?XlPeXH{jn8%oESJLoeIW$&8qdY?Ayb z`|Q`s+J2xH77nif8tLJ0T%;s}{*yw3*&&0P-iLqild?75G)gJWH;&{*bSCQ#K+gdM zeu!=IHv>Y#Bjxa0T;iY5scJ??emy8XZTfx13xh4=!TV!Pu155GBE{j6&kDbt3WO+r znF`wi?^XIJO=P~IpLdU|eNX}s=U=k$TtJ2Is={Gw7j}i=sGbUG5c2wXXRM;IE-#*N zu9tewCYCq3>G3C< z6O0E4BWeH~D6?vKKb{l(S+ixT&ddEd8m!p-Qw|0HUf9>N2B_S)*H?6$ zV1=i?5T6)iX5a>(zQ*H&9BVch_c8}Gs$jAB{j6Rnd|&h&c8%dHZE%c2m72Pq)GpIFWy^t_y+$59C zxff}uc6fUe7l(Y=3RY$8KRnUDAo$bx3Z>6nQIPd1Y`+C%8-)U^|tD zvKUhIt_=-O3dnHPzA6hX8u_MM>i)7-_qU1*4f?{V**K+3nT#0rO6OIiohvA@Ze=?c z^mc0I_)cBd5(oO0*I^NhGFGRp0h$yTBRYpP=ogiH!KwSRhI148)%9$ARMuU4n1KH; zxE`Fl0rJA}Bq9z-;ch?=hvS`Im>ngA+FW0zr+yliWM{E03lq*#j9O7njza|~5aD@_ zUPrViD@r)eMzq_w2*z>?uyeR*&GtW|1|n2sZixGD=9Ws(6(5NZOgmGg zJp;+RpJGqPoQy{+Kk6w7MOn6=)StfrlP^vd6AS5wc$GZo9n- z>sC^^)5uvAF5TE9cY zd5qig@D6Q0ed~T7JnFG5mW8(7Yj6rbzGct8^{}V%I=ZR%Uh3m7Ub31QD_N%R#J3;l zud^x;l}#J*o=*Ulld<%!Yj!H7>TPVNTWgW*i(kd4%$T*@O+-%Fe(yZ{rL%zAjJ$j8 z!m*1er>dlv{)lo@b6K&`a}ML?c$qi~C%Q?<#m(ZdTBDlN$9LOru29TH9mWK}&Dfec z=^aP&Mzt3Bnw%QmqR&vtALEH6LVg)-(0$cfz2Vsx#4iqPFKl#6v`VT%daUqfT*^-~ zNnKS*CV+JMa~-;T-3c9uV5}jXNW6*!+eN@@;Zmm4MqP!U5T7!e< z*T5?FXHn4hk!>%k%d0zkcP;fl_Q~`@CG}dsNht$qWW>rND+A}-D!{!V+7;RzbsR5( z(3>byK6!SZO^sr>X_Lw0rVBBzM__-T!f6LA3h7;ZN82Y1%j$P;4M2lr@9O5~Y`R~z zv0Ms+hGC~r-$~m~6gGz}*IcQ4Ulj&~niumH<^7Cx(Js+UjN%k#wb@jiiJz+Y*^_h+ z{A5AzTdu=%-wx_IfHLUM-T(2fE7T`e4@lUWJXYVs)K@0pd}FHH_%4b~fi-wJEO|H< zF{C+o9#_0HwGjG(`Bz`AoLaIiX`5RsI?zcXwQF72A+x66=Ilh*3cFWu36z*cG)k4x z#-h(yjf~So`J&s4cGJdlW})1BwhH5O%RG-%w4LpynKb2r@q%Gn-nzjyOx?SRYlK@% z<)_;aZ_+w;_mMcyMD3%i@Q66aBd|J`=5o!tuCi5kg0OXY9bqZmg1q`UPc|z6C5!u} z_$^pSQ~iU1$NpM{SMLeMo>8-0MJ2?ypB^Ue*>`C2=I>2xoYbM>Oz#gU$>>#yGNro*_3z=dsh2d@;o z#020aI1Hc~!#^{6e`NI&(k)BwBP!pn%MuWmwm_go; zCC{ws6$9DfW`5P*O-em2+}Kk`oe9@0CyZ}Ea*?{>WkY0DYP-MfU{$2lyJ964f~9$I z(T{zM7v2SRF!zMio*B;e&Pj_dCQX4Us{3=8pUJkFsikh;qWVWi5Lal+WmsS07>$kN zu>NqWS%>D1*6Jt+n#0Nmre;?g!m2X2daCw!G*L2H-f1bkt3~kPd#c)1MpsC^fS%{= zA%wIhf_ACYE0m=Cgb{l+^<|=Y=}pdDj*pA=jHJT+??{6K$7yoz*sodU_?5o-n6<>O zjx17b`WshrHV3Cf4#ac&Y;h%#U4}p(E)yW)XeYyeNLL2r#)eK^ zhcpGiI^y}q+0H-$?KlUVrRk|71ygQl^wG_Dpx|u zb^BI#b`K&;{hu-5ky$r@4kl;lGo5}o_k)>kW{XL=by6IUuix1ZJ8#E`lJqkVwU08N z2Z2Crpmr6qewffwI^}RCcm#wO2^C~IdWgh7e44}ZXAraha_euu@#^#6KK*s&Ujpkl zY%ItBu+vIPmepdO;2r`u9~%)8O(>;e&VD_mdo@eS{H)hG>i8mA1w*vXhxD4JRzvQ* zRunnC#d~^RV!5r%f^qW}Uw6K%-}52`d-4o}Zwq~j<1^ouR(|qt5#I6-HTHSF_QSYi zi}+-BE0N&ts*bx$%D~=VQYaJ=?Zm3x3A;_+dxl0-+yV-ufM-OP?i_xEbyyL zN9ryS_^m01-J2^6+_}TdiHFR+%LQyT*+tF>RG$=%bHzs5j(!M*V*4RV>w5P+ib4}KQ(nh70|64W>!AWrv}^{;gOAx%X4Ro0v`2G-J6O zKlO**pL3vaiOFZPxPDuFPi=nK!Jl@t%5L<;WFf!w6Q-(OxN*BlcwdU;E!Zu}Sl^s|qMyuBPOo_1Xr|Em|=DAM?{v;$ebq;YA`a`ZN+#db-n z)j4X8Xc~q&>^~OsNk2cb&ZUjmV^)WVaphMVeA)Dwz0UjPv*jh$HZiDco;2eVM$8Bm zW>|mfsW*XamayGh6i*N(G+hXfA~D@kCNMk*kKQ1=T2dPYe6*2(jl=jAEdTtHjOnVj z<&UJX&>N=65oA~GoYqYwwuN-&q@`-{Wc;9=VX>K~2HkMuWU(al;pE?O~tv2caTMh73u#AaUC#}5ZJ<^xIt~=+`O!L~9Vc7_KHL@d=%`~nu z8g)7KHS4I*p9PtE_T1s4FU7c0`Av?m1J_hQmd<&tr}vTDM#1J^%%^YdNRGRMPO?H+7u&gKSaDOWqL4GSaGIehiIqIdUEc-cHfiYyOsqM%;8Y56b zmniQYN7!2VLwfC_Mc$Le)^*Rk);&p`Pt|u59WjfZTGqL0l_{@*SrT|lYuHm*;wl#g(KJ|LXx??RT*$-$FJ^F{ zo^A_x&m>&mrCB}uZ5qkTa22&39eNX9SyOb zdbE!CIK@bPB8bQyCRe0rYFJUEx2gf4 zJ*Lr&Y#CV$b(PxKYYDGc7SmEUNY~$B%_?(|nqqv;wsjn`ssz0-2){4Rqdr-m(wYRv zZ#fiyor{a9hW_M>!V~$~M@?++!!@Xu(jeP|k~f|XTCK;-*$!a^4{6&*4{=T5qFd#bLWKG+DU!uSvOQ=YnzA1y*jrUYt3xBW^UhH% zDS5csN~(FU243IA|3}QQS~aeBUsZ^v7;m3>RS#CvFC<+5&pM326X!U@xoaFK@ldaR z=sm(nK1!HsFLLyR*QUqZo3y+S=Ssd3mcsVtqM~>7gyckKEZ@!^a=?mcitN zhAl?AN{!uytXs7C^l>VKmeePBF&+ua5%?5!F?DWvOa_fb@^Oeq*H|%ZAxhMHQ*ytE zKqdDJfn(u^a2IPob+%+);1>&gGKyDc_cJnVGi)~B!3eSkKBRJgf|g`J$cB=U?q>Q& z!~LaD39|#8w1s*@@+H;lvY`)YKnYzj$8p9(C-{@;3i|M`+vZCeVwriiVih4)IKmX~ zfXC?CYPlV1f17g~V{}vXZ*di|342-dxa48s4+^XKB}#lX10nl>*S5?Oh5T~z)R#X- z!%GQH1Jr7ryrmVq!RRy4>yO^P#y?7hx^%;kOMvC@RdaVWc_*K`|bOBuP2=re9^ZTke+& zR-{H=tLJA{ZBZ9T zS;LEvyokNMSNotKqoJ zgG#!D!5Pz6rcx5XzgIIX5uYy|vbISEMn3W9t&_bT<;;?Espdo_Uz+F4ir;wWMJQst z{U!mAExIb_?a9xUI^S^%km?DEmV;fCWdi1-9T4-Jb=>mBRlWg{uSy+v%f@>?YKud^ zR$DMUCaSu74V5JVYpAH;$|0kMVJ(hLe4qFan%`d4|5|^s%x3fK9Nr8|knoQ3t+@O| zF?jD1f~aG`pJ4ZUGM-XTXBQn5>w2-Y$HV|Ya-Isf@3SBR?5>N4xfH8}t%gxJ6iEs; z-5d*+%o7#z)*h+%u@-p+{4Fg*APc(Hj~I6r6jt*p|kF$IwQ~H6zD};!u0xhc<=Z-MOn^8s*oRbt}Mh+!F@8r zbi`*E`B^3gQywotW!blt)trhcPb|O<#9nW*6_l50YDGEY(HLj8epT8Q+y7WtSUuS= z|6uq*=iysG@3UFl&zlG1MP-=q(|Xl`E-?!zk~i=>0aNGaQ*_-PZ5^{i8^ACQ9`_|m z$6k*$LgT|v2MXuL$Da<``+w?NnqHIjDI+ z_5GgQ`7Bc$R}1VS73d{V2Hu7j;6<|(-Xh~m*BvHWC)}7~W7nP@;b#!91U-e0cZeh3 z{)DQ@AghPh&l1SSCEjaLWW7>BG4Y2hE=7as9{0o%?XiYqyhH+IV=OuM+HbbCMeM6L zPnOdfp^)3P2it74WJI3-p=;lWk$c`sfwlHIla}Db6_F4!2JBGnBYa3-oF9;U}EWWKwC5_>i2Vk54!GuhBzFPtlS*ji)(em4SPfhYAmHM9`263joH)-NM zA7w+DEt&fD*+1=tYNm>A``eTh6)b63hgXXgIXsmh`HtJat&V>LT8eg zVr>LeF^Y0JG*>qMR?r;rrUE>;hY$I@(D3SJ1Yzrbrl^o&r5D5RvdMN6ptCm%YBk$QbH=4)g? z*(v0NleD{yl%tMSGe1}E7TuMLDyLVY9sETqTT>{)jhq*pZ#ay<#G1UNWYubD6nnEU zb`^S$$$BTOoKT*5U-pC;-O0~qk#&G|@)NQ|-I=>4Hl)=?QsIBH``tpz<11v4Z+woC zk|Z1>+_A2>vj46mwEyL+p6OQrW&yZtDc~V0ty35EBo@_mnjhb&ZASJ5^YHc@zWo>^ zbYPrWI%THcKlsO&Z(LkCf#Odzd+e3mm0HPGUYX|9d`-2N{;>F`PjwfaA4-mQbEh-( z1KCQvz>7Is5QOvft=r0I=G#~ijV&K4Y+_E@9LuH}4_{Hm#Vns3~-AZl)pv##1D)xj=f)P z>AE)n6@1Jj4f>3xFV!K~S+g2(b z95zcJ--W7>NIwxtmJqB?#}>Ul-#BSSUuPK#blFL;c{B!y&N!cz*V{~*JRQgG{Qg^B zD#gTgTX-$|@s3`-%lI*dcGub0*V!jUHnW9S-NV7tMX!SZIP9h!iQ2s^RGj6E(Q(XR zbS^w!Rvu1-Hl1iJ3|bGkRa&2E%^ZoHAA7ys3~gg;etT58Gu!!NoPVqyxf!3gHO~L? zGR(R~(B9xCcvn+wxD*k^mS4f|r^iWIY9^*FC2OlT;%TAqN;|7!J*v55 zR7VYG7c0$-$^32{Z9ax{#9SM7K1V2+hJ|iwos4FZvZ%Z}Xvbj=lc9*_C+wf%ds-v{!j!mRluGmCgf#fZt5{>2;fiBS{@2Ud$#TQ8h~E=nPB^&5~w`kfgmWzM)P zZvEkiIAZx$f>GZ7k(O8Q785dP6<@jMiboFOrOvA&!kE=ZC`^A~oD@|{h>N%QJXel> ze?xFCHVr;ED!C@qKH5P|XfADhbSYVL!MuB32ZZ$8*KGaJa?j;ucXs33y#1HO=kk-D z1Je3xbG$vKPRN#v7_u(EJ)DGp#RHb(FE>WsY}2XErupozzR<166?@GnZs2&T#CV8r zesjO@(Uf79gANl}$7(Y!jWi$W?5jYl*w%I|zvtA4f7kX;U-#3{UK#mqd|Sm46SzE@ zNY4Z0sg5^~nJ`5`@)Y6!=#|gtFW%QG)&27XeC|6X7xu7xV>NBag<8Eo#AwhpAc9|= ztN0|ox3t6W(ElAOyQ9mHmb*4-_z$b4^(8AbkP*ot7_EYFH~mB;XDl4Ttlz~*2}}pd z4#zF2r>w?l%M|SS*)B{rcrzqA^4ld^wb=nO$6rkshwo`rIxS{#2a^48Vo)8__59g~ z|4VY=Atng;(KZonrO3PP20)c7@ zLtz-W&}<5Dp?tRsO`?bul}X+6vrc7jH0=-JQMzd3wcH#N?Pnj@C!whbWRkakW%q}- zoVHzHz(Yg_WzDFQm=jW=FT+dSnkth@-=E#xOOz_qoBA~S`{59j8?DLQemX6X{k@tI z!`>b-p1gB^~v?b?TwGg`dl>-hF&iPVC*w2-M53YL9zCQ50J4NzgWe zlWfSK1v*?-hE286nMAcb2FiC`O%RgHffjVgVJ%4#vk*dBd;g`_ZZggzN<1Vi>$X?9 zdG_ad#Y5+nmAjr1KYK(6M3lp^(EFn8EFpjNvn(75g;qiO@j3Kx>NAWCe>K zC!L4R3{tzv>XJgA!=n_0;Oq3t$usk8ZlZ#eo?^h|U|3F$Ej+2ZrpW)UYiQK$PnI?A zq0*p03miSSP5KT1pQs}0+$*myIiFXSC8nu5N8v3K#TADT^V#*%cc+@FfxMnLwNz*v z51bQ>6r2+dy*^yhlGpM=MYG&JlayrGF&)vGOG+4**vQxUlM7XF>UvPv#k+=*ap`v% z0C3k;h|x3t%U1S$HfNQsrdmhwhd$ASP%vhUDVKvWUu9b9=T9dD7?W?)XI^`->v?%4 zlwbT!c+PeU+d2ogl94&zu0O0`q!gC94k0?g5$ciu`~9io_jT`HI}0h&O#ST^W9(>* z%7W#}&wczsQ$;?;M2os88%4f)XAgD zD=Yzhv*c1W9!uLexH@qu*A|5U{~o6dSKvS=nZ`w;BJ)Ol>UooYliXn~$9@z+1|~{e zkrwR;@fmPpHxXuK)bGm6LX@WKuhpCfdJoc>8p&G$Bpoo|VODL)A;MCneAkb?S@wRW zBH+*Bdyk@a<%ckVxptwsYjRhg1>aa1i`ax9v)}O?W#WX_V4N!*g2K6-sc$xuC0MRa z)tm*65KyGbCK!?jGnNXjm{K{|%shh+*B(p+!-b|1|JccmXWE)HV1@Fz>oZIQ6rbBp zSDuWi0kK?imsiik@`E6u@9cxE9#-rBs%JPx5=8X9lcLCn$o%w6>H#B0NkR}3D2rsb z^!#)-EpWJCtGu6AT^2Ju^1OTrDk=VWz^tXHb6=C?NW0K%IJ@Fr9{%9DlLayIbxc)= z)}*AgSdRx5SO4SX2X_g6mUbH)#S6g&8B?_DfGFp4lqvCCQnt(xm_yJ`Of~&_CDWM| zH1C`1F(OB`e$Vg2*Oujksuhm_;x#hiI$8aP@ENZqGaX%cg}deb{Q>s+7=n=1T#_fD4{1|o31L)DU>ph@fii5_we=zt-}c^~Hv9!F|A{6) z@35*@h7IgQG7VvM+xzs_u|hZ zjBTdgiz{oZU}+mOOYi))j=?{I(i9|AEt*a50&Uznd=W2(mgF=4K4+>2oTFvIlpz6A zhH1K&o>PhiiGMO;q0Pa9*WSdU#tknJ$q3Y%7 zeU7@eGUEX(3HwnG!#A?DaG{L1U+|+|3i8}#MgbhGuEP{mY!PDM#jZ{96&_(1IWZx3Htkw&gUYdoMxxv-ncYLy3 zPpff51N*G`A2J?iMs6;Nh7g#JzC+grs4ceIEIoEaoqc)$r&x+~W=)HbFv+j5G{uQl za9Sen6@c!)=I|KF6BM(UL@HgTM-vT1&MmB^*rqs$Sw)*q)QLaeu+Y>8`GuiJIJi+Ol#&fE z+QZ;6YeUnD?{?PwrpX4Z<>|?11!wz};@{=y#0qzfa60ry)*fADN}~1e^ZJ`KbVOEX z;G4+}rUFLg5EP+63`gJE?;5*j_!&{E{`Bq&;!*DuxnPcv--vf$7~e(bcng226a9X$ zP|=OY`v`@pd3{q_(DTP`n>5noe*b9lX_auv4&T=i>%iFNN84O(&B}+YBJ7(&o)eKP zQ`W4!7JAV+Wi3B2=&CC|okw-W4>47>Gw2U_mO#xy{i;kHH>?-epy6weH)B;Mzg^If zMQKL>eFe6?5|!-VBCICftH+)<)M3?sSSHyl{aP$ANcw&MI&^4^ywWCT4mZ(vj+B@< zc1ul4*5Wf0d8Gq?y2L5J^5D5MaJQuo{02&5VoFCFwjo(hOASS0s%5JEUAUzF4U)t# z0~akSQm}g4l4ewZ!&T&q3@aXw^Vh z+xnCVp`xyF59xdKp;UcI;$}}JPJ3G>Zc+I#mdIVpRA?g;rtfWBk&>!pfZo}iA7spD zRzA@CX8ME0w=i`&LKmiB;btIWLeYCmA~L4;XI9KO zw#o&X&VyH8zaM5PxTO(j`BheZ98bY~nlZkhpZAY`@t*_3_nJIIkKXJxCgViMYb?y` z3{XM^7?arRqU;p~_J*JWAnZ?q7m%5s= zY(<$`pGbXL(3hHbDSGGky<$jrTzUc%yQnn(!bKTj!A`Mq0sSc02=aYHu!yo>&%Ujj zZ7%^hmF8{XZ0U**-tZk@k2q5(?9pKe>{c`>NO(U-lB@4$<@qi)WKWuk;>sPH=#whH z2(tZSZeLO>8$7R>WfWmMv@8$)HhBGwg4dt+qQf-8b}IT3RsluWq4;n6gT3=54&t9w z!799$KU(e`P8yFH#g>D>z4k+3_vFD?;?wtVUBABTaRrr#!_0^O_^AxQe@t*Z2(a5f z@oIP~YLIiozERJtnVCHOZ`h~l#V5}oVLTKOtR5%9|~ zPLu#x!cKz4ej0<$_t!c0KPPSH#`~OEs%u4Y-l-tr&5C;U{1*vV0!s z5*X0uVcD?0Q%)Q=Ise@~z4(`u-CABWtCLze{iFpd+~QBz-T&WKVj4$tYd;(#fMu1E zTRF+DgpMQXoGomO=cc*SeVcr7M5BL&H=E#RL(zn#PN1VeaO*n#BZXr-lATI9E=kkq z;3QbO+n2|b(LMLx<1*|Z+=iA{h;~kssnU{V`si0r#|4&dD`w|P)wfsw=)049%yYI# z4<@{Q+m9b8LsBoEy&dTkwcG>}w%n=%0@;YU6Ls2@c-ws~qtEPv#jN9h_Vzr6Wx*`- z&p-Fk(@C6B%0Dljk-kCi)LA?&R}#{pX!{4rvyA;zR2rp^V!tID6~Vs8hP$cMLv%s1 zy5vv>c$A`0@oo7-{XS=lB?zP&USgs{Vc5C1Y~v4mxR{*l>KNeVL}V8GFw1)Ebu}M8 zWG2gP-st;G-$FpOD?_w>`5-3RE_z43M96`vCkAAt&{!Tg*DcN9`Y+9p`47#|2Ngd_ zh7;3~;khJ=iT@IfJRnVVc04oRkW%u{B=(-8Ar-j zuz)6sXo1VL!j$)WNz^<*v7f#>(fc;XFt?Ia8YoGP_nN;9w-b05`?TX?!$K?9YLbs9 zD3woPY?;YjhVO{U#=^GNBIDXrsyij$q5V;XjxIg|``vwJ`f!AN^s+pqnRW|j%9CjJ zQYW=7;bPW|+qEs20ga+rT*6uTf`lNDb3 z)YD~h2zpzi!Yk>IJ`Bqf(`*^#XMGzR!Olm@-1f5(=JL&7$1{3=Ufes+qrEmd8m&G{ zF#{L{%6@EE!Kcq+9x)~WbL{$nb&anHKeos zI%5i0!#8wes@Kci;!Qoz^6j~n`Kmc9@1Rj2eDMhX%RADu*~regl`215Z7A%&B-*`d z({p7G{M_bX?{^wkB`nj~SoBMQWVE~;IMHgmMt)Y5L^~b?eSTJ#b|te9y`eutcq@v- zDhIkun@kaJOZj$&4x5kY~U?Ac>3%HC1<8>N9M(r5ZU(e$+It3tveMT<_ z8+cYEyArrl{ zLKIXvzuu1DQSH31P#%M(1sTKTd?eY-VFdCJ!{Cw)ZSdK691cIznY3ak7h23zJJ5jO|p*v%g&H7s2HB#)~fY5EFRys51a4wtq;5fBb#W0>BoAyydG=^te|?lqpnFK%$t!-#5oe5?KT@ra@Ty9kS z*iZakeAr*Fsyi7E8rXj^?KiEEqaU(kNJeabz_8RT^sn4oM!^HGHf#mgJ?MxwUg-~fJtxtZB7(VRrhougs z7jGP^w=Bc1MFX4UCMilxtwW|AEly0b`2kh9S(5$;{m6> zI5kh@IG*+J!!hqykft-aC#t|!k-qHBzc0zq58wlk)3MRVRg_rX7Bvelc0qGGyG%2m zd)r!mlA}@+wLZ=~yV+#_sD)De{*KT=x~vVpYv|dJIF$EP26OCiEOjfq4wIRKoC8^} zCn#WHx)mLKHRQA?szPDu#t6Q%UZn>1mj7@S7|2!B`EU{jfmxT>OzP8s=rhBd;^s+A zG474zJjJA&mbLx&A1zFFGW~5z9Dkv;kK+DPW+W90%S43A+2NSVlJg8~S=&L1PY9jD zlsg(L4K z<8BmRJaefHjEFOe7oGaCDfUW%Ne*-Pk-9~;uqfkh^E9UPRL^%~u%>Uc^vCh8GM!Lf zrV6vq#;i#ND*n%&QJnS%@})BppH-~6G2I9QTZ3jZU|h5)FV0}U!N;8sT94Kgi(4T* ziV)YmdNcRdBkr^$8b*RPDU6v>CN&SBeXmCQYdkwr$aqJBo_GMuCCB70)q7tgu~g;f zI$OW zpafrj8EC%pqg%FUYxI-c14_ZzVm$As$jGR7TODt ziBpp5S;?nU?u|Ud7LIQ!T2M7BBkrJ}+UX&l^QZR~CYPd}ORk(ic!DfrgZ>ENS)Q%h z5Pk!fZBU)&_^2dakXhrBL4Z;Ok1y|#1Xjaz)v=aO>ESTPl%ids`pKY%ub%Tp(I>Kb z{J({qjhSu_#_%h+#9`61zSyn$tw5Uoc^jENf8Z{BjzM9eCkhWXN{>M}EJ!R-?GqFW z!_g@^DaX_4`nLJ%ksH!f2!mVEIgTxhB@MT1Oltdg^iYqZVs;jL)p-j-N$>ilN z`gFxVo<(O38;vT|AYfm3YIr)yr9${_a}g2TV8 z|5&5{n5!z$=UAY6X_a6QT@OXJ8CX}Gnk_m`AU}tu4rwsu;BS?mLkQYThFB_YYXEj# zl(DxLS&yc=am9B)&fT0qfcOISh_cjzWi*QOK+>8Kd1EO!^L#>{os0MMo9Zu4r*pq3 zeV!$GEBXqsyr{5HPU;zom_f5|)kqD;#cfc&CC^s3x@9r?sZuL71I?Gral*EVGzl#_ z9QphxnkmowX+MqAPgJLVO_gh_kjQU#b3&a?BSPR-4}kXx2wfCgz9Xa*p$oRUbjx#8 zO<)S!OR={>H_0zicab{IehQL2*zzOuA&%fhwJS2UTDR{MLll-+8y{zjIWYyA)xP=` zJn}P*6VStJQ<6v?0%tXg3%XPOqo?R2~d+ZuWhA%8Mft ztTQF3w8f{=<~r_kFa4MKydsV?NA|*O`+ZshPc}3Z97#dt-!{$88<^eCQv& zdE=|+=5n?%ax6}#zR>&E+Ep=q_XOs={NPfYV($L%c@BXf4tdwy7`-ddzoo^}6rXCg zgseWxR_*6<1nYq0Xb!k%U=_p-{J`M!4h1#LjbKwCd2UlUtWdHuwEG3{F<|l|%_W0NMShVsNG~`yPT`R;909{!B%@8&)s61v~s!h6RW7TI41GlUtC(EY>KU1(@p51TGLK9AveG)GU0}LX%P!5LT^zo z!9M%T$ssG|&q2!cJ@@|(;q*ExexfPBD`)n*_RHT%%jq9bRs_diI2^T7&APEQa80Y8Pbhl8WcM4vE*5jxP z{tg$rwtV)VdG{0idqD2RgPQ;`5l(;@PFX~_E~%~9v=FHMq24~%BDYXIzco&YU+0UP z7B-)0<=8}fiWPnvWu4AbI{uEqF+VJ!Y0M=9P`jg8ZYsO50I~JXU_$)`1p}uC#l}09 z3TZF*<;(xq@`v*_<~4DB3XG;eck+u&F&C+>C4$@Li%drvCbrUq_2DJa$_0_qqwe7C zbp;hK@J#qB#ERQ97}~ry-0gmr*ixNW^vT|j-2_X|Ft-DzLi%tK#~Y}mpGADoTpEMd zC!~j8%yf%u3a>L8PKA9qX$gNeJLihWrz|<@Qp1T#HM;fvkwfptfE-LKjj9+!uKu&U zsymZ>Z%;+ zhtRJkhjIn=L>%)30lf?O0DElAa7I0yL-g$_XLNkXvCce@Fz?-6|F+sypOMEuwfX@g zrQFWJ3yp1Lx`U!c#DiGug=K|Z@@X811@4@m++j28nJ3oPizNi7V{mmAK8kiJ(g-#z z@^4=ALDL^FVz6mNimQCk0v&LhM)c~jMcHE8YzT!}K%}=}{6(mMKroQzPgiZzK*3H| zA!$7~MR)8I;Qq9U1xmZe>U6aAXz&HEckZlzV-$1EhXZ+=lXOfRN6JhT2YgCwDoSMQ z`7IWMzqO%ijNtwsI#b(vv166+`fVS_38ke*U<3^FjEJqw6xF(SV)43>VTmW0Rw`Y$X~k11 zU-y>e!s;fRwo)V4N`HjDt|I%Xo7GWO5CkGa!-n**v-B>mloydI%5>L-=}`RGfzo70 z@YT5_3END+=1ds@SFWib;s~&~fEQt}rotyMJmt(jx8b4HedfPCveVtPtt}oW14<~m zK}}gUTl@RA?P6Z99>pZL3*b%2M|K6E@zP4wK>&ZrOeLosP%C+Epc4 zcp+pC&K`zbQFO}Ngl7v$Ol9aNvgLv<_dZ4;@Ss7ryKMMqMY~+y6sb*mjr_Ye znI^warSguS^rl%{Jlu?fijELkvMuA4hC^kcptsn?F_kh&h|&kGBi2zB;N4mQ?q}pL z5GEpj9Z&y#QVkL8%?I!_neFso@s{D)4kme`TM8RjW7XZ8_bS{ptUNO1%U26ZTQqJ$ z|1H$@di^X>*)wEk=e0T2I?8wOUbY&5v|kSiVvha8%YLz^Uv-Of$ymc@$5d<&2rJ%P z_vom4u}?=vRMlKJlh+}nQw$axA7|)UIgj^aXg}Ke`r7%V*Z>4OJ)k?lmpbkfR`@Qi z3!~`GOU)X*91o)x4`&PJ&W0*;Et(p+d3S~^=zQ+)H_2^w?+6Mtos%|7-Q3Cl%70Eh z_G|oi=Z{|gv2RGb*u2>BqvQDDxzHV*9EB6*%6>|N&W5Y2Z;`us&ykz6b=@1JpqKr? zHO4sGt%>ciu?HafhbeX%+oM?)bN2bck<@W*rFZF@4MTqZXlr}LuhywgBwnV7`^CgM-?(f*Gxlk}4% zssd{=njO3%vaY;`2A33$vt4RHf6PDYfhYeXqO2kkjUZQC?rhPV@ zkKDeW+|W>b`et+`=rXs%RC0l{xS(obfh2MDA`@PI9Y2or@wnAb#*9H{mFS5g+KVg+ zlL(k@#(6C8R+i_IjaO|Cud!(;J;1DKg_*Rw!d;5>QzdehTl)I|@%u;pDn-d_51qnV z#Cp21j&zBloO}+pIrg?xu3oV{I@fXu{YyF2;9|g%u8LT+nEW*4V&WKEckvpXVw#mR z%>12WYk2a0R@<)nXg5YWXq2p#1%-2T*8#XTs48vFRKGAAXuR6pY}%IEG&8_Uc8t^$ z3@^qT%6&Sh)KK)aOnl)?2kq@YC~{bE6yn)6UIN=s_*0|d8Q?Da4R;BL&D@jU`Ho#O zqW^(svWY~B8NbQ`ZZ%4x+?5eGnh2fOu(a4LExUi)*1L3oM5|AOGdyQ8qf_;Cz^9=| z5cCh79!{PqiZ9!a@g1oU&oVhB*#y2{jlnj^A>F}pizStf2_CYu z8+AL3;$hhTdg8NFWvK;GKOpj(8U8T?s(wu zX|q}P(U#YTT$3n2XaqSbpPdR8SL@Uimd@ZjL(B+4LY$j{b z&6SzN9s89)**?2!TWJI?8Ewr|SM-#W8*-sPthMZSn#lW@R+PL#Q=zdD^2gjY&@{ zjo$krr{5H&A79%?XQ(c?RRT#?anBve*HV1rUnm^!881!){5GAI3N`ixbpgIh(Zqf# zGP$gsd|ON0df&y;(L5FjqOi0GOIh^`4tI6&hl#7mHCx7qOvMChaxa!Ee{d?Mdf$pG zX`+IO-eSq1xRQ4(t}OohJ6!!sJWFK4J}!7p|Fe6gOCD}$@3XdwR?o_R_Dxuj@OPW} z+vB+{Ou!B!CaqB`h#;uheo|s7rDT^7F6diM3>Q4(_~^;=GONxd16%QyT$4tluE>un zPpkruBfk}l{)CZ`L{`zfcP0z;4D3jRFjfkX(N5NATMJD{zl7)oElrSBGms|{K!X`a z_(?L!4J)0a-&9vZr9rv_{jo)66Zf^i#?Bb*f3M%ppPiRgUb`{1Gs#EVg^~FFRF6I9 zZvpyss;;!JK!0wEYUAA45BRfg-g{@Lo9ID<}L3q%zl1L6IA>> zXvv9i3L6cMxiOqo&fe>dR`)P2#^Yf*xh1}0n0&~Ppy@kT6Ol>UA$M!6Kz%dd`42Tw z(9=be#4E*CGBWsZO4Ie3F}7#?U{1(*l&C!xqWgi@R?$emiVs92@)26Q|FtAYw?{B@ z*-x%yl!S1N|2oI)Kjp)}(!n4kzO0-r@?>IzP7$z3zQXdKD<4mjCGz$KA)P(m4|n8}=ufc^ z-$-2k-YA3*&zZ^C{KC`IaQ#`PxV>6MnRL`A$mVUpaFWpXn5&Hn;PRtho*^j5MK~)8 zqa8N|*12RpbELE(LFO0)&fXjn+BIOS!IN0?{!bzdnMo9PzSm-T`WEzhG4ciGP6$4? zjB{b8%V?h)su0k}64;m~xE#gVg(lJa1~ST$C?O4Nptt<33n}mr1@6t=DsXQ|-aO?f z2J4Ui4(zqA@BcG^{AZviG3v#u41`yt=-5U9oUD^Q{cg9opv=(?WwRZxd*!1M@@e?&5*WsrL*n>t;dUg^maQ6{_P|;7%~S1g!2AehXDoeNG=a@Sfy` z9@JV;^tXN{m}_&k1o=8d71W)AY*zj=z(ajIz(aW@xdXc3*q~t{7Tj~v)-*Eal)v`r zJ&P&}lX(Ng`FR?--zeCe(ZRq5eYVsP9n1on1b)B9HU4J=%?hp_!DdCiy2Zp$Ys6U< zFoXv7gSozqy$t1U6DX7I4qnMS@k-8W`pO;nn`BE^PeTr5CGxr5G} zvAeiC?01Sb2V`XpKaXCc@4dvA#AbMZ6JJ#lLikmw=Tz8*P-JcjG=ZL@g!ey^+N}4h{ zrS>K0a6S2C_S*50(6aaIfP|o|9Hdrxj*qaKM^0ay4~yc9UQ#7KW#(H{Ok|2VKg{Q> zyu046G>z+dc1LW3BVsOJ=X^}dD&L@=T{9u8y^XZcq1>kU%fkp8udZi+Hv?H*K*%$F zss0%Td5f<)v_C4KFBxM&rjnp-qY)CPrU`b^ptqegZLpJ;3(ZL+8^uQ74q#5NBL6$0 zlVaq&=nQcFXGG`ae@1lF7Idy}M|7<5=kELHS>Wv6LH>v_rV|l))Qb{l5qT3CVK_Lr zM}pS^v>I{+-NzxUhdd1WIxIe4nGh8!09vx4EQV8gkd$u!x>w@eja*rNmp-8fHw9gy zjVt1NYjV3rz2T;Xi|1-Gih$Mkjl#c#PHPRawAFvgQ?v4}Y)X9Mc(ekU^ldLf&r^)1 z)y;DdJBf#}q1|a=@neL&JSLw~DnYh2st*Hd)Lr5cDUq_el~!8=$!*u{QgFQFLI3Rr zJoyfj9Jf*Eb<)AJ-fMz!<7MxzsBPq$tL@LQ}Z|0O@J6pe#Ytj@@Ql(`>0~4|Pz!KpQ0jPH)`u<;k~zA% zbFq&9%SHK3dQuxmh=hiyVG(@sEO799+qo&6Z&g8rFD0YlyZjV36uDYd*}ZQxBv`7I zdJGx8k@ZJV%o)3}b_J+w(*h;u7gQ)yzT;B==_~f?@TCHvqBL^p%XdFJvcHM{Ph;O1 z*TmOuYXubr0i{=IB2Bsw2rAN>^bR5=bfili6{JWf^b(|Z5C|asuP7ZtuL4R75Q>x# zNho*X`=0Zj-?{h0&4F7_w0Grv(|c&USsAG@+y^uIlMj;knyH*nGw(n5YYUT zx~{q5E9kfFMK*_5CA4=suOwzxe^4}U@V#|Qj(@nNYu%g!=F)7T;lIYop@!b6*gt!r z&nDOMv0ZcN(L#E0i)5QL|1!z!9>u2e>offVS6WqA)APh8ec8Q2hwp0G^O*FwRPOUJ zjtb#*D1XP|%F!MUh-^EdZiNZOE_{J)o-=x4MYrZZEda%sKgzPp-G5FDv8J2NigEKL z5`JOrUir{|V~;r2ck8jYtDo;M@9<2W5DiJk%rKR7;5#$7`SF_!{WvWm;T;aFaxQ|2;O;#U>4MZhPN9`O@1Om8P4un(6ZvU7!Y9;b#lu8lMX#d6W4Q-8{1}Y%tHq5! zcNi3q3rt{OFC$YpFMhg!Y{9?q9UO~qw(^ak`XZ+<1duW7n+oQzx$T8NM(+azke`vW zDx%AoD=~IjoeId#H-{9W1cocJYoo#V+d+=W)YcnhX!IwQq`MQUpd*@~z)3?Uaa*SZ z76s_q4>DHRY??hJ3(P3@cOG{0TKRc_55e(_yl?0G#Hoggj028mFaGt{Blf-VMfsCa zL1h^*0E;Z%rg`K&c}Ir=C`B8AX$vixd*eJyyr84T0Y-{Qc+@O@7I_tRt3yWvLVZXP z`47C}f$X&B${0UgMn$SA%#m$7-p78I98i`7^=N}qNXm^Outi%M%EVi}YhZ)$K`XBU zEoa>ZSB)_ZRu@6fg=h0lq*~s4FbZE@cq`jeNyu^=w^&^lplJsw{H{UiHdTyc*x=T7 z1>3u|!L2r1zzC+1y~cdVvLwn$j>7^y`L~ab*IwadbN&>8=wOXeO8V)u9g>=iuaZv% zOsTUWZB#kR33n!iUdRgb{t$S&e#8#g z793guz`tw2rJR{r;okt~nG6VU#15{t4a8fsab}C(^G0^hUTez>54@5X^)cM}s^`kR z+nCsI+Z7?wM~>VF1@k!roqw`E1@*{aako2=WFm1OIkn~hhD`|xA3rxtUX6rf6ynUz*4odaBB-y~2h;=OVr7v)|EHzl=$&pJ9|O zF0r#xQADDRTsc{&v_Maa78peT*BII_^y!)zV&1j(67?wunrAGcVC)QLUZEMV!krPN zhIdlPb?8yUXbS60_DeGI(UzCk+vvLQ^p}+#Lh0E2TM~ef#1%#Id!*mhct~y4GMQFD z^cn)fzoPtrvP~C|j7aZMo#WxI^}8CiZIz<<3@5x<5*OJA+FtD@f4`XfB3<_wvBB;v zo?*t>X_8j`;*wgN%)uR>k6ipuMd3i@gTWt9<@!L)7=5j}FoLdIv1qJzVA@ZQvp?Qe zvsWaq7&xVA<;l4+c%b5SH!(DElBD=vl7|T~n1X_O2)Gu4QKOwj)1k{lx4vOeX=mVP zeM5*`hW$XT?w^4zF6pP5?^yQhHAeCe$z`ZoM1@D7wbh#5gqX9GR_YEpzwm}ka9kk7 zaS@}Kbp?UjtSk7>jls3)2Z{9!p4I|j#1;=0I4CJ@lN3b)dU};%gS)~H792TjUQmCf zoTQ}LWTYB}?DfRvQFlJ0>K6_1wvSwvUkTv2o)EeOYKB5*+1Q9ERBM)#6`Zjv;EWYh zgX$yeK&LvA2+g;Rr2nNH>lRnME@xD%dYZwV{dSY=sgT-wPaeRBF>~>xuB&@!;d*&_ zRU%v|YfDi2`!2M^{wb(KV*+Wt(cHoRUs$6oknJn+BJJ+l+s$vFaLhRAp0GF>GUs== zeGBBwZQtNEI{NLKWaK*RhhToj*jXm1@HSL4^`FU8w_E= zEp9!mPY(W8FHmN40ORQYUAutuze^Vk{kwF*ocbCV=`cHo{fuGX`RHJi+fE;%DA5&r z?@;)9S4rs*m8|FfKSZ6HGKYgN7jX-46i|*T!^LzIqBh@+_i};L*t1s{V)->{#O_M) z#kO&Zj!#6ZN@|*s<(ewDf4HU5e`R8E^?zkzEstN?aCX$$2vQSbJ`Ip-p8;Qmbl1CWGlhCBRZFUUV_5H2H zQ5vteOlijcvf@RxOD! z$3}R?6A;J5i}x^S)G7+Mm6CaIyAZhuMlE5~0(~1MxmRqz`Z28b!x}fQ);-)4F-Ze` zs?xfy8Ny@D+h3qM{u6YwJ2xpesO5S2URwxQQ5c8lNPOPCIn%iIBDHcfsa&T|z>AeO zjH5VS_e1Ta1GmEY_hzHF1c85p7|uaz6%fQk5=m#SL_FAK$cqj{5My$D;dJA_K#U>? zVw^223rC%Kp0H}8J%9I*vl#hn^-#uE9<`fLV+YA|ln5s#aE3g2W$s=aPkOw%GB*}` ztwZc`q6#QFNGFO8;40#krJx}2Z^{7RSS0bae-9CrZd(7kac0tdg|Dsbj!uz}UqHsU z%0IMMRfq0nP(D%=aT|G>3WKmjoih zpjMn&p2OmrN{>=WJuL!#{Hzo&9}{TM=laQhuLpycMxuT;Wl(n*bYru_;?D+vPveGP znAe7UBikLqZDp0T2qR_$e%r8{YPHBCS}dfgA*V}~eM)@eGHt5dj9?{mhHcAIvpq;} zX4HX8_nII(gjJ==esDJLv-rt!SC1+8@g<*#S>rcLoL`G6J}|QTE8%k3TerG;+FG@9 zNX54dRX(V-%kjqEp>3extvUl(mQrJ%eBk&_1Taf~0n7@2UO16+c?9NduJ8OCz=#k5 zjJ&((1pcXG$YT)=%#SfLqaUG5oV1-x?plnUuR+~}%{m9S>dDReRCcg=-}{m}OuT}i z^-s`rw&0P@j}3oD-k~@jd+B8#VGSf%B6l|^yjcZ4|C@IyJC7t`7j13X(UF1SX#CNc zZCeXz5#5&K=*-7Q$5|w%uPKLaZg0hZd4g_3@XiQ)c?7AWKwXl;82n_b@rs;q+$m_p zOd`GE_}0#Wtmh%0@yn*g7)x&Q&3Cq=+t=+?g3+3gD>1NOKwAD6P%Zp6;t4ISy3RDXE@VL_^=;}Gvj#(`_2Br0fGkHlP=3EUUC-!9Vt)Ufi{WI_RO^tg2@ z-Je3kZTx$iO-%SHG(02ro{s8mQUTrX457oR$7fe*15ZUB5!*(u8g18s&V`=>A0sf zO!}1U8J3dp+_1Sb(11Vxi7Nm+^^z&qUZqw(56jbJ1 z1!k5%&(%66U)yqXJWd9u*E_R1=#}hU6Ob~If*F_33vdZ{+u>C#m1*&6%2=H#GROP1 z-}QHv2{HW5h>&L;rp%qaAsbHW&MNxlhMz0VFr@)ETEp9N2~@A;i*64yQAEXR6eHz} z7lBM4``O$&Fr5SOO1gGd>VIsyUgD44yY7)1bj@q%>*CGaF3#wrd;a@XF{%qfc)9ilTwKglZ z5xD7r=w&`^jZTAC5@lrgH~A%%7QV=E0NjxZm$qpqBoUhM`KEHCnWrJ$GDwcbt{J}J|Uj{$-RMM&YE*<9K{ zn*8CV?aubl>db-QuZj?^RPpx3Qc*LoSR6Zuwm@qLmN)HqVy+jau<8?z%oAg6%DbRknpH3fuV%p-6y1w>kQb{jv zX!^m78CNiDC7wDS94>Yc&nknV9rxa!MMX{`7v{&ORE2bNaoGY`LIoaKfedse zZuA$Pmoqb0IEDwVwL8du+h31hwxZ9jbYWChyc~%gnVIE&3nn5}MH`3X_5%cs!}xWR zc=kE^%>Y~TErbnity%+8t6T7w=8u(xH9yYgP33!_I9!)fMe4t9)r4LSRLq(MMXH!} zX{pIv(Y83*vw8K$f~PMoXVCBXrn9go7L)c5P?;QFqLe1vdMvFcckiwHt(ohf+AZ_X zX|E)5B6M4Rp|m>uPpRLq0mxF07=Ty>$U;Q-*}H;uyQqkDyFxiV&J|?0+tPDS^Ya|V zWfUwNOOFk(?WcppT;8C*FJW{qhJG-rv~aUcHdb8{8<$dP4+{7}{kU7#>{a4xXxxAhzy zCT6{FX<0(NKs5BM^-a#)*F)x+bUm1`Q5mY`W9+G-inI!jXM=blO$3DI%wAC;POXi< zEDva;sQ_7?f7KWK56c6>AvcK^DMUCFN`ymtA{XOsCXl^wJV8l(jAf}U+86*XmOtJ_ zjLFLkUGOU;9d0N2ZQvOoqI1186k)F-2V$Z=Uf#du+_u}<0T8Bk+dd;>ds|Tdr0$B2 zneQej`(Od5GrT)BA|lCP`e$#>W11?ddd*I@iOn<(4&B?~Mo_y?(KjFSSX*$kj^cLh_S#SnKpRS$TEC;O;M}FuR5;Bm6G{j7vB`df=CoyboGx zXqtg05h1A!a~s!YR`Ns~(NR6eXDdJ65#a%4bQ$pyZ9OM$0Ed49NX8U5{7M>IX&%1B z$qhz5ksF_KCaq^m;fX|^Pb7og-|kuf>ZXMQ6>ptL`X&6Qv-K=psako+1$%+2|<*5FIq~U_Xt;|84=*TY7(?kq4)$`Z}<( zz!BF7VxzWjwDL&b75c$7`-6+K0p#r??$Tc+At#69MiA2TLw%Le9jb&n$XM3!O6DOX z!R?2wCYaq3V|T%~%bI!Q?q;eniuDU(*A30%<#>rc#t(kBYp0nZEL)5ndeyK7TScGR z^morC?39MyOE+I=vTsqNaPmk5<|{}oHu8lAexVvJU~j4ja;7pJCzvSDCFC~o{CQdrH13pC{@c-7U`}( z@sE|+IB!!a8%bXWQ5U!eqAq*1Dw_h49ZvsY1@9b3L?KvW0di{TbzOcgap|h78{CMC8Sra61XC= z*u;4bj}0Zue%R0kZ8IB0+YBDG%^VG_n^&u^C4#mYPXnkgnZKRdqrtyOKB232wsg0ODcA7xnkR8^0cAl> zT2tv>F0k)XL1~nSL&vQQ?o>w;eJ}-OL0qv6+i&d*-|)5b@)$`TJMHkgZs4!aTd2Oh zfAk_U;G)@``%;i*1!tketIk&vH|Q}j4gJy4@9o~6)xDH5sX3X${BU67ZGY)cp9W!a5 zV`kw$j+s`_WwEr+gCRO*Dh$meiF!0&;srNPbj-{U9W#23mx~=(Te&{FyjLI0C0(GK zESGXN1t8NkYhNaxBcPr)09hU8Qo+-3J*6_W2ibb5phAzilt{ej5s8lgxhwp*YKvZM zpDXY#kNB2|Y8l6bZR+DPh%>8>d64WkK&veT{62`e8^^@)2)PFAf>2R9SbA+m8jS7C zQvY463@bOnW)tQ+W&Qzud-3URV8}6FFfpqBnU^CBGZIkjwp3|RtC+mue@OLgTyoEZ z(WcL|#7$|q+VJzs7-v7SlgtthQCMOPHhH#{Nb4*UX`NyTfpaZ@a@l~iP0Vxger1bj zKOsc_-HG#iymaKjYZzjG;jsKuG^2g5TnWdoW5a}I^ZWCwPh(`yo#+m3H2=7Aa_69|x0EKkZ zu}wy-4eEE%VhPH_;Jy^!d&B!Uava<#nBuG`_QbW|Rjz6hL_KL-E=;EC$)y>OUh5yt zc5n}VGcV}%`uA!*PPdHPK#$Ppu2-8?1xp(nJ{IqrP1O}Z(U9e-~`=N2)*D<oK3qLt!oA3cAir6`z@K?L|efj;>h_KhpHrw7u3B6@??!&mK%|9lXsLevR zkE!Q$Bk5zP5x#86u1*uD2#t+Z6`a->sv{J+?{(06q&!)R6_pjn2=9xHyk1)28HFiT z*6iRLPRrWXyXZ7hg?Am_ z6upB^#5v2Sx<)yvfjqR`guKnmoSpCOa_`%OSBeSK2VLpw9mApm^u3JnoWlL-R_VXs zb3B?U$e@Nl+1_MI=>x!NyNB1OU3~?`cTpkf^*ufDvSIp+94tJsH{%99&lxjZW|jBk z-frCHbG>rEs-(Vwq0J3+UCZi96Gr}ND?wEnXBB({CS+# zsO8rBO98n(z`OaC%L+8LCGoVrX!)V=?rdG{gA!lH(BpN%WRnmpQH20EL%)W zSQQSuiZQB#R8{je@}RW*S$^2XrVhFXOa|VbR7W`izzgq9_pm{(xR5crWz@5I4eG>6 zlLf|GPw0fDGkGvhzXs?w^W(A*I*z`Dv5PObC;fYudnoq^gsyc3|B2rTBX;KzvdMvy zvz{fSJ!qjPWG~St-Dg3cNo9ZJHu$DvF1zZ%+ogX#=>POSSgP67%(HTVG?WxDb zS(;KcbD&d#??JeiLl|~8J#lesae@CR;H~#dr|&^`js>^g$HjW>#=o;!xv@L9cyI)e zC`9hTjk3QcNSS7wTX*d_9qzi=+}X@i$BLfAO8rB3gN#C+Xx{2uEI&(_s_e22_(2^V z+yJwMdn4VhdvjIVP3#Rc(}b|8!m2o8xN7gY=>kUq@jUgC!Y`9+AX!s~@M-;2uiBu( zf|sz)P6$(6s|uz;e-f2AiGv%)=3%ZJ*&haXW3^J_f2>lP=mwaXq7v?5d`B>H5{P@X z3|U)TRH@ml>ar9mA6IMel6sZ*)diefIUZtfGz_ua4|pkptR5E^wfScBwv^|il884% z9WYO_?HO(F8}Z$x?YNFdv+ZQ=ku<-ZV3=iFx8607rGR!1@R*>evB!Sb%a1kf>h4W`^5sK)Qdi!!?DjoWS0KiRp%~znT#z ztV92dSpS-XBi8~;=9nA2JaHwRqC!F2TtTbjg)W^LYP*ow)P(Z(hm`!rd=DC5&pL_D z2IfFCG2U^zI1!5a2bE(Vsz3c!2E10Q({7I8T^1gAdN_rO)V8w3?Khx*C~_7|(RM82 za-Ub@FPa34-H_WnNcD+BX&g@6DX06T-Q@II;qy{~H4Z*Rj&Z1dlqJc}(Y;GsMcE;N zqN~!elU=SiF2oC8n_BrGTly`mSd=m9&#wOa2NjIJy~e+8A0bl{ zFlGpeQQA{iyurAkh44|RK|}euwo8L{a;jt!3dQv~qMWx;`c8yS z?MA}7azpQ8VgFwIBTZk_%URFa3~S2KSAG(EDWk$+bK?gU&{L&oT4-+|#DY<3PJfoM zlkYpHa9IpTSPf63?W&ybVQJ;mH*}HRJ}&=s{dG@KwKazoRU2TD=CPH=0bsE2R^~(T0NVw^(!y@7>VsKQilCc&ts(j13 zKtjDmA{WN&PQTu^$d$i&VSxk^IMpf0DF2%A<|m5Qm3CvcfR)=?Qazs`ZMQNGCDQJR z_}?_vNJX{+kqwVte^m`?wH{voDx|CZu|};9 z)9i!Qb8*!(dOloj9=E^7VL*1rvTOklR?a3v!f7PM?J?j3TsyrqdO9_iY}^brQuhd6 z50lP3Q%?RY|IJS8lnYW(8RPo-OYv2(+>VxhUjmI$so#sLwKun_id>cFK~+iiet~Nm zm$~o!aMyA8f%=>^j|1(Uz1KO+>eU%{u)KV0rkD(C-Z>{x zh_yp~PcPslz^+9SynZz?%0D(o;gf*f#QB&slAh?r-jZ5!@XLE#D1*^AD%fzz6HqgGG1k*rS2J{89RGq&%Ee?`(D~cTu z?ojcm7T)*OsJT?&YrQ3pGF_UcE0@=OCM2|XPFMbv4=XQQjqzuh0!kza#gaU$5#s@5 zMIFf+j>E>->*V4c^lB$>V~|5!3lR3l9dlWCN~kO|gVXNiAEgG=8F@UN@RK3W|C zupJTyMMGb54^9HJ@NGv?(q~=XTB#>T(yP(D1ZtcxPp?V$ZMQGinl`<6VtGAms(y=h zXNu_5P2V8Aju~%c9H>X8z3N(t-oQkcDvjW6CB$0l27brJ#-rqIB&z&aGxrZ{&-j0J zH|vhdP*&PzH7#E0*nuHmNjUfxHx=9mp5vsZ@Z!awH9!X4{4C6%-({oWK*7Q;;US~H zj-IU^>iAM!*aGbLABTFqPJTC)mJ6Xr8Cy3oDkHf&G*y)&T057sF`^pn6LlFa^@Uh- z-+H8NYMPq^L2VO~nJr}Ny<)Y!G-6(LE78!0-05dHd2=XtQ|E^mgfUG|ZTmZb{6WKK zXoV-Ebq#f9)}KJF=v!{#Qul*=3dsF$Fnhf_2BOt1$TmMxJVl@z9d*qdi;+H?W-h_& zStEYvS1xOD`TU+L?F+x0{a(A;j}6xLKSTdSUR$fB(XEthSK*a}c7)e>&ff6MisUVe zD+rjH##B~z9W8m=j9&|=L*y?53BHYl*=GWeS21^lxGHA4>=nbsKdAAxxvHcT?R%>W z4%+?XHE2AM39Z6P3-+4K9zyqat@Fp++wzW&=f!u{18RhQ{aBlnW~O@%ecAC{ob$rz zHfPK0Ka}X)VLeB(56|%gfR*%|bSm)8pzKya+&x%eUIF54t(_ctixUpKX0_!y(9m1j zkh)RpjPM)9DOl2+>P=co{)SOUtuZmUQ(!(rw(O8Us%M6BJ7a&(UugDZHPp3qmA9PK zVR)<>WYfO7pn!hReJ~OirB?3O_PRLMp8ms85)v`mT6A)M$|AL$Jm8|~>eZnsbQkQf zF=5a4`aDM|n8%0B@Ad<&QS01tF9bIJxcWD~ebw=_`~WShJ&HHQJI-63jv8RLYLK#~ zm0{RQX^+SeABP2m5;<!|C4&(tFyGBvzgD63V?%ke2g==R|5~N=Hmiy_R1tc78h^KDvW2 z>$d)?x}SNS@L>)}i?@-B-swiT!ugMiOcrxjBn7q-a>{bz&z5IWpt7~kdl%(?qB(eY zS>>E&5vxa$Rrh9cqxz2euW0Zl*^4cI+|D?qg}u<7gCA;brsuS|~ksy{Fc#L24;mjk!K;Py#%*8)df2DE?MMYGD% z#M1Mi4YDeevr9<{yc6_SEI=(D#@}CNPgu-f=hdFny>}wj*8OZj+0Q5+DJ;@|(3sUF zFN7KHpVe?uO)P*h6QmRA*M6{BlN(DsJn!09- zl-!yXfSEA4UR=~sYz3NPv1LVv9LGQgRB)ZE8BH!&FT_wKB^Re?drj%lI)t_S>@vm& ztW=#ZKCU$kKX8_r3x%FZM-1upqEb$5Mn1a`l7x;<1-2@GG0X&QH7#S@&)*{$2jAfD zYkNqv@4sSwL0IXf(lWh1F2GW+*s^FCOHMTfY?~?Flyv#DPTpqhWa?@iNeEuua5Kfn zhU6a62X)``Te}oWUreSaY+u`2*aU!= zfg`&qQGyv5T-A=0=3?dKR;A6Ya(W#B&PKT=ETx^t4_$2e179Xo2E!LfbGt|9 zj8Lj9YF`a%38)sC%JI?B*kHrF6OI-{koAIoo9_UJ=+YTwfb{VRYyf{hr#|S(N!A%< zXv$f@fR}H{b~b#-)Z4sEzi1=9V+qK2(rx-~#z{Gkz3<%VAyI@hes;FF4fc)iW%W^0 zix+nZHJXJsgYG(v$<24j`tgc{ZorX3VWDgb z_eI~|EwSBUuLiXV9e1q8*0HQgt=E1SfYr#vW3gXO?ngT|G}O+VEZ56SRh;b=zwkLf=-FX) z6Swp0NA0Ib(j+I?;3JZAo&a^R!u_;!D)6Qva}43Or)CRwMkaj)RX!5&T^et@ce8dc zLpD^s*6*O~oZhw{BZ-C^H$bFiq4|o_{RmcFZu55 zQ!G?*ly!11i(_+akx_9{lGb=vq6=NGI9r0Cm3y8;0>}hpz3Z=40YuJS_?J(0CNZ|3 zcE6wDo06RBta2qhQ+3ovWDN+#{&IeX`%slI}yh*@~55i7}83oe~wWMj$c$5 zoZay%4+$+8YGz1?mm-!qrJy+^0qs!0=FtLhXJBiN1Q6 zcEGE{GyTn+BKJG=#rs@I2A*>d!VxSP@`vgyvNY}HDi$i?_ywzeN#8i>WvbE2-y6}t zM!SAdE4#@Iov;~3@|dQqCEv+OldN*`AOs5$o~w;X*A*RQ0(7)Va9a~OpXFL4)-sdy z0gJgass<}#vHiLuPN9J&i?QcE6Lmf%g#!*RlVG{J^XqOd#k--|crPC#%Yg5{WUL*2 zAAI}VH${qK@bHymPCKw_3`YqaweE?O$sCZF zv0@8-0yg`5yalzg1A}XdV^g)aw`IqT=MnhnI^okuBJ4nR2P13|VuLh4XZsw={fz~( zeKbZUq5Q?O*N$j1zR>vc-wP2twFdnotPY+q$f0A2{N?o)IRui@7Fz=_7jFew5F7q zi(I9EkYLEeH8#XlS(YTh%D9*1?oK%};j;`C0y zNN#22^%?shbLX*+ZvQ7@<(oO0QkHSpN*n>@jXkxt#4Rm*jGSc)4cb>OxqYhv6zOpa zjYpc9HSGM7GZ_hHQ&ZgUgw*;bAzg#r=;_QNGst&=>~0W75Q-`a;3w`s`c#j`q_%pk zs`LnA9v!K6+{ag?-A!bk(qWFj+*U9Xz3&6iYVFv^Byyyh!8*5~o-)+jy`0hY8(^^t z8lGdB>-ssQa1oKh)wtfFt~FpG1S_3g_n5)beL->&!i#prjH&%SM1^ms_lQ!I{YPLw)Z{X;Re% zmHI>$BFb7d@(18)S;IL%*D!~a1yw=maczMtqwR6wq9&Br&*xsDb}9RHOx$iM#S;<_ z=AQQYM5H?wv@@12NGk~5knM0u5X~#*Y<`Zem?&o6*XwbwwY%h+##h;6EY0Q|vEK!W}LDzLoAOW6Dji4${KDP>)F|z3LL~U61 z$XG@eCRWxs=gN5$hw0&)#!upB*QCjK6+JZ1oH|tZblC_dJw502c4e#+_XW1NM0HrV zKWPi_(zDd-KUT$0$i&%AcBkE4D0h;7Nf3J-zH4)oARc3XG@(Upi+kju*z`!p!@76oj$yUmW^;6wWji9J{v_Z7t``1X-SY>9}z}r$B~Mw}$(kq(Zd* zBg0W)7GmCga0@Zx(aNQA`bEY$igN~+T+OoW#)O2`g{*9w<%aHxZLz^HLZvj|5lQLj z;BT~lL+NB8S2LlNuciK0Ru>$(InW3u8E%h|SfwNzbE;ZJvZf!?q@lr|RZidI-n=<- zJ8TmVT~i+lEJhp`ZDw685bH|gsQg2B)}FODq#f%$=(YH`!{Vy(j8I>1e|z@j)>Nt~ z&ixEtznFV<<}IGt*bU$cLG^=9>jAW9}fm`DvckSZX7a-~8 z-br7tc}B#(G5uxwtKy=_TkIDr`IA>EQm6c);dT%TsC?4PN76_7rt3NMu*|v?wA2c` zl)KJn{lU=9-~s3&?V-{&&#)g1{ZWMEr%&CX_>-TzQ#?!g_4e7__;+czxSR#cb$#}?Y{Kh*FQoB*q>yjY z=l`f!?VCRn%K{&-W*@W{@a#EJO5zeBP<1z^U9PNlT{mYnw`v$6N#xyODer3d!H4`_ zzi05M3Ww37kVt)YTyDEp-0b2fvnZT6G6&v`L18r-2* zy^7u&hp$`BJ4UIy*GdQ*!q%NV$5FGxs1NAI0*keBrl7_dgrUI%u*@VLY;U}guGh&B zp8jj!v!rU^uAs5rVrfg&L+T3A)(}jjn%qEpd;wr?6`JDHL~#~*Z02;wU4U=q!Jc~& zss52DvN5B2YOVgRk#bRZML7rfBR6*S?X>Ib>&&$jy#WnfOnmKBFI_A=xk5z+V6z`Q zR@6;xi|Vz4-fNqm0FiUOVo56Or{xs~|vN^$mYGdwCPdRkCmt3lFw2_OF zO$yj-GwUm6n~0U-9I-gsZL8uH+upOru)8kT$P_4?;3207Q>Q;~db8R>yo0X-XfC_6 ztAXDdroV*N{c(V~)5}+Ok6y@kWMz4Dd1wj7+l{w#pK{XT0?Nn6YUe`=B7~B6pKbRq zaO!*1%N@$9*`TZFPxMx{jlV{@u<+lyTm4Um>`B}A8hV3L!pGHHYmnX(=yKrfXrS~z z(-K#6qGL!Oo?d2oCxd#MSgu-moZb-4YIJb0nLn-m%B;m65k)rbB=kISFvf!{V`8y~ zVLOf>rLvaOK?3}W?cPXdtJ`G|smlgyz!~^HIv23x%p}`ievI-r4xn#bKIc}Vjlddr z&RL!;hF(Q?wM&0BsXqE`u?(-c5moPP4^@li7ahzA4ju#ZK~S=2eyZ#eU)0)(&lxyH zgPnZh)}(EwPWaJtCq4r)wt=6^yCUB^N5)>u}fOf-ogv83g%B+chYYCgSoPir` znpt;W=jMx;ZL2BT8FQ1dwSnmuXl`o-R!pO=ndFo4s(~rv0}hz=P*+w=R6o4VJ*aN4 z)Kl+|k~#IWFR>#Cz6!=2buVX%^6;QPwN})ksTUFIWy_cHd`0 z2c_~H0OAGNIQ&oE#O|TPMyQL=^i_$f!jjcB@yq_!K_%E zpY>L*N>eLv%?vjKiqNk}8yv+nVN8TuUNe}pB2ULc-Qs-ViHkn8lLbzBXb03nK++)) zc>c_os|#lo0njcUU>`Ofy8CREXw7oX?p(Etib)*9|D3`6wpA0-LOd~T$lp%)vh_gf z_*l05z`d(}rIg(dWNYUX%ujWCjxBH#Zq8t#XG8UO_X`jL2Zlw(#F{emZxa;kRA4M| zPu<3y@3x~G6XRNHGz{{uZQYU=*A zAdR>6F*^zO?JsQcB%E(l@4Ok`KXhU7_vN`((b`nyvZo=utzV~L;zN=%=RQ@T_2{)7 zmOb9Tog~5_-d`UJagDrN5ew`J=A7AhPamX&Yst}_+h(-kT+i>gTy2!Q`dA#jX|pL( z-miub+s+xX6qm#RlO6fcV#6f!&r+4=26!{=)Xz&%T2mn+qAXceOhl zkC8d|$PW5(-A+kRD$R#7-j8u3tv92hd?!`XFI{-9cJY$Hh5zn(_<6p_-~JUVfKm{< zRrcy&ku-4O0=FO;{B40C8}YReuI>CSaI82lUAS-y-2Ufr`z;(*!1%X;?E=P2^6%g` z3K^w{pZTh?KhppnXn&D7#{cuN6Mo@>uC14sgDcGQf1iy-A>*U9hzl1qY`v^(th}uL z*ZVrP-z!Uf!N$NtE|C4tMlYm9!fOiY|DMTUA>$>T5Ae-GjsG!G9v9ucWF7}MEV@U6 Ji6?&Y{{Tz9P=Npd diff --git a/powerbi/README.md b/powerbi/README.md index b5decd59a84..5102fb77a52 100644 --- a/powerbi/README.md +++ b/powerbi/README.md @@ -3,12 +3,14 @@ The provided PowerQuery/M function allows you to read a Delta Lake table directl # Features - Read Delta Lake table into PowerBI without having a cluster (Spark, Databricks, Azure Synapse) up and running +- Online/Scheduled Refresh in the PowerBI service - Support all storage systems that are supported by PowerBI - Azure Data Lake Store (tested) - Azure Blob Storage (tested) - Local Folder or Network Share (tested) - AWS S3 (not yet tested) -- Support for Partition Elimination to leverage the partitioning scheme of the Delta Lake table + - Local Hadoop / HDFS (partially tested, check `UseFileBuffer` option) +- Support for Partition Elimination to leverage the partitioning schema of the Delta Lake table - Support for Delta Lake time travel - e.g. `VERSION AS OF` # Usage @@ -35,7 +37,7 @@ The function supports two parameters of which the second is optional: ## Parameter DeltaTableFolderContent -A table that contains a file/folder listing of your Delta Lake table. Power BI supports a wide set of storage services which you can use for this. There are however some mandatory things this file/folder listing has to cotain: +A table that contains a file/folder listing of your Delta Lake table. PowerBI supports a wide set of storage services which you can use for this. There are however some mandatory things this file/folder listing has to cotain: - a sub-folder `_delta_log` (which holds the Delta Log files and also ensures that the parent folder is the root of the Delta Lake table) - mandatory columns `Name`, `Folder Path`, `Content`, `Extension` - a column called `file_name` @@ -44,6 +46,7 @@ These are all returned by default for common Storage connectors like Azure Data ## Parameter DeltaTableOptions An optional record that be specified to control the following options: - `Version` - a numeric value that defines historic specific version of the Delta Lake table you want to read. This is similar to specifying `VERSION AS OF` when querying the Delta Lake table via SQL. Default is the most recent/current version. +- `UseFileBuffer` - some data sources do not support streaming of binary files and you may receive an error message like **"Parquet.Document cannot be used with streamed binary values."**. To mitigate this issue, you can set `UseFileBuffer=true`. Details about this issue and implications are desribed [here](https://blog.crossjoin.co.uk/2021/03/07/parquet-files-in-power-bi-power-query-and-the-streamed-binary-values-error/) - `PartitionFilterFunction` - a fuction that is used to filter out partitions before actually reading the files. The function has to take 1 parameter of type `record` and must return a `logical` type (true/false). The record that is passed in can then be used to specify the partition filter. For each file in the delta table the metadata is checked against this function. If it is not matched, it is discarded from the final list of files that make up the Delta Lake table. Assuming your Delta Lake table is partitioned by Year and Month and you want to filter for `Year=2021` and `Month="Jan"` your function may look like this: ``` diff --git a/powerbi/fn_ReadDeltaTable.pq b/powerbi/fn_ReadDeltaTable.pq index 373febdeb0c..0b4f6d5384f 100644 --- a/powerbi/fn_ReadDeltaTable.pq +++ b/powerbi/fn_ReadDeltaTable.pq @@ -7,6 +7,7 @@ let DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", (x) => true), + UseFileBuffer = if DeltaTableOptions = null then false else Record.FieldOrDefault(DeltaTableOptions, "UseFileBuffer", false), DeltaTableFolderContent_wFullPath = let @@ -23,11 +24,49 @@ let in Delimiter, +PQ_DataTypes = + let + Source = [ + Any.Type = Any.Type, + None.Type = None.Type, + Day.Type = Day.Type, + Duration.Type = Duration.Type, + Record.Type = Record.Type, + Precision.Type = Precision.Type, + Number.Type = Number.Type, + Binary.Type = Binary.Type, + Byte.Type = Byte.Type, + Character.Type = Character.Type, + Text.Type = Text.Type, + Function.Type = Function.Type, + Null.Type = Null.Type, + List.Type = List.Type, + Type.Type = Type.Type, + Logical.Type = Logical.Type, + Int8.Type = Int8.Type, + Int16.Type = Int16.Type, + Int32.Type = Int32.Type, + Int64.Type = Int64.Type, + Single.Type = Single.Type, + Double.Type = Double.Type, + Decimal.Type = Decimal.Type, + Currency.Type = Currency.Type, + Percentage.Type = Percentage.Type, + Guid.Type = Guid.Type, + Date.Type = Date.Type, + DateTime.Type = DateTime.Type, + DateTimeZone.Type = DateTimeZone.Type, + Time.Type = Time.Type, + Table.Type = Table.Type + ] + in + Source, + #"TableSchema" = let ExpressionText = "type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", BufferedExpression = List.Buffer({ExpressionText}){0}, - TableSchema = Expression.Evaluate(BufferedExpression, #shared) + TableSchema = Expression.Evaluate(BufferedExpression, PQ_DataTypes) in TableSchema, @@ -37,9 +76,10 @@ let #"Filtered Rows" = Table.SelectRows(Source, each Text.Contains([Full_Path], Delimiter & "_delta_log" & Delimiter)), #"Added Version" = Table.AddColumn(#"Filtered Rows", "Version", each try Int64.From(Text.BeforeDelimiter([File_Name], ".")) otherwise -1, Int64.Type), #"Filtered RequestedVersion" = if DeltaTableVersion = null then #"Added Version" else Table.SelectRows(#"Added Version", each [Version] <= DeltaTableVersion), - buffered = Table.Buffer(#"Filtered RequestedVersion") + BufferedTable = Table.Buffer(#"Filtered RequestedVersion"), + BufferedContent = Table.TransformColumns(BufferedTable,{{"Content", Binary.Buffer}}) in - buffered, + BufferedContent, #"DeltaTablePath" = let @@ -152,7 +192,7 @@ let #"Buffered Fields", #"Data" = - let + let Source = #"Logs ALL", #"Added Counter" = Table.AddColumn(Source, "Counter", each if [remove] <> null then -1 else if [add] <> null then 1 else null, Int8.Type), #"Added file_name" = Table.AddColumn(#"Added Counter", "file_name", each if [add] <> null then Record.Field([add], "path") else if [remove] <> null then Record.Field([remove], "path") else null, Text.Type), @@ -162,7 +202,7 @@ let #"Grouped Rows1" = Table.Group(#"Added partitionValuesJSON", {"file_name"}, {{"partitionValuesJSON", each List.Max([partitionValuesJSON]), type nullable text}, {"isRelevant", each List.Sum([Counter]), type nullable text}}), #"Relevant Files" = Table.SelectRows(#"Grouped Rows1", each ([isRelevant] > 0)), #"Added partitionValuesTable2" = Table.AddColumn(#"Relevant Files", "partitionValuesTable", each try Table.FromRecords(Json.Document([partitionValuesJSON])) otherwise null), - #"Added partitionValuesRecord" = Table.AddColumn(#"Added partitionValuesTable2", "partitionValuesRecord", each Record.TransformFields(Record.FromTable([partitionValuesTable]), Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[ColumnTransformation]), Expression.Evaluate("type [" & Text.Combine(Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[TableDataType], ", ") & "]", #shared)), + #"Added partitionValuesRecord" = Table.AddColumn(#"Added partitionValuesTable2", "partitionValuesRecord", each Record.TransformFields(Record.FromTable([partitionValuesTable]), Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[ColumnTransformation]), Expression.Evaluate("type [" & Text.Combine(Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[TableDataType], ", ") & "]", PQ_DataTypes)), #"Filtered Rows1" = Table.SelectRows(#"Added partitionValuesRecord", each PartitionFilterFunction([partitionValuesRecord])), #"Expanded partitionValuesRecord" = Table.ExpandRecordColumn(#"Filtered Rows1", "partitionValuesRecord", Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[name]), #"Added Full_Path" = Table.AddColumn(#"Expanded partitionValuesRecord", "Full_Path", each Text.Replace(DeltaTablePath & Text.Replace([file_name], "=", "%3D"), "/", Delimiter), Text.Type), @@ -171,7 +211,8 @@ let #"Merged Queries" = Table.NestedJoin(#"Buffered RelevantFiles", {"Full_Path"}, DeltaTableFolderContent_wFullPath, {"Full_Path"}, "DeltaTable Folder", JoinKind.Inner), #"Removed Columns" = Table.RemoveColumns(#"Merged Queries",{"Full_Path"}), #"Expanded DeltaTable Folder" = Table.ExpandTableColumn(#"Removed Columns", "DeltaTable Folder", {"Content"}, {"Content"}), - #"Added Custom1" = Table.AddColumn(#"Expanded DeltaTable Folder", "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", #shared)), + BufferFile = if UseFileBuffer then Table.TransformColumns(#"Expanded DeltaTable Folder",{{"Content", Binary.Buffer}}) else #"Expanded DeltaTable Folder", + #"Added Custom1" = Table.AddColumn(BufferFile, "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", PQ_DataTypes)), #"Removed Columns1" = Table.RemoveColumns(#"Added Custom1",{"Content"}), #"Expanded Data" = Table.ExpandTableColumn(#"Removed Columns1", "Data", Table.SelectRows(metadata_columns, each not [isPartitionedBy])[name]), #"Reordered Columns" = Table.ReorderColumns(#"Expanded Data", metadata_columns[name]) From 137f7c378dac2f506fade74360b95255ea4159dc Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 13 Jul 2021 18:45:20 -0700 Subject: [PATCH 061/291] Fix the sbt launch download url (#98) --- build/sbt-launch-lib.bash | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 44d37fbf3d0..c0e8f06c7b0 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -38,7 +38,7 @@ dlog () { acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - URL1=https://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + URL1=https://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR From 1f876ca138e9a5378adb76b71f618be3036c900f Mon Sep 17 00:00:00 2001 From: Gerhard Brueckl Date: Tue, 27 Jul 2021 05:23:32 +0200 Subject: [PATCH 062/291] Add inline-documentation to PQ function (#103) * add PowerBI connector * add parameter to buffer the binary file before reading it to mitigate streaming errors * update global readme to include reference to the Power BI connector * fix issue with online refresh * add inline-documentation update readme.md add *.pbix to .gitignore Co-authored-by: gbrueckl --- .gitignore | 1 + powerbi/PowerBI_Delta.pbit | Bin 42623 -> 46386 bytes powerbi/README.md | 10 +- powerbi/fn_ReadDeltaTable.pq | 450 ++++++++++++++++++----------------- 4 files changed, 243 insertions(+), 218 deletions(-) diff --git a/.gitignore b/.gitignore index 2805ac1afec..c7f066d0b09 100644 --- a/.gitignore +++ b/.gitignore @@ -20,6 +20,7 @@ .pydevproject .scala_dependencies .settings +*.pbix /lib/ R-unit-tests.log R/unit-tests.out diff --git a/powerbi/PowerBI_Delta.pbit b/powerbi/PowerBI_Delta.pbit index 63d9fc88c86031debcf94a53db26eac1db75f4c8..c9714467ecfc6302b03485189ed10fb6c4e47a32 100644 GIT binary patch delta 42229 zcmZ6ybwHHS(>83=ARt}R4bsv`H_{DCH`2M>DoEF|bV_%3qf$#sHz>{0v82Fz^E|)z zeV*_8=j{G<&V6R)%*-{{%>B@l8pIQ<_exKny?XTM5$dCZawHT>_9^&Nt@G|N4*09# zB)lJZ{Qc3RHX&?S!DDV<-)%v*;UHqP5OIf7vd5{gl>JK}wrf(kgq(E%PI1_6&I~IN zR%n)pRd$z!ipFJh`_0^~s8trGJyUr6dbw3WB)>-ai?^-F(zc0snMvvW!VS=NzhPXI zrPk)hc@lusK}=*FUufKlBF*Y8H?gKnpE|D(w|^7R$tplxNJ0RC_cv@)g<;oqI$1i1 z!?0_#$p#5X^dwi7H!$Cr>d_bGTH}?$j=fJZ!%h^DBa;i+Apq0~(7ra1gB*h>Kq%qLo@06o9v>pXouyzS)9KK9-1XpVPp7 z#<1Bpj(G;R%x4|}&grV$H!rAfv6dB8-8lENKkebz^d z0gd)zgVB}_O7WlRy90WRkgiH$(ccrkqm}nd{AcQ-Fa;8-BmaPiQ_lBm`t)!orOK>y z!2HAfF)@<7#9BM%hT1*ZR~K=r1wo%IM9U0KL9EEyb#B?E718$%VfWsijXU`6rv(bb z60a!LPLACcSuF1xuiK?78+Y1kZe|MQfXe6=-%_^UOEi2Hc`VI^b-}0SfKQ^7U!&Z!v$<;!Ntx zH7)->n^fC}p0O$n`p1~KXCG)eRNhAk`@dfi=@MF)^u2RS+ir0FVJqwzR=b{*?+;OP zxtJ)Y5c+Y*%*MMj5ra&&xUpXBU6&=EtuWx00U82Id)Dv1~kJ~w0{RolZfK_ryu+hpd zI&m94tdplEY<0^1aTE>h+n+1o`Lz`ZlBhtLZokl3pb!Swfj4HCXh>-#S} z;B5Ma)8i~@p+jpyndEhWZf>Vo?!PZ%(1mFA2%XV-oE zHngjn4-iDHIK}g<-12rlh%sf)v^{lSR`=T4;<2l?_LW^>XJziD>G=rYUVSlR+DR~6Op58UpJ=35=YAq|xT~^3vqyIEfgdHWUQ0#a} zUPvxrB*{HoJdBRgu-lE)!+m5OX8Kf9&hESMNwHl$^4+m!Vb-eWU*_MV^mT4f|B{3< zM?{#Gs8TEPW705Jp>>W!!=p(ao-7Mx%&Gdk{4$rwIwxCw>Hs2U;CXSOUY0DvwsP6{ z==510Wk_)!!S2_|wEzlal5cp&58Kuk^ZA)9goad;499eK3yhie`RC;bG$V`563bmP zorPDYlOcbSA8$5Y9UX--T4PP4O^#w_`PW-wbB-GAllVyuS$O&Bq}@#N&?3{@E3H*c zM@Qz2^C@-NFg0Rs0FOyi1R6zk=R$UdUBkTKuJVY&WW%T+p75Pbui{R<@oIXBrG1*4 zg;naI0s}UcSmxs3BFnK&WL;_Y4d0edZbVsU)f5mc&79vp_+$ zgs-fwz9mACZG*XE?#&3^zHFIIhSd8}bBxIgybGyb+Y)Em-8$XF$OY=%^x%Ydn=YNv z*MnjeL)3dT=I?w;%W=YC0eJQM6QdYsV>m@9JB7w(Sr1q6r=v36Q|PFN!u6`8E2a{T z4}vc1x&Sd+|M5(UIrA?hpP<*>vl*M$d~s@i$b;adkZ1AfOWIFdd&XZsZZ?&>rGKrv ziQjN|%k12WeCK?kmeT;aYYsMGEmNz7*Q+~Wg|OlIF4D$aZDi7pzF{e}SBnI z69KLl#7?kt-A+cjNbJ*msaJTHxQ(jcZWp>VI$a~OQoJjMaQ8Alm+09+iXP*sgLkIE zTLMlkhOIGc)H;FNqdO3oTJ zxX_GR8WrcCy1gPVu|L(i^v|A-N*CTo0bm3^TYs?ipaK2hLPbfBvU;&1)oai~)Wp%s z?~RLzvoD%tIRC!5nPfJ-nJc7+)!~piK4~DK#z4+??+<;P2*Z&-kl5^kC*p24VR3Ca ze!H5W%VN0Zf6(H{&;;H0i7IY+iGvw+0L@0{Q>N1b-|s1wTua9WF&ou81pqWf@s(2c zaemW}?$H=UiNm5wRYOnKBp@fn8Z{yYVYv1@+8!cY%PHB6tN|k;?UF?nP}}PneyBk-$6qI5|+NCzt3@s0N18;+30?I`6%X0H#J|(L0JmGu=24mSEX^sLaCy9qVA{S4CX2XG&4ZpJGOkAAsK(*UvlzE| z__~vaoEgf)%IG&ivm6kmU<6BRl1eGN# z^oZ%GFLsMM>DCox4?Pu%qU)zorWC}(=FfNJD%D$g6L?6dUs6N_ouaG6x=%TDI6uhp(hb{TcLDsIgk9+xhY=`%}U^ zDNsE?^_R&uV|7zb01##g8=bsU?(_`HuaKES*?_ZvH`AxrH;4BaJ=RuAqN2t3)3jj4 zuk3$?iRV0%7&h$7gL`RiW@mIC%4iwh@CT+uih%ahgFA~d`S@*@o3 zO2f*2ytY4j(N2o=rL2+hL#i~qHf$0-zko42lrCE*qEZ~w)epYf*vk{!R&J38y#o>F zn+h@F8$_=K;Z;&0kN*B9O|dFv%(9bHS8w z>AE{7X?fOKx5%=bOpiqg*(7PD)|OtsN6AZ;b|hlje7aZEID7rwZbw*dF5gM;G*sr~ zS>;O@M{PaL%ZReroLeR(`S@*zcL38j~7e0JkfAqmOm!{#2Z@gzsH|_xZWduZG7q z%FeK67pZj}%92rYkHoIa^35X|g5&S8^WkTo@){q!98AHv@S}a~>a~PRMO{O{MEX8C zfn((_{a$oYQx644P)^1B2E~xlOfRD5a{sx#Si?P;ORFd7FrA!Q&TR*0R%r>Wp>+3@ zcf{rZazt=bI&75v5i^^DQ~fpniGg4JuutHBzn=+*g;#mCr-eKVj%A0+n~E=fG@M!` z)$-kZ=T%@=UwE#uVBy*M?w+&#%I|ELs=c$Ay{|mX!4u~4w|(4f9<_dobBimQ)$H3A zZrb*162?SCWAqg|?iSxduYb??xB7xiM}Yf?ldS^2b`4)n=Xxtmem`TwENjP4^_0ZJ zwDZpDqWphAFmOb&=}CCuI%VF_<{Ek*H#uzuQBv6zRFolUFWNk7$RO>Z@WWdI_v&+N z#eE}Qq;p4Tp2(21RYGWX=7~3WKSs{wAgZy)izqUtytc9rn|+kU3?z6xOpla8z zTz|hUu2qNG>00L2IR^|hEvQXua+)9c)%E7&f&e9wHNWLxi3ax6ePX43&ga4tLkxB5 zkn2QZJH_iK+UhQvX!~-^VR@{eWT;ux?bEfV=v))dMFAn-{)x3%dHL+?=Jy$Hs}BWN z+C}I%YMYz;gxF|nMt;DrgwJ^ZJ&wO!+hD&PUF`2~AFEM~hE&ABP|v(JIjB@hm($m< z&o^T12CeRQGir>PO4lWs-7Due>^0MD%P%G-%XO=~&LU5Unh<_<`xhmXM8$Icg##%f zURl{jjovdECLQL}0h;YUE1NFleiaN~pjJ+a-eO0EyZV}Bbi5Y|sO;Sb$UK%0Cy8A% z)W!87Bi7F=fW0L+P{cC{0r1RaHXhyY5)4geuKy;<-b(x7G);L(@B0hwpSM(-+L+pE z%lFZyFaaHv$OxWT_Y{xF-F88eY`L^GmEeOh21H|wtFofc(*CDY=GaB7^Ta*WoH0D< z%JRs5y|y%EY8)obg?|@7-ALi(zOJ5B16f*`!2-(yinGIk(pfQ=Z=D4%i?f$*F#%OQ zx18ylFOJmp+)K5#Ek{or>b#@;Za+iU3SQ$|moIC%d!$upD>+Am7*0&8=pu_#S2<&{ z;c4uq7CW)xG{JgGjIzS%W@o4G9+|I+>ZR~`6B1*f1V5o!@5P=18YS@}$O&zFv9pph z*JroK4*c%FcOlm(Mm26JH-3u&pS5*4LO!%uHu*)XbWiIoof_TV;E9C^&VQ-<%8K*} z*ZH)e(#6IzF#3_=>aA>9q|;u`LZATgHTmJNjqdvP;k(E8%RA}={tYIBDv1m^)(~Qbl=nOM1 z+eZ^G_3HlVY?v7YLCDMRtQtNP{-+FYgk|S~*gMtS)YJ+Fd?E~Mt(AZ4*Bz|3)N!&u zLRxGF&8QvN5)scU8aPMWv|Z*+Vk+?6nP(tbneD_^(~*Q;9lIfIgBz1H-;6xrF+59U zKCul>Z2#)m(3SQn4pV`_R3L6>r*oKg>{WTT5DUy^v78JqihZMXq*bDgftRPpsD{}JTHeK)U8tM8 za%m+w#Fdq1R3~)69|URA=_IOBis<%>wUXz_BLT)E#e{7cj4j6_TRo&wtMmOE=M3OH z>m)?po6l>|wz<@OZIiqU?B?dTQeN_Lj&X5`aIN}omTs;orI|aC+WZtthRB?SuNL|GUo2=MeehUmS?{i1lTbNIU0S0dQeGF* zC*V(4+Iid)&GWwoEv>izG9oU;S@X;Qe z4*1u(&X|y5MiwyU@N|T*)g{?s)i00q`qVu9pvNRHt1cJmT2=fejt|L7I#{ z(NvpgkCH^57!93JN3@q$Ia|>?O@InB^`%TDi`J&1% z+ye7;ZmfDqPmeM?w_?}Z`;1zc_PF*zw01Ygjnu8oolT6m=L$=lgafWTU+GRd0VB{f zK*brAzl<4Dx8_o8INxv!a8wKYWZ|kAZ;m0P-S>3}wF5bjuE> z8q1Qx!&coROcqeeROl4*2^J%ARfe(uqE`wOBLWpGMr1hPr6!`yR_CUZxxV(~Ds;KC zxYeJRw$iUovogpLKYA0tI;8MuIW(g|=Ysp$-=X+#{8H=6pGho%r?U}p_4;?I#hrQk zuut-D=USkjl2d%{_vkX*6BL9Frm+NKyR4I1%(x(`fnj{3!ARsOz%1&yMasKmS z0mOw17Ch2-8}zP8DqTDn>l_n4spb~q_%&e-D;y$ldj=Pt$893!bwqJka0X+6I4p?3 z^&VX+xI6E`G=LS>3vQ36;*qwOUe7Sv)W!3F1_@0+PDWuAB@A;ZRl%R!`a|MCH0a+M zK7>BO^UI3iz!J0f<+UEQuPJ_5Wi(mQiv6W=ceEij)g19T#ZKTEw1R9W(7tCcE8|sM zg{#-A6FxSKTdfr+r@?x}V8&Qh&=~v2_{M}0eRCl08-~R(0nJO9Tl#}bL0+cR(h{=X zBHiA9Zxia_Hf82&FOHlqtq5mkyZ@ zzeY;@M=1}DUFd{VTHe;q!$zp1AWBEvTLK$ zTff1?6XP455`DX^G7+@?=TC$jl7_vFQu8u>=^fJC3ID`%*@;V>zX=l$Q}G|bL&tiv z8Pz3Xo-T)K6XaoRHf8eU+Cg3@r3LCPFLVP~Gwy(oV;p=O@FbU`P*($jx;|F?*pQLny|l6q_}$aHQ^qutZ9_s63&EFA zK`UCM^9v<*D-ls2dQE{7bRWz6z4Zk51i_i1U_`pT)ryc)h)imA)`-5BMi_<`;A-A` z+8Ri>OByrDX&q2E`11I)hD9lqKu`84d0Ws-%+(7^g4tYpxpHdE-rGQ@--6sb4*8ZQ zELNJ^oi^;vyi8Wcv$zUwwC-R8-OHQAPr}bMX3QSabxMvZxf+Jph-xH=-TFvy4fH-= z=~8cgxH@uv_6IX8Ii`z*#~BR(LcL{WiX@gd9`}_(< ztYh6wJ1?+`xGobgo9EU*23}(P!>Lp7n$0mvISHi+TKV|JnNK}v06mo~=cR#OvY^)L z0`VWRY|OAQ(;}TWa`D3EV)C!_TcAwxuWqav)j@IRRAmc={p`qJNCZ%Fn-bhd5OK6d ze?nPpAf*n6rsOnXHTeVPVaz#i3XcVBMUqR>Pk~Y3R zQ;ntP_MlaYr3d3uqP%Vj8)$+xK@j^!ibN)B#*EsIJTxMk)#tsVDCTNQ$(R)PC+^^7 z(BM9^H($D41TM64x2ra zc}f}OH_IzS97QgPO&t;|m1;-Lp^vrX(jPT%IBos7R((c=*GqWvZs!w%_G&^d zfvRF%s=g8BI3=DPZt)5T(7Q^@R!vxFzohbDyo)6fq4wQaqAj=BebkhI*eFFGziAvm}OkfaHLu$7S zcX;SZrt+g%@`W4D`#Om3a=KGaQLV6B`Fmr|^CSPNjPbNsSjO`CGqdh-b&*$A3&>I~ zrwF__1o|2AFwq#41i;k_JTBQip_MjyFp$d`aQCtFeSCF8c>`pR_;?SQBc?G9^GWeL z)fbQnz3(ATdOP-qkIlDQCz~xpwtT|~&uazGS>t{ zRXukTmj)Ixx5~I|ho2j?23c{$8O+7aWdpqWwOqUwgg*=Gr0G=mb_fuCRX0!H%#&tW z49C9&;Ma`i>lmyBM9fj~vWm3%(D52A25_Bsw#?%dK+P#nU>cmw*ec^h8-6a?s>8BU zV!b3I%WBFm9N!>29XNZ68~EzWxCKQr36sj^)$k9ij(Nf#=Sgz_RG-2s=Pt|?a8v}h zwM57&;wvgusDS_TOO?U3eq})3*6F>pAxY_=KKR*h1j~gA^C732y~Y7K^J#i7xLt`F z6$VZ+BVTa60&3ec-w&XYWX)8uZ1rcn>9OOTALW&d{`TCRU_~=h`I8&+{s~k?${Ed6 zOZ`NCfEIaTmaUma+1YB&Tq(?fh=R;>J4}sppOv%K zq@rx%^X1lkXoG9Nf(=)I7Qnk=>-!m-C6bTK6qG#$^=l66+udJ6ouS68#kx^o%>b%T5Uh zFkAMtFwGxs)c*mDHL}%kdz|#v1$4TAsQ=@g3AfpGRZm-hYxHBQyrB=Qp-V^f%XM%@ zs(K(_Oilepw`lFl{6+KpniA5m5mnHtpg{{zgG8uX=xjrF_rGkEj%M#dw70hpx#s`* z72Y$sovlm;P_dF8FhZ}J+=L9QnD}NDo9!=}{zVGb{J$9-ynUKSsyK8vba!UksC;70> z03Lm%7I9{Kr5+;Y8%mvNNV21=2eU)$Xgh7`!B~Ed5@$))%S7vUVH>Rm%zQh8pT_r^ z9BcdS0IVCtdLEc;1}idrhoi*urnY0GCAqF%9plF6|K8OfH2%Tccl3|G{DzY`;&%Z8 zX^X^!+WIF;IvvIDppm_h!}oc;yX{93N5{1m2Vur}2VCu3MpWrpb<7)Wx*9 zckzoqq_@n{r6zwf8i!tF_{6bln4J+;kt*=h0cj#j^87rPR>Y-^g2G^aVi^Z=CQ>zE zB0fVRFE5an1{UhvYpEbRcyn5;`Zsr}k2*-JTj&}bKetk(HVe6}H&T5_{1WXMA;E(y zA|lDh-6^-aD$kdf>aZlw$1@%&mkQVUUvQxS`vIYVRbV)|EK^H2HvlHb{KG?UoX`f@ zB)Fcw33s;2Qi(Kf>msvK;jxFQyHuQ~edM1vfT*QX4_oZYra~Dbd8Ja}8vnDWqb$Y= zrq9EkqvRDLTX>@nAVh@cD+PmFf6&LXe;|sm{M^ZSG{W{$y#2`i#ZPLpPk97aN26>z z8Pve&I${%jl-$(5qG-BDFqj0zYg9E8L^vW32*(;kIQxn{>m(=2U`a7^Ox>h2hx0dW zg_86GvBxt^oSRk!e^&ZZsp4JOHPu&Joqk5z%#`d#cqQd8F~g#J534VH1_UX=${=E3 zRhFVsS*E;Zu}Cf9(~`5~evoqk01BEYefEvLo24%(HfU%F zlQlD^n+BpWALHF{#@BF@s6miCPsUJxl~t=)rL%X5bM0xseMw_&o3N zmVUMa3l>{tJpU~sBPn+E+ZXNik)4NQS*<+)ze;@VEv5}^nGIQ*!>}Z`lAh&L>4kn$ zb#5|tt(&MfWRB$>Q>8wr&lNTw9pA*d^$X6mH2)GjpB6ZR3$CXtC%^^499a;|ksak` zChJE>^xe2YYD#Gcw!X{SIATb6EOMYO<2IyOpMG^Zq*L{Y{)y~FxVN=cRU2LH95JxF zD0>7S!qv~UrPpwj*g}uZS|jY`GH4FDb!*#*)9z=P(v#6ysM&y_HRxIe@Ng?N3@mX) z%vuB9hj40k*ZzYAnYd7PykaZ9jM~+8u&gW~xSn42{zv1= zO0i&FD|~i*b3+Sa*(SNl8j{q-VSxLFjYFF5$|ARuekRW}%Seg})2Co3@o#v~JE{jr zN)-trXT~fR!g)bfwI_;T6&j=g02^K%33T^Dt1j{b754NSGz}*>q9-WodB^2r|zuHX6nTpij z)ho_oX0xHnt7WlBfPvb-MJ|0(HE{LBQ*9ZbBL~4z&;#m4D{OJ}95H}7gVoQ|;S0Nn zI=C(|Bw{!o7+6iE*R%X`*C95i#jGzD^{i^_iG^gcLQ2!6YfPx4k?x7lA1JX-j|Bqfu(iVtR zuZ;Amqjm#oo^&r`$n^ViuFeqqu>KB?rlWlQOV47{S7CFV-k}i}0R%T2Kfnx7dcR*l zD6hoxqV}l9dz*`wo#&a%b2C^xCaY5)xFtI9@q~z2t}6`*DRb|3e7)W2^{)V_r8~tp zcrY3^+Oez0#0=6_TojdffuAJn%81$Ro*e>%=yFjK!cx7Zv7fj-+(JHWtHJN7|61Iq zqx}7(qdV8Kb6UpHxOZIP&pE|bFE;roj0rpjm!e_9N`yy6<9$0`gNm3NlGZ0)?s8dx zl42z`W{3y_cLuLi)V*x*i4olx8>_##RGBzDWF1zKCcG}SqI!n|XS@knmpY-ongxo! zc)|;#Jsy}OVNm_Ha+gL2#)dgF+bIX{xbQOQ&p^--S@!@sI9CgyL;cj#wMNAD%D$72 zS3fdb*3DXxrqS8~m{}vFTIq|I@oat%3&9)o{@ck+U zo^wsAGsb+7$1jX0;~80@0O~%>T;>G`o^HiQ;bw8xHwLS;N~r7>r4V$GC5zI;!Y5%& z0s4ckU(4Y3WnUDysZ&73bN@}}hIqy)fEI>NRsvIAtOnU|SWBWNRXigM+}k?N0*i*o z{Gv3GBSyTNj9ZNG1vB?Z$( zy4j)h0YPlRb}ND+Wcc{+INj>ZhJK{}ZKXF;4WBKe!-$hv>Hq3hE>Aee0%LP1Qmy&a zH|{u?s}3v*Ap!^1!@wvf;_`ghIRWb~LP}b_LCZ&(+AWX4>c!+D8L7QiW!>9va*~@W zGFDFU7Oyh{h3KB@exMnr<$j(*qbcB({Ixd!1n4K66Ss(P)68)A8$GpvZ5Uq*yZ+9z zLjxKN7|@!oKkSu~!5*)hW{&XIr}5+o`H1+o0?nm}I5`;oi0PHZw`L8uZrf*>CVwGO z{ZUIEg!CFxHzGPmE@dKcRPx;6AguN09Uv1JU8g(5Jb|&dco9lS43Izj8 zF7O8|68IWgCr8(+&!#}9@9~=h%4;o-%*K{y4V&lqJ%BQ+;#`ea*WhXn8-i*X%Ds|3SYa;$QV zHbaVgsQ|p2qO^&fyV#l!`OfD=9(?7LS0jEB>>|m_#i8u!qT)(f)asQS-?W&Qs`=#3 z+b@XdN74`NQ*uPr{+i-irTn_jIkg0$l>OmP{H`wPNC2sq!=t#pj z6*jY=4)XjvEM9W_Nhs#aciSfeXHQNpM9=RfGft0^`zFhYmX-Jz0&*L6S);PA7vD_I zEI;iX@hWy3#~wxEvF};C?P>R-HVbM>+5+mG6G}H%%liFpjzf@znGEG5Z5&{q+t`P! z4kU(FA1Kwu2V&^Y12NPtbHoc@Y^=6Jfx^+=a0xz^?T4@(3x5ea2>Vl>`hg&kV1Ax_ zQo@(E-EwCWyL~S5&)@Jk_?&|C4hw8q-g&=~dpylhdTbg!#d2>HBn@4az|i3|qquK4 zh0@gTv@vKEvP_sP1vEWNgSY+s!f0nd3+*qajB1`;k~!K{mpgbr}k1gmxnw=TO$Rw}JFS z#_1BvR8@D!%ghVkYUDZMB`->3I{52-itjpk za-A${xFO*JD8NM#+HnnMk!s57!nUh7wq&GQqu9Bb20 z#L75&#ZjiB&YEAv`K87!_V-kB9!n4_UFh2~cBa`OXVQb-t>Yd_i%P$qDcvVhHl^q}G-!~AGgy1& zuiYoM**Os2Qf-F;u$Fn}pUSEpb-de(iu@fja!h+_W!Y{*l|uXL#lPZeU4QnEzFLBt z8RN-)Oug%S2fi9h-lr=Wavn+70q@KIvA>nRq01^Y2pt<#IQu2%*8SU8l}G@wonHQ9 z0t{`UP;D@37O6N}jh;II2{1P|p7@-~;xz3C(W>;^`)IAh3tB_(#4o&b2(U7M|0*TcPbZwUT)a z$nUJ~xT1a1>VBItc2C<{@m$^e?W(>46Q70iwX!#?eIcDzPgO^&`T>FUv|PK1R&zNoU$4EHcde7-=@e5T zLa*x|VsxUtIb+s~YFHdjcrxGU_YQ$R658v_KyR8_?b6>vkntx)A-g;sjiH z1Zt)-?^d%tiZg$2{dzJbE04q68Kw<^J^xVAfE|nV0gcSz1rToG9HzjW7f-&lnEi@O ze@dY4XBh%?ckEYoFoV5qRl90alzj~)d_rxiNygfT93au&Lcc>C&C0f*%x8$E4L4Aw zTfF9T537%@N4L_S9e>M{&awqz;L$=TidA&8nb<7-ACWiQO3_XhiB~2jI@}m3^rnv7 z5fn>sw)5TZj%rGfh!PwGOdq@g&maMBX9#kGznWKl{2%JA-c(-0>zSfmw9TM$(e|Gc zc*}{XhoOAG`h8U&Z$^hXTVa|V!&%o8l$ZL!^|ltKdosmPIXjloW(IU(&u=7LpKDRo zF_u4wJi-1<;o@~P3excH-|DXZ*kLB6_i|$%=rMlUC5fCd$_1fmMB-s4S1POOR+%GE zwpZM}h0j+v^QU{ZXik6mulb<;QD-(Y+=7MOB^t#AKwROrvPD7bFJ1f9>s!}iD?eIp z7hXm4uGXv3CJFOD0&M9@vFFb-wQAF^zrQ7U$YBGGerR+q8Bm(C(o09kA%`=n7fQ$F zVRO(lChG-#=uD+f8152=r#k)M!|k?$A#$l&S=&cpoqleB9u-N3i0 zXLXaDmcT*)p}ww+HHu+r3z7X0O71p^1;`qjLG=<*v-oSzHaecvQD&#hIL2jXeIuVx zg)D3lUmJqe-W<4Y+8rTyqdP*dqwc({);G}*Oh0ky!4_4Tv0HX4c}3?tMaVSW+6y#o zup;t|&R+fcXjXa|9r}-}s2An2%%$pPk_v#!&hvT8`pcXx{+n3o$uf+gRqDsk&+-y< zukn7n)GL6Ed$?%*o>+BC658wyVoa(h53PY>oLrnIqORJDUnPvft}8Wa{!N(_goc8= z8BH^EW_kW)`1!Q-hlKhPnfCdJI@R`pK~gBU{iRnw2-3j%I=F&Hs154mA8>!Afx31d zG5M~QTw^&H5jaiv&>HouMqgAO| zi^CMBp{CS^Go*qo)88X-L#1@=& zZg-PcH7~Z*2#Nze^>ZVKCs7gORnh{oH47DAJkg3e_NP(y;WFS+&GI=J>&x{S*W-G(6+`zW1}!4KP<^AHR8m8T z)DImqcwF?pDhUMt-k1^XI(wACU)?AvhhJKSmlgC0eppYsJHgocyq}fvFRqe*>+{@L zhFjNzU)RzY2DQ6P2L-iD=ZeOI*y5mIUghW$fg5^;G{4t=vjvrICL z7t9Y_;cH8B&9wNBa$Xw2z^#yt@Pv+fYcUQfjQb%44{l)6RjhVyTxohormdJLM-eBe zWT(D0n@W8y@->71^M&Nrl0mYd?tBgsx0b24jJGs$@*)pgLoU%NFS^;M#i~C@1@YPg zv{VH6<;9?-)+teJ{4YzL*$$+fJRB_F*Agv|0v<358Xz|}PBB#*6=Blx-b9B9Q7pU+yM=ZwvLBg%|Hh1ToSyXeGX_w!qj)nLs9JGd$DAIxr^eAru{ zvNTCqmbmNayE83vqoh1dZbAO^EAH)ep7ey|b4_NEgRpj!V+)9< znOAbl4h*N@soq3fXPC^7UW`cQ*z<^4nF1QdI-+%lN-Wl}^LO?d#vsBfxlwe~7gBJy zRI&=$wAdgzR^G9>%NKZ*-_tKzV18fty%}{;;7F%RqwnYg*0EzKp{xPm6h zn^ehf8_hdoY)?Dftd6rkZX^i!)!_)i)*teM#rkR+vA6NN-$7b$rbOSbMm1_(;OH+O z>zNcAD59{c^3J0Fg0=-LDT4=Cw~Q6XE)4wXoWgsm(|U;TAX9 z?_g^7(@Sf}6-3t-Q6MlWgZ>AY_;%DEo-}lC8z<^#mSzHb`F6X`llM7G0nw+A+gN%2 z5#kJz6W(F!glDn}gitWwC1qcEZnH-^LP*6pi@xIA@pOk7sh61 z2dL-AUhUgx%(^;^+odP89B}XuHjb`PeL@_I%v`^%CS6Ve+X42y-J*r{g2YSM({3xh z20XLbVlaUG-|co8l`4*k(}Ti1C$bDrcM4yx_)T{F3vkQR20_VhaS4_*yjz0huB^7v z_o;?$tg4(-T%~Qrg`KHLlYl9frB>6N05d&`!-g5wm0w;tAZ${U)V z?Lhj4%||VjBJJ37y@#D$bd!nAdzwxu{VF}r{0&H9$#T3GXM$rFd& z>id8sgx8~~bz)nhsg=b<{q}$7U5V3MzmXj*?q?Uog?H@}Yw?2D&UT$K1kpEyx2sF4 zqYT!1&W;xxH_hhka|w(r%qT0vIum};H<=>viOHVUIc9{=`x)#D-$PA*0l$%5hS8&; zOK$_K*SRUS!oG{_=Q_q9ePX&QR_)-s%K^w1xU#|t+2^X8vt_R9{u%9=i?qApH!|{n z_+0JrxzIE3j`Vf7v(5Ds>dMH}{fVa(P;!_NnY|3j_98q}bHywZdO=t(7;a}`g_#F% z3ku)DuI#+FvKI4yP-(FO7-D0~16%#d$j52{We8m=t~|~OVXx97Uc2vu9`+IX4glnR z8s#wPle0Y+BN7Z?lVAWV?zGf{-QZm%!t?m0!<36JKdFuUG z&y(HII&5gt;uqR9GxNOWf1OZS@*I11^yQYi;9I;E}|Gi9uut2GX36^G+Q2#BrZTTQMP6I z#|1R5DD;YG32Yh^w0 z1XiHtrxH9&1TcopJDtFl%;5sRnD@OuArc!L#Yafp<|NBFTY^U!Bqvo-SI3q!9xqF( zeVkx_*+*NNQ29e1|I!yRYCW+M=QUobwru+|zyiGFv-_A{9M60|~TJ zx`gQ#bdCrJj^|>7&#SPFK3k`~S7NZ&@3j6#$lZILFE#6nv~Gv` z%Zm-qG(t|YahO2D6tuIr2Rl0f?QB0z&BL%9C-pz?7|M(Un z-TxZr@!`b_6q7B%ma=@O@|OS?og_{9F~B;6i}8*^%0F&s6o0bQTg~5sq^R==wrvpa zxIY9Ydxm4X{%Y^(I4bz#(4d(Q71GfH<(LFM)vdU~J^wKMNQHtwx>a&Z$wbpewhNi` z{K53(KhtW0D`i|I`bwOqSy$MOm?g1!GnhX_GIoVT*Bs*MKa&@s%XD7?SAY`YiKo^v zpH9CSK;e>@yidA4FQ+-|x+{eh8w*;2WZ;@Ns){$Dn??uDc0q$2W+YnawE7Ho0}**y zpmm+mwsBlkuus^ZDFYYZNyD!3NPXOGUKncT53i-XOsp#NQkUpRaf5OFJOC;7Qnb5zhT3otaNSof}RhZr-TI|q67 zKc;c9KzhP;d$sN4ner!S;anfT0uk68Pj=^1LWRlU@IsuLP_{MQ()PQaA<>Ce*l?qj8Zerm8b zfhw4nM!+Rl8F2>92R-D=L9oF~-77g4%mn4lw|AzPXeZ-lRpOSspQm|*r|8pqOITH< zy_mHQ_J)Q1je^cg;-D(ff-Org6aw?0h5X>^g5EG&94AX+UyQ>7;{+9X+5^^XwE3?% zv2kVRG&Z<`FBxbiBlUm)UyA-mNcu>wbDBx}4@G95Q}p6`-u4D$UyBB6EA_rM&)c1L z?q!-@f%5f}T42?@(9czwX0jGQzOx>e1s(3Z_ZOG*AG3x647L58A{lTy0U`wtJfAa9 zH}ffaD1%bE95aT+?09qiy1&_mqE_gLl*qSwR-aJo*T=n7=nAt?l~Wwp^LF% zbI87C(BFRM^;fRwf*a)JA^(}ec^~_Jo-ux@EQKz<4Ak%Y)Y`&_@tY)Y{APROe~sUy zJdEGS(@R9bmCLf>1@)gjHpqa1SBc5qdXo5DHJ>w@B_sQv)r{k9moGLDk|xsG5^h|L_IJn^{VIXHxGA>Fqr;PkFP~?AeS{u~J{w6#78eDnsZN4MbR;j; z!rV`$4x#~f4xHYOTlLL^9jx_3zrFi!LR){iVu}qp+mh@Y{>eJd=#om^Hh(OsY)6UA zKGfK%pvc^%H<%yYoGp!ny1Eqv&EMq&6egXYw%5iU&ZpBhR*I8wmQv1Jww>_XQq?g` zfd!eumWRY=EP*|RA{CaLg*r)S7W3F)rCxw50AOhk-*_@NV3BAXIckpNl)edrMqD-R z%YcXlnYJnKlGkKz2V0eQS*qvPNGC)7K1a{6-1`j@DqkN6l~4bdQ27KBDiN!a+V|rD z^i+$aS0vK|CzCuF)<^X6!EcWTus#-Y;J&Nxa;EC5L}!r=r|m%|t!~4VMgY~+LESW7 zxzyc?dg$h2QAh)iYf`N|IAG($q29_e`d=+yOJK`amflP5pWr0Xfa_nksb15P^Sc+< z5j}TNUY!&@I>OEWhpw-Vs^a_GH9#q)L>i<;x;dl*k^<7*-O|mdba$6@cem1g=sI*a z($aU1-(S3Uz4!jJ4~xaxYuJ0v%;)($v3Hg%2PhgyHc1ke1G|h@jVcT$H3dU|MPTt1 z(U+hI%woh!&8rPEH?@>kdm*^u>VCei)-3<9l$cdkZYZL+#7=J%dg_auLx(o^omi$%R%LtG@0Y|YNWC(Co8!P6~{ z@hE?6{H@pA3A@-SZ+cR@$R;bLhzlZPq?1!n=Vnw!c+|{Nx|B>5rXn=e79S!mXvqae zI|z5LOF(|MZnQUe~Q5MU(^f`N5n?=sx9(74fHG}7w>r864gDA zYZ34-JyOcZYU{x}J4#*59vpSEqyy&bwoeb8+!!DoZcMl-c+3VLHg4$wRp@P}T zm#YTn8s=+>jrww;j^`~dO-IA6vlq^si%CPji~qPt8s`pI%YLkj^XZWJrP16*a*^wN zs80#-Y|@n~R~p=}l6$JPn#_oW)2f-~ZgUA6ri^l?HD4y^R-+$_pku&jTMbe(L@TFz zPt&Z{4tArr@WWbGix{2tI1b~I4HkF8T^Fu>pl18Lg6Y3m=;}!YR`7lk%mi{LX^t&*)tc#P9u^paDGd|1vb50$vl>VCytBZ?ukaROWoFMw&0mRUzNR z7fAAochM+09ryY3D&LNN)Y^f44aug4Ccro*Hf9g%OLc6hmEI}5Iy-?Ln0NJhHO}$F zD(Hz9=c~2&vG<8Fn?xR}0Je3YUe$$CoeCv}R^!72d=%Qvcn_W4*apCGQ7$p*fpF$i zs_}tG_bEuP7iSDtHF0vEjlDGUw=a{YQVi~3}p z8iOgk^B4(nQb#svFxt_R6-wK)_h(SO&Dn_+teFf+!18tw&QJAz{=5iQH>=rcV0ANb z`o#0>=`Pvg=65G=SNQs@CjN3?^VW4(oUEx664fHEV)LUS&M+#*7-a)|{b^2e0~RCs z5QPNb$VLII$^=w2F2s3yCaHJob;Kx%!Tx%yb6#`k)bo};2#7EtTYs*(teN51Mli`F zqm)Ht>ieUqQ;aFD(PcmV{lyMlpFPMT$d1gf>lI0{U6zN3)oE6uspS$4e7bbYwN^78p0#i{p&3{& zS|A<>NWJjCO!HW+fzmYDkV(7+r4cw8idC7w^$)%)kadw3a5ba9GUm=9hDtAt)sj}Q z^;p($Y-R*@(HFHthmta}*U9nT=)lkzB+ELnf5wf=2~@Vqcs_*Q$2C_0o+ra*H`d(F zp(*y7R@aswaK5$#i_S|C#V&n#B^v_ouSIU(CpW^x*o2zIgh4C%JwX<$lojwj#6pO( zoxA0-9j|2TX?G>%C;DBZo4rs7s9SPxp#u;9~ z4gUwwJSjA?AI93R{K@?}-JvjsK3s+}j#}gn**PG#NpyZ_!Vo>dU3kqma?J=a)t6?^ z&u=-JMc*YC5kJg9h-U{k^@}LM&U*da!f4+#P)>iQ;3;?=;*bS78(~-?MO`db_Q|SqTcTqai9zk*8ag`Yv>$A`?`MHGumL2*mRl zZAlHPe`0;q_FZeYo39*{j9nBnA2%XMVz*%BF^5(=k`eLKSz2#?@8jV#5a}yku}^%1 z{`HT!=wd|TBXK@i2a4m7qfDsG7)VJ~wS${Kj8X-59d3*Z)hxl(iYRz$1uKtb-jIuB zQg#MtQW0b{E>=wN);BgK1MwGC^0%*slfz_4ruAAo_S+wq(|vj5n?(%H55N=x1d%8D z)L%PD!F>-va6oYsc~S}Am(Rp>psgU<4O}3F@Gp(?O|_rOM%fm;jVT+Y-mZ^k;9E$? za)ujBrFC|T3z#IDQQQ;iC2?e4`x+RRM;g6^VKC@t3S7O3p8@{Zd3=5W@gu4NZ!#hS zXfzuQ6l^c?f$gQO(p08`FNf-IM5B~ckq0800yv_n`46I@0}+kvI}4FZDD$3W5I2GA zS}k+MJ;fjd6?3ia>yZ(%?0`6NP3NV>pcq9#gyMxGMZw5vCEye#31V}csD9cRYambZ zog+hmz*|{Scq+@=z9ezR?t2vmEt2rw8qpt-s&^Vma}(N8v@?bXD1pt=GLI5jsKr8U zr?J;%Hr1KeBZU`wDb^!FI!z4Zbi*UWWPZu;Dv=Zz3xXXPdAKME1rzlSj%H@yXa=UXLnHLE3?yM*y4D@#G2D!%7 zUX|p|53AgW$@_|dSs{}kDb)ujr66l4NlWiwx48b|S(XJSsvxO$A}6dMTXy`mXwhOf zLcA9PMm|_i@Kv01ub?-Go<}G$cF?SSrYA8cKi|=S%w}W zZJp>2!br9LpF~XOe~6g4>pAzIGgpj&W9i&VgLY=lon=@{KOPyEiKOWP)2_3dncxyH zYD4J|#P;So`k-<*L#vj~CK%*l#x%j3DyhlGi7`qObg*RB*P;lbLFkg` zgkrqn`oJD;Eod;ho(*<%d zR%Ih<-7gQkhbim9S2%)m$waG?=ht8~l~zlqx7MZRI^tIA5}RhJ!8|@a(r;z#*boA| z)d{*$`BF3V0`LkriWS?w4Svb4Kii95l}TRR4u(1V#v z?66x*e!mnI$?DHHEx*Pc{Idkb9^OsgcQ-Am7tpEtCQni?TDqs0xQm&;c&SuBe;7HR z)4EqPxA#KyS%G);od832#{8sU3$P56zPTzY97=EvoiEoaid%e*UYA>tTxy<1{ChWW z^0@m3rDdheWi{#|HD|n{E)TVnqBm%TBE(9db_eeO>B6s;ygHQZW{5vk$vp5f4>J=W zt}I*Zg2`yuKm>pNsd2T}Zu5r_!AM6Q;(~hxE#VP)!uz-ta^1U(Tf0l~E~oF(9S+Os zL~6vfbX>8k0~2?L;2C)??uTo5YIhZlZL6=>7K`8CR_`~-@>~-&wyOAs-IJ`1w>i5d zu!=oMA6!20PN&7G86M2d5YONLkS>wWZvjsjPoj$ta( zXb}>>U)?=DXcfLShCydt+_^j0R@ZhC@Fnlq7O_psQ?}=HFOFZc1Ml)N!K!OqY?W*$!Xd0cdJK-8c;MIxjWe5gY#78&==g+={WdA+|v5}VlU(a&*~ojx9`u&@k|2N2{Mg_CT5ji6c@ek?`ssG-I}2F z_ex#g&rbEIk>CY^j@dsD$~Rkh3=VY0AR&OEV<71JF0q;mEcg4KmM(BX&Sqd`R_PEn zisGOtuMhD1lBrZVPMOalLE!nzxhZaMg zUMN$lS)s<#>U8)?8bM+#H9muwM*8rC$U(@;I41cR9Zs*a#d>W_qBW1gk8r@1XPx+D zIKz59b=kC;0)$e+!w_E{@-2|ovR0X)-iokKE3`_tcTk)y+a^2<*vHy|`65mo7 zb{?~2NpBXGNMDd0uWW}XM(UBzzFtco1PC)~KD`$-B(5A*Sm@;mnFcNJd!Qg-3S8S> z_}T^!B5agYgKC~R04|o`Wf~C^{6UhP?)1-EjH!_UhrJ>z0X4;eCf2vSj}ye~&=F=h zMP7vuH`3v|QfZsWzbT`T@XsDs8KaFGiUQ#;*>Q>xaij}a_1`R}p47tmtRr}nJk?M- zjqq%hRbUS%VqQO;x_{Qoua?%-82O{&$4CegV4}xu8YugCDlvdBwM}}n5m8n6UAq9b zn@8`5g2qVi-VdPc=J`gE^h-PN+CPBTeqfr`=mZyUVN;nmb&5!&{j%GeFv1*NBXMN! z5m6dQOi$KD5x|5Fk0pOKrY`S!7O*gi$~cCy-;HV=XZLcW*LT^JGl*_t0+% zUXtX-94nX&Hu-M%^IvE!RQbluS}Y#l#nT@1v11nRU?^(_;E46?ioHtUjEH>?NDa@n z2|%_cJUQ1>*Ma?Y@aEj90(WM+3WOXB(Q-HgCFmUQh2T~j@qb%w4oKkrf84A5oOQZM z6y;sZr#XSuvA%t>7)jzJr1oDNjCb+7#dYpp^D4{@f@L$M2v+4sA<7@O{}j$KAdGzj z!dS+~lNDoNRkAoYp+=S^`c5R}HM5AHYAO;lA z$pI;tk=;s6nR-&~1^Kz_0Y}B++d!IwYsxeQDw3#|5?cv#l*AyM&HhXGrta3Nk+Rcl z!M8r;X!5mLav3vE52E70W-?WR(Kp*40xO`+h9vgSawLo0)4@T2ubL{~;48g3PxMca z2--r!NM{rB+QFo6#WXI#X=0N{F38ql8%yB->R_jNaJap0q5Ar|pq0Ye<-opYV|ZKk zq_@!9sV}P;R7#ad5W)H1H98RyNd|h`EojEa(RuZSa7c=e~(Q zq8`zY-y3_C=mNHg3AmgJOMQ)8!s{=QWTS+F)bp&y#gFYGBxRB%x7o;IiBysmh2B9B zVF!u@*NAn_?!xwC>CkRhA{3W0M#hKfCIa1JN~EV0-rZ{ZveFbR@Cmp_OC zQLuSH4k#DS0qJzuXIiq7CJsKCxI#K7{%Yf(8TNaKK=A40^t0vk44&wTCXwZ0S$16Z zE9x7`_2kv0@l)YeGhwS?PhK6nmH9eyPy*S+ZlH7i0g#6q2ZI_|*6tTRpgM)>?8dV# zu~E>?48DIWRYh)+57hd-!98%sUrtRiWF!2l;%AEbc(cm)M;c3?@4B6cU=%7p8~6lh z0H0oVWDP0b8Jr8(<)iZU*=n3&cIr9jhYghP)Nu{YVisL|ED)= zd)}jCPK)1x>lh=E>;~tqA+g3ms?F=zV8WP>M^s9x2dGd3z#r{UoL~hf5*%Q%oLjUc z98w0V?gKwU8#~BB9iP}6gP9_%KvuaW#N1744pShkXs~wP zk1XSy_1?EdahfZr6oc>}sog=7D+(&whuBEMu);5FeNFlg|Nyh%AWpekAI6t>G3`=JJh#1~A3>c*+&6Do*c7^V$vk z09UR8R3>-ogTiF$IJYT9P}G_Sf11P`H9wzd_ufejn(3WAZ8^ioI^P|Wh#e!=urONM z1LRf*ujJ@gEg~&-urUUIx>!_`8te11lFdO;rm9r)TIZz1mTq*@BEVOF-xb-;7a6s= zSqZ)?z}#7k546@S-OIrwX>34mK`Z2oJl zakU!GfK8_tgVvgZN>F|ech(%j_htd^tl5VuY7QS_}=s)vj{QnL+Fo@?6L2jDPvL% zMQ)xz-zNdE2~@MvhPaj7mJcB1@D`udEYf)u6Lx;tfQaL27D#|TBp%`(03yG4Q-BwM?upM=> z4tr)2l?ao9P&)uO0}9wc`e#K2=!vo(3nDojQ50zfr)^!3bl0}5A*pBmOOt>XRM)0OMXZhODz9@lt_sLKbgIjK z_*SB_d&6shD)~Y?-%CDS0;Xa2{KWc$Nu}E}#t2y!H^r8%Ja^|i8b_aMO})NgE6qW+ zsatiX z*Si2!2cc;P#04DOiYsvnuwlZfg z=x#Uai4Lv#g508SdT#CB0-U@So;L7^KGNh;Z|-jKdOx)Es(PWSQUG_z4_0HoV|Dz+ zBB^nwP(bOl!CZl9vg)ms)S(d|J;Eiqhn_%JwZl}&-VgK;zIh@%Os+2C65P~IsFR9r z;bVol_z=d|(!D~kMtAXV{|`Q`d^Fe-^JfyIwkV5to*Jg(V$Lii#HR-;bh9iEUXcO}(Eihv z9Mbl1(yXl*$JN~OgWQ0&R02911l~8w>0Q|Gj!@4ut*ro-iSzXyFbbg_R!8V8HDnvU zrafuP^cznjIb|X3;eN(`2EFW#@!;;q+=a$(e=qbKO*7Ig|LDtf(jm-_snZt?nrvIv z=M9DKO(X~JoMx`fJ$767uL+vE>7jMn$?gWilQ{{-rzxPWpv6H0_S3l;ydf#~! zs}1?afgOOR31swB1Jp+w_e-%T=|Fpisiado;l$cWqW_PiffYh#3cRNvhaO#1NxbtvhDy~mUV zC;jgOGz%V)u?~mbC!}t{4G4g^o}hoOAZBo_tXYD8t-QazNOe3> zF+8l|Dj)8-d-Y3d%4)jm>k=Kz-ZyD9ZadhjI+aj69@I6Y&r-FU;*i-7^fM34yJ&AF z>`~eOd|KS~Bhe`2eCx>|T69ZZS`nQA{M`bVkIdI#ZM*KsTe)e2p$3fp>?RDUWhd(Z z#to6375&n6Xr*A=GI?=fn})-709UYHL{;E2`d|63V6Y24oVAJ z7Y_HH*|IB4@ht|t$EW+!(f+v`3sBzuxt>3LMQC)l^T_sXiw+K$&_lnbk5R9hO2u@)2zhG%~8Mg{wVyn!s}y1 z_(y(e5>%L^!sj-Rbh@8l9?Gu?TWTfVY8tGslUqe_Iw?_rBk1uW%zgm_%%eHM^{92u zYV~wzO^W8+AMeJAY4v}x!D#_E7flixr*%f&6h1p|Y_`*BJ+>H})IsgErLPbg`{}|$ zx#;oz4tK`l!`K_M{emfzF(e5*$`G_r2|TSr>GpECwO?-O9gj{|AMVAKP#KFWx`z6c z#H)e(iud8^(l}CIgn-eS{V;cC&{`x&c!c2Zof1A$YJu5F6yH_fwS0aosPm0=o#@U~GsS9aJ{IpcnX$x< zt1@e{d7yH}8sx0suTs5tzL<>scGbXXlHei%y(qL;NLfPB-e0Nr?UkTmOd=qa`b&F_)#a<~JNyfBkH)3>-1f#J8)KNTvpuNX(<5c<@ z(hybfuDkgiaP04++VJHG6Y^S@1l~VNL%;JYxjo}?LvmO0>GIgKSKf#yOXayIEaXa_;4uqD&lVPfP84@;7u@PVM7Nwr?GJ)AS6zvMlcRtF&`R%>WN*}ZdpuL z{Tx=Ykq_| zQ8A&p8Ksh?YL;I${@J0aRp324_AG00$Y#MyletcmCi zA+sEz^*|OLG*fpW6X?6Q7<5qYfMKls;VAevQJ$yXR`B5lUo;zWOvtZ0+m~^YngN)R zZzWWJYyBd%tbP62&HQigYmcJ26{ea@+N9KY#nMl%#pT__ejt+av51`2x)9W-J(JK=;fQL-co=Uh2jk*?F|vFS#WJCs1(R~D-x^@2 zmeo=|yXkzxKPZ+n8rpdD(7Ap0VZj}!HV|vXj9n0cz`~#}ZOarygo<@-hcLSGrf)P> z#L7}I4W3P4K(a;412$yN1VW(H>HCFKHUhJQfMlt(09pFX26need=Y-P32d}Q%dfyD zGPP`#lJLfmVpPDcXKvHOY~M*`Q%;FN>z;rxicm6>>2+agT!IviH|H}yDP@5A35Dpe zM&RV@rs19h99_oPdHyZ=$oJqw>RZHn5WzmFkb0A%&;?PtD4BPv&rC9MXnhw=wO`dq zWtqk+YeH_%aH=plwjpM}XITM$cfU1PD%|F5sWTkQD))}Ci9IV1rT!7lvFn;G@chgu zg7QLnJ*f(J*IA zw%&H74PRzY$41Bs-*SBHqa=-KZYpUjM7Ubsne2PM{`BO-I73J1XsX8u_^Npdf7Kin zw0%x@R)|w^4IsY8FMzGh|10u(`w|{`O*wZ{ zIB9WzMKD@MIw79?I^7m-8q)de8R~8aJwxr+|M3jHwxF;WE8ShMD81v_pdUO{O5ih2 z?`f6Sh?A*Q1&OMaa!lj!5=Bp`kieOf)=I6d-w4txyg*U{#L(hTpy9&%Ko+%pfNn(9 zhhd%BZl=^691{e1At6MfFRMnod-WJU3-J;u4>$7ZIQHpY;P21P(2x=Zbu+2t-f~D7 zc6HK{o5q#b(EI??L2jaXhPqU=zb{uEwY} z;o5f|>TfBI%8aLWbBpC$o$+jfikG!}cr3dC)|7T0R1^)rA$8v_I}(~(mH9(2;AcwD zDf!-7CnK0py7*Z?9e*4ZbBz42etsu74!AcWK&ob#No= z5_kV?s)cq`C-%Fr$0`5y54xJWm#-4}kO7Y7T6x(clegy#i#NIP_dtTuq9PzSPE`AL$I+izP*F+-XYI8kL~?P2l|Jamnx8A6>DWi&O`f9yM6*`&_b9NsBqW zt};up0V5uTT63<+N-Qn$v6sZqW=yx)O6bwb6no3Kqs;i_k<05#R|*)<5FmxCM}%IE zHQhg`7U8;j&5*j@2~0G6F~&PKGH&Rs_QSK+ZypV1!REi&HG-9`&(2mak)7efR`9#W z4I0LiUCNY?u8)U#C$IUBSL*2~W{=#9zLMBn`5$b7K?uG41D>Indqjm>&dWPU#-t_7 zC`Yy%S_e4hnp2uC|H$o@n_>(o$uq7s_F3timPpD21b`l64@NuCYSi}7GG6vStVR?) z3o=Vgpw(zLZzu9i6BVuo?!sntOpNzY;odfb2&;;$Y4TBFu+Gh^C8hLJf_DG7FvA)Q zcUu%G6Nl_>hmzULtj4&&?1Ed0RkITolrk2q0fHjBDsp)QDaBfbh**(s!#+WB7T{|o zNmRIdsw-nz!Q9}VpSD43uX+kjPYO0QO3w-}#EwociyDhOo6;sPPNV9}_91_i_6jZ2 zN$)GNk_i#yaz$LxD&8yGyBh4RNiRGinGICKmM{^HzjSk>%6{C>rXnbxSLqk!b+l(! zi@}P3zd90TWA8h;I!!GcK{lpDEewD`D){gu%$LHi>Y|6$G~_!WL(svbr3{7(+Wl61 zV@ot<1q1WuGYi$dr>g!^GuPOE4yXvPu;i4a24_qGlH7|T!Tcziia`mD>mP^QUaS=P z;?-((r@U35=*?KYmaVO-VJ~fp4+^Z_fDE3PV_q_6SsU;CHO>A_30XhT(UNXGeKT-+ zb~f*?!FA*H=(*M0_IOrh>vWG4lyJ>VO#s4q$Ll#y_ZH8fRQx*j^kQZ^YH-iBP)bVu>6^L`K?!M7J_LWLbrm{(6R| z%d!fY{Kml{9mvH>44(Q#^xg}hyZ)T}>-Dq=(R&WTcWaN5LZ5Pz3`~^TR3E}``_|E2 zHw($++I>O>uYn76|lX4AB4HQj=zv1U%=@soJQbI?Y0A`yV!6XX55-6K(llAp0+HEktc$oE{ z)bc!mrEKKbqG2wxFx}P;4Vj%(u2zhH#g``{t6K$F8VlGZ^`^L_c31`}u?Y`m0&1>2 zH?yxMDAsg;sQafVY>p#P;zxZ7vIfV&`&kKAFiyu13^0!ZW7uF+iG|9u5IX53+i%8< z8HvG^e<{&VNT6f%Q{|()&fX!sB-O7OiDeba4+C{4?ja>un9QfjzfVn|(=(I)uY`lh zx(u~q|9;l1a%JOb^aFKLEK2m1_R5I}YfpY@(#29?wSV-3OFwCiLmEzb)@~6JrJp*Y z8?Y78pHMCpSO%B(4?e0IpgeK)i6e)EPWhUB{eR6{*HUSS14cSKQuoyVWQht~9t{Z` zlWOW@f#)6e&{ycUi1bKaC=LNEtlpLgVUcPEt$EX3L`h23l=0!{?P~2B6k@LfU8#E_ z3@mL2k0Ie7YZ#rS?NA%2w2bV6!Q?n=cyQdj{LnLAlhi&2#xKEPW7jyIUrEfg$m4f& zECA|R6pNedWehcZ{bx*r?d8G6PcrUUr9Ka$RSo<7?(1#D%v{X>Aqp7cRt6J0C93Jh zH{(oi`0oojku`y9V+fDb#_ZEsMBz2bkzisR8PK~oiJ!>`SF4C*JkPM&KYbK zLuR`fY9&s#V>P_#f-Oldw=*EM1?t1M33n{Y-fBOm0dmd*EQ=xYvf$2NpD_2xi#mFF z?67WFKQ<4%YE8rpL$OfkYspi&rHtO!N&h4tFR`h8?AQm|O%LV_U$IIyf?dck~!YEoOp+zlN{hc0IHXU2cBh@q08r1U^XtwSeM1xVEMJ;EW^HhzQO z7f+-M+|OXy4?6g5Jh~;BNM7lMf3jg*6wA$<|z9ZhyPO=F~hI4Q`G zd1e>Q6MK5R2o@5YbY2g7-LPtBGC`x}AoY5~D%*Z4L1g8LoQH3KkQ@-m>-F+Q&}PD} zAH!}ez!J{dDYfEFo|{m@@rZ*yL{PZW%dE}6GxL+&l10^SKhkWliFt5JV$6sBndwmY zD9B4op=BiR_hM24|AJxJ>1|e6jl-~cG?(UbXdaqJ;H9;41TUlNV6B5BSBJgYJi)<1Hv!5C=za;xAyyu~M(WMM(r=Cn;{n;!40?IazV-$t05V z0Kdh+DFOcozv;)!d>>X4;v|VFMYB_GBZMc&%R0!qN4rYuqA-XN&uRROIxEyNP_tWV zkcE(9&MOG-HE5SI(g7v#zKLy>WkQSv>cWnSn?$UHh4wEKJ?ZL+H%+@`cuW=~NEt_Z zLh+&Csa3@m^K#&+RX`^hTbr^AE~nfr^T}CKI0e{PBu&QGK`l;Y^GaQiSPx3aaA-M7<3L;_M$g@*Zfert?X#$>q%F4 zfP)7K4`Q#7n-!1_$hmtym(7bN2eu*BF=(3rKSz=7-`wJ%h^o9Iy?T0fHom*Oi8qvy z#B6H~NEqiVA~K~B>+kKjM}~0he)xZ)Zq>@I372Q=WU682d-vW^}jM2*ga$-NrsfIw1&Rz*ZI~L&r zYe@ZveZv5U*ggzSMH=CmIy;zNfZfF?WKSyS%Qtdu-CGali;2*_@?$1h6T7*gKDkt8 zl9q+}&yL7hU_jNVKeKP=7gVF(B%h}qC@UpE0(lf~;I6i}uwTH=1&fNgpQEryWaF7F zdU7sOa-?7{=?31{s&sHu7xUqpdKkuKJ+WA6{X`;*e{5z5MPh6_jy7T+IKrGs8sm#l$_GEYPEd zkQR0y$EgV*I$()`+EH+uvBVS@+Hs`5c8-=TQog)Vw$bR6gYv*b?eQ7EbUQCTodB3y zK4TJcxi7{D2xqEmHRj7V8d!YUcb{aEOo4>iO z(*y)P-1XCf+e(X~!4j8wQ)^T{6WydaHC#f9cg5%^GXqzU`u?LJMFth5`FsCSkX{vs zHkjNrMvpccRiAKfLM%SLv;Wo)FJ45HYYm-Dflry+J!#r^%+FkRK-G+?W$2ZKkij=K z7CCMPzdO>7qHKQg9;K!Q!{O1${fBd+%weK^ zvZ~9t-mKgzCfWBCZe4LWC-`7@T&g_Cko~u9iskVT0rST`a$5h1Y2Fa>k9Hk*?i#y> z`%-C>KCekp<;9ft8YBh)*9oKycJ~&o6UJeRBlG}JBgzMIp6@}<^U_rP7S4GF{N+54 zrcQk;RTvco&vg6<%k(Y<@KAtUw^H8oOQF44>D=j>Pu> zKO%9vPBzE;I_@pDBj*{>D4b*GnkBXw)uYMLqff(>73ILjetJRE0>ciVyBB+XKRg4- z+v65OCrrjT1D^Vc1v><$)7KK_aOfV{X?@08Q=9_hAK@dq-sBvjm0H&|3RP1 z#1{K5Zv^Rx$+NlPm9Gal&>iCIGldzESY1n(7_Q+o83qyb*Gh5POasc(%^&d%JMUye zBuA%~z6{D-Ay71ujGws)_8cV1z7&mJ_x#OPT||E5p_2)Xue2yKY;$T;@ffBy^9+=^U!Ec(N!uZ}z7L#U=9r}zCf+!43lZLU$x(oykKx1XUeUa0h6204Ti3@%#aF z)zhHdJN-m@`ner(4n|Q=d6#x^w4LK>^CpSX`O+Mg##wSa8Q1M}bjfi61gl*_6>S(8 zPEkBpiRVu83WBKCYav^k!_mAlDm)E!cXWFMGNqcEjg^koR!T;0--Ep|Yd%3Emi#C8 zDAIB+(4d2!dzUh5{9S+>P>Ire@44gd7VfNca~ap%z_ZvmK5>mLn)$%;@W54@6T4hM z%@lfmHn{(=gKd6~d>QBRigBWrQvCs#fSSPa1P_u<=vz1#P97ggI}D6NbsC(a9A!7u zrChDBTha=%%2JgpJ2;QYb2zjzHUViX+6Rv@E~_~ix4nMlt0(Tj!KM4rOSeonPivu2$Yq}UlI0ofpeH-`0d`hwcH}r#-TFLl;ov+~>G~|& zgh3nj;C(MFC~S6=ds5wBi+H*itdpU=Msp-kxfDqne3t$T`v)7@z+euQ-J3l}BC$eh z2g^Pb?Qf5#9MLL(Hj(UJw0zaO)KtU&Z_Kn5H|SS zihVZA-ZeP2Hbjzzic+;E@@nxXy9w99+G_J)6MwSrLk3Bq%vGg<&JMBz8Zod38+X|U zSa-?#y~ocxbkQKXSDKc4<8j`(@8{O%+@+(d`&yn3b3Bm(D&=+LX`ptehx{E6H@1E$ z8(i`0rP)fA7?uM!AME_rJ)k2)>D;7fpjfV;&i~Kj&=C6?jcQ3^Jo(mlODa3R1c=v$ zX`5_c1WGY*2?;$SmfP60b`9mG=*wz(`&Q4PZ07v3H?Tp|?uX{L{Dv(d#?`6zv|EA+7C}z3IIYim*{Jtr$%vL;mva4*i)mSxh zeq{ewZZl((74c2tKbPvPn>_e@1)l$LHQV-`3-&UGy z;T|ns2Xi3bo9ox@n}DL^44D_4TwawU;EAg@`9`wk+E4k|x)l7L?GtWKU{8?nrn>R+ zI2JOF?~GY>8NjkM=Sn_2$ZfOW+kEDXkniARH|5oiIsIkQd-ZfV^RsE9PWU(}`mfI^9DN z33c#MMr%ZA86z~bUB&-hnrD`B@*>(soLKEB+_KZ;iD;#~pKkLj<-$6|+4ZqEzPJ7^ z;=p;H{hJMj_4g>XiRd{tj4ve9Ch<^n?j2snH2V_0HQAq&wvBwe#{Rh7O!`l5;s}GS zY#PmQhIwE3x6wNff)qDGNIB!@bY=c|+Yte|#o%vK;BTwl2wj`TV;I(c)ZEE>i65AZ z|EO-UrNvl$r8v3%>}sc{Q*F}1p>wGCC>5BP<$(2NeIkv>($STrEx%cRA-LYkHMY#D zar|Ln^BliO)+amHE$O4`b#UyQHHA@=`)lik%Uc(Aj&w>$UPYE(wBjPvXz2>akYbYe@oX(a;@C;TLwZ${h3WCRoi^Bk}NWm*@(z3X*{| zvzU>bmcu|N%As!$4J|)I$mO!+2OZ5|T{fMm>>=TV9e;XNEl!3C@37hqg&1;5Kd^P{ z$_^9;RiJe0P7y0$@P2m^A$;F`iDqz4vw!_4>pXtH|NR7oultumZc_*QGp=FgIZ`{@ z=Pr(ckr8o2xGPYLcqF;{>3psuyN=t_q-_>sud*A+ckHcg^_I{$PjFP%7UbX z|2!pmF2zqHN#*+hM)PsXQpm?;gNl;csNho)+HQ8Pqj{PGKeEs1@zYwBA*4)XZyPvm z!(8OkeP^klWdl-r*Y8%2h&1k>GF`?|*egsCY?Wa92uE{pec?myySpu={q3_2Ap$c3SnX%SU)6LTnd1>&;AB z!+ZOJsq^y!4YNuVQRyRFOR;+~KZMuFv?Y&)mni&`katU_mOevXOpdvgDiuUJR*vm{ z{mgg>1k~qoZEU;Hqmw@5ECb2S&z_f^+Aq1M_k5gn)`yxGq?JkAed1;+3g<(%Bo>}W z#Gnwa;SDPn-9!^qn5U)a?lRurK};*Jd9Mv~{lpVCYdctL_ethsa?kANly18acL$$y zzDFhoR(aOZI?DEePBW>R9zw)9rS@&K_u6*stJE<5rlD9w0;s)_W)<+@gU$sF?L#Y3 zfw3O02N(8}E52BWRVwI*ERjtw(_GGu*cHpia!u1dS}#LZQFS)*?{=?e^zo>C=Tag| z>b0lB5uPlC-$b%!hFYh3V0P2pWYTdiKZs;;rLAe{ylANynpsGoQxiLZ9$)R(SmIac zsqTL@Y>|o*-bO5c4gvbRrK&&JYP)`STr*h}nM1sJ46F(LdGl)kNp2r|C03JM<^1%9 zUE4D|CFR-45K@kLfWM&b!zFLQ90j5bimUmw1dS5#`p-N+uOiPB=T6(kF0w~D_m(#j)A_Fj~f7;zEXx9N!5E7va8lJ zfj%WZmHcNV#*7x*4L&Lxri$sx_X%gpriQ3wdx;cd73-wQ1I}@RqRtkTxu54|3hC&& z!v?G+u3uiw9&-^mR`%qiabuJT=ilg)xi!dL_YPg09U(gQ)Vn$C8SAJ)FCtb+4DL?8 z{n?2ZybK0ErCDE0xYX`Y{X_?RmaAqH$4Ain%r+XU+uUu~c>>Y$C#Csk*jnQu_m{`a z$~2e4-HGWjv)>a&QcX&&Jli*|qJ1CBqkc}t_wT-B@(Om{ug=f_45%y*D;cf@j6=h+ z0IS!=T$*}j(ZA~^$s+~j0;;61P5W1-r;;vdVi?}qhYiQ)Dn91{yKu8$H9#Gd|!bylS*%mKK z>sqKdQ^wQS6O(r?_K+on^D&6y4C6>!)kA2L#HA;fk}J3_R~@)0TJ5g1 zN1w04>urLNHThs!u)Y9iF|wu*9Rr>cupPORE)_?G2K#RK?CZCFV6}a!dY8@uU@pQ3 z520lHwGdJJn~pk!H`H{4e81Q>#&Y#3^;Qc*zZXy7hqm0xny1r&=|8sC@P&pH@#iW_ zko`}1ZvHV=wMCuCZ0ytnn?KenZ>8u(i!SPPFdMn-ZU-0cOUFxK9g3E8nik`(=hY-AW5J?otFdcHSyTpPsT`=|>QwbucFqIqhhUg;N@= zIo&iS>9TuERReKxEn*r5WZc-LU9h1bllY%z_ZpKlb_ZXxluL~;P?L18?6I``l}n0G zcx^1#ohI7FPN_m&^GO`biA?x5rap ztySLo>3#Zwa)5TlIn^HZ1ky$ z(C%&JTPqul`u5&5+vw|Mja@1|fnlDJgiUmdwyqo@$r}o9{8`c%-Xm%K9}apSDikVf zC>{ugY1^1Dlf{EuX>JX+opX0s(u8w+KxL1?*$^}Go#L&nwDSq)atc}mv-L{1 z0TIMaUUL(=C>Co39y}xsE13P9G}34#!vOYg=^rNltiQ;6;cJ0Bi=CzgbJdb$!*^ASTpy-!Aq9PEFdK09yo4@Rv0!nFL<3WvG zp!T_N6)pqwaSSInyu3t_-wqewn>3)^l7(3PC3|?zHe$c|Q`aI*|Am70+wJqwok0fD zU7Hg%qDp)>pr!Bu^S!tGHzvf*kjq@ zSUdQrGZ0~EzXCtzpzXrCR?Yqu8+W}R1noWtp0e%MGEH@ne4YN~oyK<9Iiv@bQDr0Y zL6rY8!YlPK${gBc-8$F8CLx*`#tJLICSF_D=iQj@3!u4d*F~ar(`Rf9-vBW7X%$aX z7X)b;WN}zp`cMZCrV15W1`9548XB;LvI~1!t08Vqx;e*^JH8Clb}!jV@X0mariT)R zT`yf6RQ*Zp(55+G#uA7G<3>o^zB;l=kXL}pPn=_tU{#)a`b7hD<1``zoGf0uWQT0K z{G(O)6>*6(Q!{#)Vq28r9FXaZwbJ;)Fd#abAQ0-EEE}tQDCCkiZi*y0PHK2hU|HE* z#JIH2OGK4yk1)A3;%T%{JQu1KFetk5th!~L-z+y7+;eZve~*4<5amywA!E;xV7!mbE&bHOZPxAkXxU;g_096di0CgYd6FM3M+xxy zb-nNtn+i{LN}KBqTA;v#Nco)v$uV;HOB(AtaX8^#C6iTrZ9a-*Z33L#i(hehE>L?c zl;r25-V>_E{^ycI{yYOJmHx14R+hhEDkf+t13FfZm3ELyepp2}r;l%vzQ+t1zdp{q z;@|UPb3Yk`BxSlWsI}L&rfE1Wl;o^+1nz|kh$p~1O)>|!08y~iBYBTwf!xwcYEJW6 z*}kk(E`x@>^8*_jwdAdew4=R>jXWzAzS6_`eDLeGi?ry&TX^2BIKgfj|2L>L+1&+> z63+xG!90p5f=WHrMF$xs(yJcgFPH;vy;$4@-480-i`DHV%h)h{w%Yy0FQD|{bq@9| zAMEIk-pqD~1B0zSIzRtC-jTKNyyqS>%!V|5t?HHu(|YxA#9cgfjJX)WZRYs|voJF1 zEGx?8xzgBZhhM=B%~9~?>XBTPopb>!75}`+qf5@%4BcqJq)NrMR2U{O!>qM#7VBU7 zNs^a8jicWv`%yp->}?vXu4V~sI2IXERG0Mn@*w*Gvs1?@lQG(ZBD>T2vr+%ScGuiuUUKsu1^f8Em^&YV!@PrL`aptb1 zDz!4w=3FS;`@*=9Gcj|?vv?4m_hL;dhHGOwT5`F2kM&K)Tp#5CU~R@s$jmI&sSfHx zx7A~y+MHC~Kq-dKJx18L-V&Wx;TiLpr`yn3(b>LkDRCfH`8btD;Llu%e}}hB;#9n` zjA%FTXz*h^?eK-_k5;hU>X6gVmAHi}Q&$BFhu3RGE!y;n4)O)nMW0q-(WD4H z%9boxmU431J+cU8wT&{mdKdOpv;fa;r61%=wgJZust!LkLZ&Wvny1Hd2u{-X zPjR_*${jSW>Eaq%iY$(#AF#UA+0p{5o0{B3-QWwZ3zhm>SZ%V4Nr;hLtiI%X%Wg%O znaYs#?!c(TX$Qh0Uh4N?=aYujPY6ciIC-P467Ruv*)~o6Ax&6GUEpcCBl$=U5jarg zCgP-IW1_KKY*ZOdJus#VC((ox`h**2!e7fW_x!zw^O+|=QIsj}wD)weFdc}1ZO&x2 zYm~f|6O6GqYBO(JGBH_vzC95XB>gkTJ(y#nK~IQ1!Baux{M*^(GV1PIj!QChf4lt1 zwWjzenk0wOHAY*KnmL3#^BVe~r{OWmD$F$7fdbd1++gQ_G5-lpp#G z7CUYi|8f8zy#&kC76l9@z9r(xqatDzCl0x1~6{s(2|#zFGyBHUyrxNCYVxgg&GDOv`sT)v3RgpY?Yqr z3`u+v40%q9U!-))P4XGs^uUU62()fleV$<0@?2M@yo;eUf6m#bb@Nh1XE>w%xiP2| z!b|n4lmmEIxQiTkj>85lmtE}<>2lMUJb14eki~n7+cb<{J1L`22cb=A3ZZ8A%ri?7v%bA-jV@&?-nj8{0~&FxT)#$+QLa) zQFZs3SAm2u55PL{;%b`0b3XOAuE|g+^;qsygS_hY$KoXXwm4s*oU^nOI|V5N;Zwz- zQVJa6TfM8FRiV2~+;#2ab1%L3PwBI@oc3r-7|PP0>$k9dpCmcC{_aClL8bS$ad%0> zXZ+Upp3yc5#wvE})f;B4UV*clk`?Knm2ENn7abVzn8XyR}8 zz5dfL>at5rt*xCEsj&v~s`dAQeud?j|J5;P4f+X^koA1T#_uY>4c+^SE#-f=tu(>! zR~S<3MPCjh2~Op^R7v-mmE6N!>h0=sjp5{-O!}R0$MTA4FGbP8e&ICLXftev_g*^3 z7#np|03bCIG9x)%x0|91HWYy@jdvIBI38*Ig38cmLvIPhy%tJ;4|`R6Z-`AA=}o%R z6dGY)4#HlcbhX_ru8Ym*!$M?2hqHg4)A8^nc__EULKthzNJ17`VyVl+2C;DMEJm18 zGo;-K^x>(X$*HVe6+M&cg7*%tN1tTJI-!>ZFx1Rlmi5v}dVZSZ#HTlnMyM0bDsvY> z0W-Xe43glN-BF(5rUNbwHu!e1y!R{{I7z~SXca*2elOPVSDi6s(@(8fT9CGxR%@j5 z4YWr*NN{`NMAOLA6({41TaGz-31=q~iiW+`e`IZxSEW+(vDPUcLz~lm%Z$jdtyCNU z4wRIQ`dh1bdJmm=z80c{GEBH4p+(%why6@68;_;%+Df_!rYKwJDq^L~=km7o7+S)A zIdcD^;rfNEyx$aI?7MO00<{J}2a_pR($pU=ORGJG&L7)b`!Q?j{}x(#4Zd7SjpfRcissda+Q3-{Ln^Vy}6b<{ul zlI|K*YFJD#u7xw@_MjB82nHX(U}sfLp~i+myd)HcA4_($k2X2JR#o|f!DXEJCg3R! zH+Ql+A#jv!JcpAzS&f0t2Ebt_sPzQ~E2z>ZtAgbEYlHbXn@q+Xw0qu~hJDVkl2cjA z5qY6Yf&3~X6v>)F8!l>L`A%S%!1eHS($)1!$OOt$I>;jXP>{sKE$}+>P0~U^n7F@4 zFkYDZ=F12S1=Gk!TKDI2_F*FBE7DEXDj<`n+JRl_9ipIeH8M`HIN;aFnroKcLQ`}k zF8#=zM>}EaVa};?d0>RpV$yJAd1Ap-J*{K6NdGf_{(={SWxxMAM=;EPW$T$cnKYhv zBRRSM>EZhmUj`3z3fl(jA*g0!x6)^x=T3dDer12@kW(MUg}OG=RbbfU|C$xXFwncV;#iqinjmxMs!D8qs#4dT1! zPzblwOuFec0X~!W@a?1%))Gu1+F|TAgzc(1QO=yJeKPo{09Y-jh*ZnRow4>$8kQVh zk2d+%`1Izm-pYw$vvFY3GWb{;S=)@<2AK^ATfV0yxV%5{Os;3^oiFHF^-u)5P_Xc! z3^`?{w^vxwyA-pjgZk>F3U;G>Of)8>rrNx>VNnv(Mm#dFV@YbT`6=l;1PkU{@wEmI zo;%U%+XW^?0D_@p8vH>n+1=NaQDw;%#)xTt!e0K)^C|z$eW|W3iXJ@kD}HmXf;g+h zwtHiz+E#|zTUWpb_C?u&K+#=(N3MUAo|3Kj1ZJ6wbem*N6s`5!?+AI>hID!Plbywd z<@rnaYZ(0M?#`<3B!$L3U#QfEWmEoDF+aMcF4|{$_UC0A0qo7=sfr6%8p%`g0M?4gSUb6Ob$TM$ zjc*(OP`u7x!jnl_53Wv4Ke)+k0!z+PWdo8)Ol9gZJxE1AWJICBm`N+->(Y*h&JGP%@*%_QH2x3OQ9n+vL%d z%<4tjYl8PjTHD#NE^29JLdXKX`B}xIHPoB87qsSnBw-v4t=KR1>+6I2&KX&6tF8Uq zsv;+-+`PTn4kl(#9k9wO1KxHP5#CTJ)^>$j-{~PvC1Ygzz2uifO zLHVa^CIw0jZx(sWyN}_+6kZna^8P|eOI$Ch@Xo%FqGPlN$viRVG6`xR_=dWd-jKnD zz6&3T+N*RkpUT@=!&YdPy_hhKsgxqMp8g)vuA^Oy51&@H7Veit{F&{~Zd*Hh z!qqYFOc%O$H-t1F02&&!rd%5YD`{oqA2H4%Jot~*a7CmsZ`(e#Ek$(rke}IwcEC@! zw^*HOd?nFBPc3aDL=R?Y$y&_GMi?y|8N>kLDAGIn>*5u-r~Iaa;>`V1-}=#D1Pydrgf5*<*dc<(u|BOU z!(9)3pWgy0lFnaA%kz_ju&;!ZYef6>dy%Xc~G%FPe18{Z~W9DK(Xmf#%0FQBO)LA&d$2 zSLFFx04{Wkh?rV)Ik|Leo*`MFv!y+;!dM7h)UJ$#L7x0nb}q0?GS1;X3tds#jO91o zcFMx6y2~0Id6YpMEtaMXyYZfJ3Bgc^p5U325wZEB(E#48LGRPz^1_ zHb_o&I8;}-T^XiBTg-E=vGl7r*8C)}vBowlLP?rH{Lo~t$QM}sQAR(Icvnr`GDBa1 zN@mlBVB&WotjGG&9=pqe`aeRO=a_*py(U>hoq647;xa4X(f8yk06f%%>@&qor<_lOWl3>qt|iNdO!8xr#3Qc@ z-GkNhM<;~OZAOntfRvf|9#`_ms;+n^tXWmpJNF(N$f>W59Or?Ug&W7yVqo+5+fnIP z{KR3%q1iap#Ys*eleojB(KeUn7l|+|+9X#V@o)~IUh*)dF0ImflBOH`{e=O$w=j8< zuy$UYw}wmfw;pxdU^)8i9UC=04Y>r=1XAhyw~fC8u>}wQ6{7DY9~0b_pIK7HaiVXM6JOvR+EU^C7b?0irS3l<^HhQdg#RiZ z_G6?!Mnke30Y#te>%<4H(K$T)-My*^G5vV*#;j6Js}8N>TAnmzz6i!+?y}BgpxE{4MqfH-kE4_E>}AmR#3v_;$Z+(Jh^{ z3g52xr1*y_Yj;VRv$z}cc7J)y!$%QNOM;t zp4o)CgKuM<(}JUHGG1t?O5do#Kl%JN%uyI&<{)gg+;SXs8q|3-P@!1IMj;Ceba2JN zoJ*b6oAp$%LvrzHZ*E#@MsDJ@G}AZNlAE5qNUR^j3i&qdA9(H6G~Tw&QA{f%BA>eH zEPU;y*ZyqU9uYC7?=k?|ox{E3?H(>S&i~^(`ZoLRe^?gCktF_?I0w*9z(EEMPNUe< z=x;eZjQ`dJuOV~0EC*1P=QsMI1H9A$I2FJ@odzWiTQc^AA8 zdWZOamx=z{L}gfXSuWLo3>?g*!6ht?-pp10Up?eq9vLoaSu{BB<-_dqJ2=k&0A~W& AmH+?% delta 38468 zcma&NWmuG9*EVV^0wSn%r$|Xl_s}3A9RfoRA>G_6N=VMoEiiORRVI*Q&(ax$e7z|5o|#z2~=X-FkGZPihj554j8eROm?E zCIEjmuZ3`YBoVi6iK6jha&PkhzD{$HiQV2df)A~M(Lbll_jA))E4Y3#esqxYZt0Hk zVz<1r-SHZmY!k)q%P17O_`dbL;8?pt(Idv*q^U54;9f6>fIfTaE1eS466MRy%eIB< zm9&D4!Dd6JwU$F2%xJ>Dro5~4qPB;aMFam`FXLnHrHJvoOe|SZHkDHr5a4%QwxT+| zSJ7BlG@xH_orlRQ8aVGe7j;SadTKKNC-BwY&nBHeroMuh#lTavZ$T+m^|FbZf1|*~ z|DE&&M#MeHsOcbnA+Ax=&adn$J#*~)uD`pnX&kM4{G#T>3(N^qPwb+_UMP0#81j$I z_xh`ATStXgnv?dXOBBr*Ko!5XtLRByu;|Z}&35el8G*%|zV#3IEP~fJ2qZa;?70=B zAL+U$cw+YaJ&*T&Lz*J@p$E@3qgXJbuTT7@J^gc#=hP*hWdX9kY}E_?6lA!BoZ~yU zRoER~s-LONTawvsN(e6AFS*e0Km8uCD3|V?<73l(KH*EurfQlA{NX)JFN6hRfwEO* z|LvB>UGaUjpQB~Xreig@;}1<&3d)=pzcvcj?@CdfpTC#z%$Mj0!Ss6$anRw1wJ=7(A6+xSCxV z3Y6WoG3+B)mev+Pu=4ThC7s8-q5A}_)WIp-YSLPde^;NksU><1yV(=??#L>+vW<^^|r?tXSUqik_%daHfhB+S;*B_te~#?YSH4s zSgVf`jV5WKHTBA<3c9r{Kp121jb*a>#*O6HK1hYmwJ5-XR}2SHv+^lXLo@T*1(V|# z+tbOzZoZXPo=G(6UrwGoJGH(Z=kn)6EXQg=?#=DzXH-2I*`1?yh=l}`z1ZPTPc zU$yueE3g-$uf68Ih9ZRYnyfD9doE1aB4*ZgE}I2iB4FN|7WRLu+WL=-J=Pcgg&b>N z9s|P68KOMnme;kzPXk9x?Ymmq>;sN?(+gY&>3Jm=e4o=WF;ld(MtN3;c{^qqaGUmS zPV@?|sAB5$OIn>g<{Vl3d6ao|?q?s-St0Ww7ou6WwTq_7wfCPKH}NryI>=i0^5*W0 zp2ZKXfB6EvFe-wVSzBE;kZG+C;Z3Z?90A{i*5)%C?MlRVXtZR->7ziUl(k%I=t|6n z=NMRHAKC746>KC<@%O!!Yuo)4mzU^^ZRnOA_>_8gR_vjPV@umo3s(iv0lT#?X2jF_ zXWj;EuU{`gu43E=btUM0En3%RVwvMdNVZp`a9CD>7%&I?|6CRigp4dk6?(Qk1fo!a z)>dLW3=3ZRG^4t1{dJ~ZrgfDLYNoh-?cW3pf5+*^Tntc_)!py0v2ln#iPB4oEUg`5 zI}Pi{1fP%8s=rxeOXA|T%b;lKVd`NuZOT=Y*C~)s++!{bdUk}DIm@JA9RyHCN9gh7 zbfb7n%NNAjv2|sUeW}c8#{G4;K)+|IS&l*>@L~jAC|~x}8SYj1Y!G#4^GNZH4i3vT zg~CXMoN)IrBu_U)hwO)LRiL0MfDqDX=FiD1c8)l5+|<{)7|_BUQETOns2(+HYI?s2&>h#Sh?Uxq zAgNDh^IOJvT|fyr={)3y6iC%&oE(P?FV48?vT=pq^oOw}vvkeVL4VcvK3fR?bvQJe zCd3^eFJ`EKG58&f!?}w^EZo=b_!tj5I|-Z?SZjZ$MpDmsDTufqKbqvj9f>@57&Dnk z63+4zlo9&lR;?A!d&UgJ*7~=?_4a*vb$jx6h!$f@3`hMK(rsE1KPcP$WtUFE*OOm| zJGr=q3p$I;tTA-^|BdQ6Sq=NgxCo7~UQJ$`nZ<{xiI=^1iBuh^DAA90nsZ30bx~dH zrLdiH!{zH01SJWeQc|BO&5EyYG3z1gG2P2O&0JrSno(~?2{eZm1^Z}ty)Lbn5fkyLobANi8ap)cDRMRWwEH1U-C*S zwYV*AR4Tw7{BTai-+Wume}x>K*Wi&?!*W4)n<_9@>W)iW#Ei0J>Ax2OPM55uO*K*r zs~K-+PR`beMhwhMlW8X*6)hAa3LD}O>Tw32!U^6Ko38IM)F(Lba2VUs09{CK*hJhM zOnG~7?9&c}&k8M6vf|;EjwzIN#Z9vT2F2vk%>eCzOb($FFXp|Mis=<--cp+vp>x0?w9HD3p-En9 z+-{2REc``p2+a)X^n)TMwO7vYQ1r%`bsg@6rcIAPY8py3`rF1Vyo`oRhVk(^D0gUiCW~*@2!cid$W&ysB!WmDw2b~Z=ahXyH1u*^Z^8|i(yneeOW2SCs&;h~Pe8z1d8AY8ukB_OU5y*CtN+<@ zE%9MtjCxYC<#5j(Zb6yxuDcdFn?B0f{-LEzNe??rc)K*t7~wMrxTNT~&p#Hp7&1;6 zLO-J9y5pFjLkfQuzx%4BKI<7}q)zMC+{{G`MK9&@x1*Fl(T5=dzo|o2D9wIaDI)8i zj6cwik(9QbWr=or%F0?&-Vq}Kd}TjaTM@SM42# z?@f^s$CE{d0Mm0ZenUj>MzySClrW@e3QYYqO%!K?3M^)ur6ZwiSZR^Xju)GC32H%I z*}uf;1e96cRTl~$E7A;}H7wE&8m@i6<1 zaGJ$T2f;sl^#awm6j?zdYR>HD9qYMaklA+p_Q@ew)YJLEaqHVf>XR{|Fy5!bL!)QN zJXB6@w&RxwtLRh$0N_tjKZU(!%o1Ip2+eQss8f^;&vQ=tVl@0(L3+1N!&ea%hqu^| zc>BVk1YeWo@(Y;rC42^Fa)wp7xOwKK1rzdLXTp7hMQ1UE4kx8=e)0U`G9IVNeV;(K zE&huaciWdi+)>=1A&}kH1wvDmxk~!IQ~Of%EEyW!)+EqOolC8mTK}JQ(!_e|q}P1r5cf_)Q8-2H7?BEx1y?Vp)(p`@&sB zqrAg!%uV5>l+&A5cveXaQPa0LIN>77UXP-+F{C*PuR!-#WZJ^>Pw+r|WAl*~QS7`i zmG);pAooP|Q5|yQpc=ng;*I(U-`09eS8(Gxh1gq~>WC9XTAu03PqncyqkKL5PU36oq#O)n;9iywQG@$e3EP$7R!8 zj36uHO^gxZUlJT?cur;OrB#N={L2C!&a0o;Z8~jJTRe+cg^GD; zVXL0eq0ze`vusm3zeDu!)D1`3GfvcK`0k7%b*VyI|0=hO>|Mv6I|x&lu;nI#3-s;v zFkIAy!RBh4uq>$@EUuH(e3-P|YN==tG2bhX7ede#-Q0yM`{|K-UwRXk(4{D;C^zqp(&|sO^Gl%tBC=22dx+c5@ymy6&in(`8Tw6Fa|AYtXdr4S4Q>)A^!7Q2Etx zaZ)L~=aEGMuEP*}!C0PdO(wnqcV@Nm_r$27?1hz$G`pNVD$9hz<34&~G(od8d!rYu zUy{9&_gif?`?%oUr^+M}RO%jO89zSr$l;;&(kccD4rBCAqO=&}*MOO{4rz0tClQ&( z9m|n(vnuSZ3}`04O7&19fYS|B4TKnlBGdIR>r`;u`$>{g;P%_i~bPP4iceBnI25177cpXeToG}M}QC{?sO)%xm>iaR?uG<7># z$p3554`M6zt(BTU)`20kxkFj$3ZMHn(f6cAF?C=j#vaNG9NRv@cl%;NHIyjMEbd~w zszaPsU8a;odOl7nP#_xrT+=X}F!E>u%^2<`AQ7kjQC~=`XoMnqE*RTU#zT8(YYJqe7+znxJ?_p(qO~b7gdF^*%%-bmtmcx*OWkWBZ@bbEO^K-z$QKYoxah(9_YHB)P z%{BqB%tksbAVqhqF(nB?|8BQ%?`@mTB{=6GOXcg5&^YAbo=yxfr^(-e1=x9clSezB zSW;;PG79oeulHMC@F+ggD0aT;nEIUi_up3G;;7LH&3vVXiErNOKJ@`AuRUR0uJ@#{ zb+|ci<6V1N_^t8dD?ecy?)3h$MG1StqpNUW11lUVcKtqmYvE|+dpLzntV9vR&$9%0 zV;Za2stS5l=eT1s_Y#5K%I?S}8TwGob>=#Ee7ogLY8=bH|84J`Si9g3H^x0;d}kHA zGrG4${%~PwdJjKhLgGQQ&ft*DM%ZL<(AcuN~H6Zsj&dE03bW zjPdR7)Sf+UyCiBtQA2RqChNXM{`>^Q`$uxSeGoJbL2P)6~|VnG-)pQWwRnD5zi9H1Ztx><*826WupHA?RR; z)^=`{qy(k;$zkTK>40Wr-=DJR+Y{Fjt{zmFERI!)SO*^ZWZ7gi zK0NO4QKXi5goO`9eR2K0Kk&(6pCR))r0I&`h@!$wwT|n|>Qb)9#wrhB|D$z+hi0=p zXRJyQ>Ss#6fsmnHf4;FbcD*N9{FO zv0#&AmW4p@Q|S7mt=u+|Nl?}pt2qaXk#G()jerE3caSKsa%WIfUf{MEOV7(4qCpV; z9g6bG^(wpd-H9iyXko`2M3s`Vsjm|yF%%*6Y^|6FtZw(06Zk#>|2KwHP|$1bwTSg= z9p)A{%^MYPgDPMJRgjXC=t4ft+MD!zBKIuXV$W%HmfQt=g}J3P~y} zj3K?Dc7&MDZ*6RI}h9OnM3*ss&WNtq$-oK$tJTi6oVN|Iw` z>u6MYij(b+cB8(f%BuA@Y(dx0{(xtYP?6zM>ST6Y_s_JMXo{v|=IJV$RuV#&9UB%y zW&n-FA^RGRc=e~9``d;Ekl%))>UWlR3OV)U02gBiIEsAF(o;2(vQk_XKB0J7wx`)M11C zhG|Vx<*fjsE;F3$`ck(Z9%lV|N#aFXA`N6)$1{nqc)-D0p)j5Wi2gj9duE^l6WlMU zriA=f`o{&i{}Cm%J@^!!$u{s|_pz81JHr{D$H)WQz-p-(S_ekuq-rx63=;o5`2%mY zh8$Rc+s23({f;yy77x}nOJqHLail7EQHv>Z7&S41*}TV%Mn68Pgp`qzXC-}d$Sp4o zxrByh0~UI3A7B({?4#AK7|DrL4MT&P8o%Vk*dTILxHv&;m>AVE0yY%NLaIBlOzL-B zK7Fh`w-k#{5H83wz)xXL6MB;qHIYG^lGu^WuF~8?M1*-iu=U9s`%?Z?t4Rs=t7aG2u5il@D4;b zcjyQjQ=~uPU$#;}7OuJV5U~W-|Datmk>c!{NhmQUcIq2yXa3us{EMVFIL+eDLi8gn znU}>XN2XTjS7h)+9J{Fu^q9y5pmT~AMErp)e9zNqb~}Jn#-b?%kZKF%WtRdihYXix z3S^&RceH|Regogt8{3IHF|l&bwWes+{^>mH)V6(ZA@f`s*L6jiNg(n`D`C&m7TZ#v zpQOXP;g_@;NOXj(6fUIMjYtkm&CkJOv&zdh1uTD}%9zqB-Oz-bA3A{hQ2!FQJyRGgX+yJ)f~PfdEHn1!1AjIj15@Gad@745$8wmvFgiz!$EaT?x>{vC~UJ`kss z78D|Ha9VbIlTAOszsv>+X-u^DLr(mwz1tf!0Q=)h6gb@J?vmaSG6*ZTnUl^#x7(~f zb$KN8g;Gj8GuXG24Fr|+M0rl}2gpNl6aTU(zh;Iqf(5ub zBPo$e@*JS>I-CG>sn3e>2TPCE{9prNgPdi3FKL0r<%_a+Nwbr4EYxC$CWS)#UxiEe z#)8MXgW_6`cVBx3C>(Lja&9sCOmVHvgvFBayktdIv=uI*YcA<^9ygTW(A3t?p&f%Q z>y{hWS$h+!*Y1?0$yjkB42F}ZGx>(;^Md+v%K=sn!qBR}M{39Z8<)>s&yZ(8_hMPE z5dLqmD3W+}6C|IefiG)>M{c>BrUB?hw#e!=UozHy{LeOa@;&rE;K>PJ*6w5)zhX5l z@|>50u$c&o#MVM40>7>^Jg|MB6m$6=@||!0kRq_(&)xpPylL~bl~qFGCoGFc4IOI) z5F7tWP?Oxjc7H_C7Olyg10PA=vqbUfE zkMdO30YYAn#J>-PZ3mJ&qT(}oz{Y7Kfiu* zlcry8`o9*R-M?bWzF7Yv%V(mlG)cZti9h@o=wqW52DREiS3lloo8vX|sk?cBHT!N2@2Ko=xm*ZW9eBZZ9Rh0X9)?w=?|2z*O0rpn4L0b z5KG8g;eq2NtI1ds`z|qxB1MK~wtz1%NLTIm5{0viGPkYiy5z1?)hCcmnLdSc``kZv zwS$hy*6h8J0!Q3DZrnO3ytZE^+A`{Ujv`$q+PoJ>`?iIVD1O%sKv{H8IKi)y9f`|z z?buQcciCkVIsF4T>^1Xn@th9F@Uy20j6hjw1|Z})Z8Y!p0tl#wb-MCG4F7Fx`gvyb zy)d@Y71HMzazG2hwAizg(kf%s=Svd@brqFo#fP^cQH`?ve_)e*(;KO*yy|-&YPx*f zX(kGu)I)&eEZws52@aF*4C*ejO7;ts1`J?P62l~j0P8Pi6JhjC@ks$bwotJ22_6uY zZls&P>2nJ1=PM_mazVgTH>vWzh)^9r)OPE)ZBv)Lb$Ig$l^4mz%Q`X}s|UDIZzKL4 zD~M7S1Zjgsxu)vxIO)RSoESUs5ifMEb5P(1-*A)))y)I{(&1dTiYjx1X!M5a(FS;b zH8ZL3KJ0h<@v3!Snu7Gw>xqM^R{soCo3iK-@>j{QXit$JnGBX`cFcNowBz^wuVbhB zh*!y7ZjLpoYPp-HY;?o9xJEz)NZa#N^dRRc_}|mq|DN8ONETYoUr!&9osv_DJff#L zNoFeHUrh#78XA7_pP#1ovTa?6uEbc==fqfppPdeVHi{Sg>?x`+H1BE>*go=x$osdn z>wUL2N(jmI+ex-c7Pk^Ki@EK=5T6=|xc?nDSD&sdT+gP_B-+~l_N9I4)nw9xiH&zq zavs&jIU_UjuJLGxxd~2`00>g1Z*Ur%`#VtY0oJh0|55iKH}PU76;#wC)S+jh#I_Qi zNQS=s8nSi^jYC^q$wT(m|9JH7^rPNlXBE+*wcp?GA8DaK3)4+U+w-hKDl=2!Q@xJ`Fb2kAZe#+gLALZXO!8?0xe_vIX-h!_t zq{ixt*e2s$1Df;Sz`*-#GKOV{R=!OKH zy!alekm2`rS}>zqn;iHL-Z=da-e8qIZ&Hc&_qK9lZFv^4;eh*$uIk4^GGfpE)>f~y z@vw}*$J=^eo=ZOlVGS#kXbXlU98su@C6H%Wlx`n#I>r~umKt~y)px}h(T;#P8*j#1 zs{DEcatg0orI1!t$Rxq!NHA*7&$LO|z%4Z$RxT>hu*XENO-fVBX@9j)U ztyLuLpCJ-gnB3~AXMlS&T$&FH81x-?-*zwAT1fZMS|XK_VnOh+lN(OoI9)LUdErn1 zax(y!vj%CJ?ERiNwZ9(SW!p$|{O3HbJG{W6ML(cu>XoNMZ2LvMnIs7DdvO18(Ym>-GnBGQL8e}x}$xWX`>!nI|rNF zL>f0Kta@I3s{%Eq_+K`1JZe>Icgc5A&&5whe3?r?PPA7#Ez`ni?Kgd>x&G2Qq4}kv zIAaDBJYJkJy}BvdtEcwu3)UDf0Q4}TC^oDIZ-rL_$c3MQUt51Emo?7Nk*mV#$tA-j2M)~g?A3PClg zIMUTI_V!abW_B{42VWqrT*)t_Gx)H`hI}T$cIM*^b%$sPRwmHf03{HRT=eS@ctog! zfMhnIII7;ywuRqqM##WG9$oy`xI2~mZW3PRC$vfYU8u5&1%V0z2hFJO$oLG>yD_53 z3>HmEQ_;fc4yyioGM<7$p&F?JC<4qkNsM#8s7}4h%9`F~f{GwKZRHY`2MjHzJ#%^x zH_qiq+wL4JAMjZf)Fl)BMK=a->FL44vl~1A4aN_9R+z*093_gsC zD2|whjlr26&d2D;&uBKXcgrCa!j^lxFX4Nx!8LqhhugvK#V9^aaq*q?s-W zq7u%c!h4z^D)E%m{~2r-{x2dl&(p`0A&a4LZ<&0no@z?5W3A$VQReR*e=7r-^xuXX z!roI6e$wKsy)53dB4vGH+t+I1ILa&;jLK&p{9JMd;{E9rnET<1d`(Pf#uEHwrWou| zzCiWu0?_WbV+VxTQD4dyK&#}aCA(O&CVhH%9Wibfi8{Pp552Wm#QI2X%JMNMe2>3P zVb}C5AQILTB3YJB681$}+P1J|o;G&KM@E`16YK+ganbXRc{?{8nL;0HH=_meh@3Y( zqH^o$?$BdTE4QBPXL;uin?%x^?{eTgHM4arigU?;&i7UO*u<~Bg6Vf!@ zBI{-KwKu)pENDE@`_th=O>6&T!K?-XNwAdyq7<;5at>hZnir^?4vIxrDkbinnqxNKlgkx-JtM64rUd+U{(Gml*`EUI4}uK_WHHZ zT^|&j-C69}vt!=Cal4Q6209xsi???-BaY&k1wA+Vb)W>B945k>9Qr*-!4bp5mhhuB zVhz5U7%RfdDts5_hOc00<(Qx}GiitS0f5hco9utm(!Zo!e=j8zFzb#HKzse==hD_WzRdjxSE5De=L~1I3s{Y1@Q9 zb^qm73a>-EXnB1CjB{Q-6xZSb%BsifQkvX<)4T*k8O|Lr;rOjs!%6`+mEy7I>0ex( z9+VO`>KE{?*6USw{gvyJdC++IYer$q{|99Zi-pOH%X<68O$zeFtUbJ8&j>{lM?#}apQitc}c`*UYA!o z#wOq&-aQhSFojbWrvLJCH`oXl=7Dct=}G-hWnR$8xOh!IG?@ts!CtLF{;ntYbdzvvY2u9&-d!B9&Rw-Wf_pr&N$1DJW8AreZV!B@o(brXt; zIo0Eu<%O1`4tj*6PDJN6VcqqGbK6Q|Vg2>i7Wj5wJUbVwjR@L;Q_ci#CibeFU=dp~ zKVSNXklAMB)EM=j=o=n}Tk1Dv0A({%!8FC&Kbqe9n|)2Ia1%%0o}dE%l68%82Xp{B zC#T174HttzWA~*$i@u5R?*{z4`A(_UQzwa)3n|#HgRAl(1$*-2O^bmt)v5>oLjVef z2K)mFK!d@}We7!DC9xMJ$51g-Wq8oU1KUw@GNoDmU*6x2-FB|CYJlRxJ}swGbfaAp z*UD<&+uo-oVmg3&f26TYbxX$~{@(;Wz4bTDO%Ko0qh4&k+9aeJ2DsLH9XQD@Qd$9qc3I@U`2z1&W4WQ^hvQZ@zT-L zINIi{Q_cxIUjB4jgeUOiSJF>cV{FIcGjZ%+b5lCsl{YLTz(}D?t1ZQ%n`>n3Dy7Sl z5|P?fwXMiY0h@Re%xK=s&;w+`4R!rzWTUrRiqn1EW0BfSd@3i5ol3a(qGwcpZSr^A zlVSlaiyAW86Q78esc>PfYG93(u?E>&(lUkszA)BA8z^vq$J$jaK2WvZDUwOB3%pQd z2kVSyy?`Jozic2$LpMk8qQiiDpf09ht91iFbLaCu#q_C<#N%?I3aMqUqsE7yRNi9G zB#3%%3g70y9eHfaB;&X{^ev(oGczHrLszsNBcR_IIfs6AGW%U$r$8pkF7Qb46(}U8 zhslL)k@|kYnARk_CSi}>M}-byp5S!0X&d)uptoam8odfkm(<-nuQfaPT60u$)rICnCR=r{b7?Ev z;k!iCQvYDE-`Vs}i!ZO}dmeL-`@Ax2^j^A!Iearv&+p)|H`6=E)fa2bqd75vR%MQ= zS_Olr%eNbyYZY#3I7Ey6Pidso(2?kdJ5sOJ_LX4QwGVI8zVe>K>U`XXHvSy@%gPXF zNWOPP)Q0~8<73O!&2W@7jlW+qgPn(vNJ$wsAZ|g8r-H%LN7<&-<-#IPZjm=4lmK8%L z9C&97cCE{1$5J0T5RP&mtJ+jS!PNKOiGPinB3k9qypB@sXgQ(}H6u$>LGeD@sSG0K z_%hl}=z`zCtncWsMvlNoTeVdu-JiY`Feir2atu{7+RB?MrtJ}BlaD(_J`GMj7B_pQX96Wr3T>xpU*PG<*&DeIGcr$6 zw#Nbye|eenqP0$xQUcz4tg1=8J_pBD{8uMaW@s%mA>fM|>%6&{|AFj2CpQIWGvLiN z4qqH=O938lYvh&W_;94uR-;N^36o42)U-_L`VDXKrw@U0xG% zsEes$CKFW12Vpxi?R4l!@Q&45$0w}A0~wBN~EpAl5iWmyWB-|k<^5V&<$QMO+= zn1%qH7#!2NlQo#cnYIc_L4YHlvIiCOclr_P`l==PL@fKD52}lDEs>plqDC?WcTNt>pM&=%9=c&HmLD}$;d5)V?>q0Hh0RZbb^@K_z#AU=2akT}1~ zYIY!}J+u3_6V~^)?TrGtoE;F~PpMozaIZET$DWkPm*>DOy{sY6^JA*Hlh3d8gTw-PB5v8Tb?11~eyrmO=(0KI6k03`qU&`jQ& zjCAeQ2xtRb(B}<=LjJ9=H-q8KQGu{A=DYg4bTBxDzuAC7c)V|fEJW|G#~>!+5I+{V z7XfX}L^(HgLHen!eCxyWTsO04`riIe)gHF$XPv=(d+v1SqNc|KYb>GmuU$JtIe*G4 zkKk`I%Q+$`0G|(?$}IDsqPTa!ByTa}P$yY+PutiN7PyU;1GASaTvifY1rs~Y8O9p# z-(xduV)r2gIG3V0FPa|wy)d93klib*{48f>=i=n9a)Exj-kV&dXM3=nFYomFK-u@~ ztMe{0{&FOjAd#wJ^kH8B1z#`yAHF;RRLTX22mo;)RYtIJ-;MaqB#(U@NZbJ7nQ5wG zp;E)XLQ3{cd+zvv%6DB=LMDli2~#~3w0HlK-^)6MOqU4``AB1Jxcr9%*$sq+7(rFx zix`te8wu7zP< zk7(9FHCKL^HEsLvqrL|j-brfnb2y@%Rl2VTq+EGFn06cf)G8&w)gf6_8(vXx zMKEEgQzU?EJK7q4>i#y58Im(b;r*d2tGls$7;3iNLD z9R&h;9KYjS6-VA%H-%sTk6gY;UZy0&aW>Nwomln+L94E`r10!0$;LVx_p?h6b*C#z zrvV8mW~qqQ&vGjfr=&FvT{9nCN=|H7+*^+SRG%#SYHeymAxaw|0nP|(b-uv~8_Pi$Ri@F-Y!l`qXsS^8iZWn5gKl#%y zvK2e*QarBZp!4O}kSZ}Jl){Cteemd|Q@6*WPm30$dsL+Bc$_zk^8l|%AKZ2dGo>Xd z+LrOmgh~WK4gu}7eU-~}NjsPRM@1Z#x^`jP8UZaiGFv=uO=T{|mv`3+0C`9%z3#PY z+G#l1b90EC z`p~cuSUk&P5&nJ874}eASN(EaYRqIZyolFNMCZD7lPhY!tddxDcqoQp;o_^EslQx;fXd!j&94=k zPcpk&De8a)EU?-ZjR(5DSF0|Msrk_pQ6J;1^n)5$vLEx`qF58Ug~Vn5hK|BKlg-_q zC$Cm{epV@8Vtf?DR25RjRstyCK47_1HKql*xq+hSO*KE5gQ)FD zu1ASlZcQ|oSZ{Fr6q+HZ4O~CoSG+Ry@!4CfnLKuLY-eiz?PtW0$|sfXs|gSkuz+Ju zZ=;M2)lC31O#)D&E1rWa1}t+_*Ee5gSKX|gq~Fj7>q0!}jRq}r`u+-;N%F=NzqVg-V#fuMMuDr;F6z8HR5d44-_!goUfKp% z%ghJ`3DECMIexOT%>z<%Cmwjmcz}gY9u<}E$rFpfbhzUoVc{T(%tL@S}rQW9S5RQQcNh0yK+(Fq*}1hLPrUHe}VJ>3(t@_W9)W z46T!kI{2*2J=@)J{;FZLw*D~u(2@Hs)vmw1rqx*u9ps|? zTRvLP{Q^=ltxMn!)ldZqt_k-Jo_goE#M4>{hs3K<_rwc$gS#s91k;Y6axDT1F-{;d znFcpil~hCORr50PYAs8IGSZ;nLw6XFpr_#|u3n}1%=q%kG|9`|Z2RDZcF&M=3v0=U`CIf85*d(& zMHL%wu|H%}u)MVuO=oNi-=Qgo+r+Cc-qc;YaexuKpULw z>1PHXRRFdm+i&M*BRXW33))&7o)H|9$||#*fz`$8O(go=MB=KJi7qU#4=r|6G!7DA zLXP@%6VpfV*;(yc+-&#!7RCz%=evj_*d=Mi#)GtSD)7x-5I){a!Mn}& z4+v6~qB8bNmOjV)%BqwOx-F3hQ75O1sE&Zx@Pggs2=I=4(&R!9(suN0k>7~N9~As| zDZto6Jeb5Hk?!A%BMrfeU;i`bW$thDOG{5{qP#|7cl|4Q%JG1pt6wtRY0|*cp>B%b zGLbI3MALT_L>bee4K*oV3c`#)n*T_yXCer!_Fm6&e6X2S5sE0m=aA%i42ElX?YS+u zcOq3X!osO@gbpItTS@*-+s)F(9U(v+n+KZe^WfOtw$=gd6%(vkm!Lgt_^x$tL zQuStiPC@b}3EwHq%BhjUAaAl*f&3febYmDYfi%fcikIjQ!1UZ@FS^^nxTb$Ok(KZL z^kQz=mFW*p2fQc&s=dWPcXg;#u6G`7YZo za20*iSE*8}mq&zETwB5tI+@ruRJ4Xd%OUbLbCeu~94wj%Pdq}BBu_Dg>RM17J-Sz2 z!R|~^j;fq=2}3SK*L1t7+`WfBqBPGl zEj2TaaYD9|tn7Q$V_TJ>T&<0Yky4j|N6yT5+bP7{zYsq~K$7QXRLQITz^T1A;N!PU z>r5G_=26_CAq%q$j75{(#6#D8Xh4bEAvoq0aWm$HU!l=A#FKR!a|zI}*l?LevWgNE_;`>2ZOfMtk54Az(osH9*`9@Qt)8*^;2LI_HnR=)eg`Jm z-oSORo6td??6Z1DB@aI#gX=~TTSo}q5Jt>hK=CUQ!7K=F{)fmu^&Q^W*o|JsBhUAGhIycfJfb(7^?_ROq%zLGElw91*dks)FA2pVv zj((ppX|Tf7Jv^Q#!!Ok)zEe}xWo5505eG!py}yHV zBpefo?U1l+Ty)iqUsEJD7-gAK`H{oEws2q+FLGL*8Ny-#R|$@>(i1P2F!;&m;VOi z?Ho3F!I8$7z~L}LX}kNOC4z4qBEr|AWliW`4KgX{8z#l&zf21J4@jgC_p81;SH6yG z{s3FG)?@3}^(Bq`{pn@Wa>uL5udR);jN{-|u~AzOD>ryxB%gdTz|g-F^It6VA1E}? zXRN~D5Qs#(W>)te7>x)zX{Mj>j$g|LMw5xR?P|v2W)zBFX&#@}MYq}fKWx2qSXAr# z_Pw`(A|(ydEm8sk0y7{X9nuXdEiuwCFs!YVNVk9t(%s#nbaxCfBAr9m2s~@{_x;6t zJkP&y9QVwud#!t}&vl*WIXeQ)bA!CKpI7pMrh=%Ie`o?6a(|^%RG5}DzxHBe07L>H z9;c48e7e3omouvFCC9B0pB;CCkduVdtAA!6Yw`oFPab*9E>4R5dHNFbDAS9`)nWYP z-Po1V=_+eHGTn%=R{!3qE#<_tGJW^GqxiF2uY1<>FI2CM81RSM!Y6jsHu-H;Wa|nQ zTSZHJO0U}lj}8Y`0C->x-a-;61ZzQDu~gInk!QP;LrvZp;QtLQUSong*Cfrl6}M=a zmVu+`t-1-156C86C$|q$9$8ZTMy^rmEgO*!4IZBnpT#*y`2aeKQcJwtM-KDt23g&f zN?VU4Y%4oXwRFvv^~6z6B;MXV8Wmru{wG|@anHa{L*kfMd%e({Ic(Z{DrzIjK)mK> z^`GxPN4bo~?|Km~Si4yFxSW*g;m!xbU30)-RS#E2F?zo=x0l^_@_isgnal~&y_K<| zZk$5v`bnf6e@&(Lp5zCyPdo3#zL$Ezf!&%*&qS9qiS>fxriq$DgX7k)p3g2PqgKAH zWR=$~udOaU+k9}~Gtm6|j7=O!f0LsnwR!)9px9M8@DlW?`9?zYY#};rT+xZ9fN&pW zTTHD65YeE1>WQ}KI!UxC>mS)|>GL=t`4Vu=K&$@Qv}C>ZH|Qe~1brm$KpzQ_a(%(l ztWVqztQ(^F^Vi$HJuFk5Cs2glQR}7nUW;yunWS$8Eyl4pC1StyoRpZ3Hudu!-+!pb z!Nt#F=e4_3hg$WpzG8f<>FOMSCsT?H~bk>ORQP}pb+!OrnD3IdJM9>&E^flvqzg@_DD9&Jmlu zk3aR-ZNWm%^GfrhM#k&!1_X7-ph3?7A5BtL}ZWiBF3!!hVl+ zAKTL$W7&*$-@?#3+f%YLfOT;;SQitgwFiNv{%k@m0u`#5*C;Qs`flpkYH_>Z+ZvN! z>4uI}`!195%Rm;O^z56J_vUwhw)Bo=ZQDiD)!e(<#N2sMDZ*^Ju|a=On++E8V-a96 zZ>2zv>tY+APJ3-6wZ}PXxlQVFH*>>8lf;J&&WzbEU%aLGm^Q|)7&FyUR89#)@usftB^bj5F}zr_DJtIvVB))*G-|ufNF-jdcb#?uihiR z^0j4*bG;s|utE=(GBXU>z2KkBzz!}wgUJ5}Hmo?XN%06Y?&{%q_;Bb}3?Pouar(!6 zRA%w-Uhxqb{g3#r#GmvOf4c|z7h=fOr<9wdjn?azT_Q#1h>h~>RP|y+jk>|Nz767& zoE78aQn0f5ot=g&IM&6xHYBTmzKxrkC7Qxr8D-l$4iO}-qdvORiw|gBJ=8Rq9vu@x zErUL#vG7U=!#-`$_X5)8Kl(&KAyO5eSZrjJ82j*JmESA}(cY065lRrSdh*%Ua4{<( zI^@LQ<}H%gvl^EG#1`0Q9hG$Rh~lp#SNrzTtD-Ei_PmFfjpu#zB51p+M^X-L$f(@z z_%qTJK_Aa9=K3(p8}y{4JKDzq;AS3zWM(JvIj^vF3IN9sLEyx$L-x(}_qW*LoR|?r zxagK;M&-#s=9jyWsVPG@*)ku zFRqXUntC||BJeCo(w|ut6d2E~vAK8cYn%MMQF~@RyMS;!#baPze{t_(Mqxt>ZkvYh z{+wZw4ZEsaHw_MT#1F7>ZjlT6Zu7oxQdT|+kikYYJ+ zELp=%F7kA7U}R5AtLBktn^(Sf9#QlgK>fa=p`u~FgOU=anoTn3GB}jo)0VJkgs>ai zv#tO#P3P(wF+Nz)0!lB~naN&x(#8CP!;L$-FI91JAslY#Xew>qywd*q=D5D=@e3wB0;Y&da#o`uo3 zmWhlc)V2-mB!J))+KmGz$f^ezo$<992TmBAY77mk#(ZMr-fe!Zy3_P8(4^s-vgDyT z$(dJylJfm#YJVPAV;j=Hsqp@_xlhz4Z!ueEU6^-ehsWNtYfUV_}4xC`d#V?>o$v%h!`G?yoMyzh?%}kat zLzNRdxwn5Wu?XV50+ZJWg^(VaP=-a-QjBzKzlG&pVUGg{uH>Q<8 zFstQ9R~NO~X~ia+iVaa}|6dn>b?A>DH4fZ=@q0FgE;PiG+6?p#f$jGPZfy}8ls4eb zJEd~xRn%nMRZ;*qCvMbJx}^K94i}ML(VHU|K?FyT;A5LW#>$TZv-j@+>`k7YGA|Ye zHPX#sc-P5AJe?{N2I#%%*Wh-4JGHv?tpw~i;6DaC4$R=mddCXvIBVhkSI8e6!Q z20U!-6x<|PCoA8L&uDfzjtAh^;H^@~?$4!g*aJN~1 zLh>1`7GCbqtlff~&8}2%zqxO2E7vjF^LcK*SV$xxP(xr)aVN%%UC+{=wx9e8J@nD9K$;{3S8kXVTs$D6h&7Y1v*e1vVWR zb{bi2-M`ZjROQaSJgwmPeWLt`J)(YI>Cy2h*en0tlPDS*(BVQTF={leCX#5!$He#S_;@k0a;_zpIP{;dzh?-M`V0@1%cDN;bY!y09d7Pg z5mhQQ8Ct4LeTvtZ0#^o%0*YM-;j1=dO`us6aUTN6hW(eAJiyy_!6V_?Hh-Px(s zzBJNPX$abj3t8&nUWw2ADqB_rDi2jr?HmSMj${HF$(Oj z{KhpZ>)m>WUVh{;7B)Gq{RfzTc+tmh@%@pOq|QjH{JFaf?6y2+<9a+KY2l%GQqq=N zS@6{pWf&*`Xkn0S5d>TfCAj%bP}$G5=a=9B9rsX#ryZPYh!u_-zPq#NV3VU3)N=6Fx)8DY$kxe$ zPN(qNp-_agwSZUy>cu?k7Zu>R3C~cqnc8g}*1&LBWAa~E1N=AFKn%_vT&~SU>@21z zb)zTN=`>GI=zVy+i6V(si(?vO)|(i4W(U8EI#gkXC>plbelbxC%vXOM=YMJZ14tw#)qcwkM z_5YqDT>~{)PR%E+?RMwHFw>c~@s(;=jeBJM?OFWhkds$T-R``ngz(@g#%gDl^$2L1 z$h**!G1|;WV;KKWlj?v;QA+xo>j(OMMNv=Z+^`H?=(jOvbBN6o4cbg_MyX%{T{XP& z4NlO%$aV`bV}`#(UEUJpVer_~@#qph@x5i=D^gix>4WVMS!>aXd~Po;rt!RyaAC2? z)Zpfyi%oP4X>QLXptFv6;OCF1nMFAX5iifN-DT|kCa)3qhp z$2#pLg!M{5D~XFnD^e2sqRQcufar2s#N3eMG|{_zr9v1Wp*zrOzl`C@%ss{q9+)sA{eU3En-?v*=od!6$}!=r62 zCk!w z+Cu$@omOxGRoUvkuHbi9b`_n+4%sefFCXt<)A}O(aF~MXeOid^=3qEU`4G>OBjkoV z1I*gL>vaylzMQSNwLDm4S4Vj(0O70Q@fyFu_D?9TtUth&^|s|KC(&>mkidUyHS&{c zHl&ECOTP7?BuU%ra#O?RVkNl+74o~+d) zxsrwhowVIFQgC=v>EV##wN^gy$t%UGddJC8LXNy;JVagblM+Uk8C?=Hrwv(j&wXLk z#)$!E^OJ3kJ0Ic!3Kqf(wI|5^!Nj$2hMZ$tE0kNc8P6bm4*dU-A%~$>~VR zrjJ%=^?cuvJiVr-SB1TnKCR%L&5Tkc-@fLbU+@jS+piubhYVElhzSiJ*lx2NQQ&r# z^9Z~aR*c}a_!NG(v!+H=3HbV-xd5b_1`*;J?SkT1!=Y2CZP1@aD0u>#aTRBJMa7xg zcd6L8*B>p-694~uxK;w`^r!(f9{(wx1AR|T z-%uo77-6DUsIU`S!)wBIL^8svO7egwJ#E1Iwcvon^Jnc+6vCS89^~jc{L?QfOJD!y zlV15I-w+dUxqpY_CK>>*%{A`YNP}3mnU5^IG^VrCq zL7?46QnCG^DJxZ4j@VzFgumjZWC)`oCrE=wjZpeXE1_r_rnDUOGQB(iQ%Hjt;{9I` z^^(>GkT)_nTDMO65NeY}qzEQ@^(LM!(QwW%d^H_|B*_~vD0A97y?-nOLOQNZ35xxl zU3*ot+NXbV4H~{HoT|0=9i-3_le(X~%_k#rHK_6Xw)~;{>*QpaR3xjc?Pmrh?3N&%A>`b5rju*Jl9UVe3dyHv-PjNN**oAaFg)I%VR8Xb9T|#ebx_Qv{ zK2V!evG}Q!x7C?VP6DwptaHAtrjCwDQ}&<=NWN@7-I{^LXUDk%u|qDg#7@xBF=Yt& z*d5?r1H9Encn{*IM^Vgfhq#w$J_?XvMI2GutiL2K$Z;D2$aaLfA+OTh`zCn9H2mHx zGl~DL&_&{ef4RZzn=-`w^z4{lf}vymINGA|-eB3ivKl{41qqh4>VCFCt{iz4t{f{> z?ECmbzGsA2p6F=l=CzAuNao$>7r2t{7I&#+jm#8#6xbfePK^+g?EV|y54_@Zm11pX~9qaxv)ll&j=e@edjsDCqynDDB+QKuHaldGuA=2J6~ zTfh!LmAJaqxw6T$L4u1@QR4Wwj8D)V%>}RFp)&XZ{VMxM*7x5M3jfvi(R?e|!Vpp8 zFpah_rM5((6eX>-U%bMsjexeU?5-zE)Y=A>1-U%E8(H74YIVlf~EZMaYjkRQUqs= zkXs!0H-e(5q2yezTM1H3roustQh}Ouy`QY&`J`h>dmn>-M;uPEfh${C0XUw@dV&Ju zE;SzEW=_F@XppCJ|K+K8w$V4TTp&+1f;_bYh1^q_JB%51{3wdaXyN)r$B@qESJkDd z7*RvMuW69LzRMr;e4A|$U%geijr@YMZ6Iugs7K5^7Shr_1aZx3FFSAHr+DYT5AQ*O4<*v5^ZdU3M3@%HrO?(-g9t`T`0b7zL_bY z_G?)5?#Zz-KYJ4NkgXxw6x17-f~SHjO9OoXzgO{!wq6x(I)sz<3ai#n$A*dD?VI8c z)10#L8l-(|RSn`tzw$k*ct$Dphp;NVbR}ZoZwBRYgLLeS3ujc9Di}dlou=8f5~sAe z5(jdcW)D~Qw>k!%^mtH;%nZ}UZLluuBkaG{8zsUS z|J&b#{p!AFEW{TZ1MOk7iXu%87^pBnSrH3$3vSqU_tAGAgF?IH3EbKqkZy>t?QscD zTm{a0^M!KIMH{IEfYux0(n5Kh^~TQnZ^XM>b3KF*);}3+i#0Fh=M+2iGowmcC^OX> zdpLCvz}EM*hp!t)cX#N9kNq`DW7Jq9U|uX2yo#XWx71)60z zqI!!XDnX_U#s4NMevqhM^12zSvI8Usx3HR?6KyP~Sra}@jdoaCS5kp*VSLbD14wtm z?c>kQ7pcm;?1{DH1k0-6SMR_34eFI+!$7@qifN&}o>Mf$2WRCnzgTRNR4knYQtkrj;ui0`3`(H1Z;9Qq-Bzgm#1ZvS6g74zR*<%r{|hGV(R zsJZ`zRbN1?YWW|m@=?DW1F>ofhgC*ES-mrktM+hQ)r8}!N|380b?fl*?9olM^-hrk z10U8G9!?eEJDNCb6p_QMA8V$Ie@tgAh}2EGmMbLKHOv+to0qbWTT^li*>=hn7ei%i zrpbnFWI06(E91sbu}!*Tt@@V#gz*}HsVjEQIq+bsGIQo5IVMpFfJwie_`dV~AF1C$ zus!&ravD2=&kkt)D6SrPl5-5ktW~N$CrZhR;5U{iz(JdY_QDdJt^mhu660QigVjW_ zoYO&L{QVxv=w@Xf)UfUiIgzZ*`JYDB1JfnYf(R1?Ix(kHSUl5XVmF2S&W(B=d#}*Z z6uBtmhfj^1fQCb8xVO&9?q8QD^ai}ql(5f0t5MCcqR<;ejwsVpy4BuQgO^#QI;ndK z&rn;(u@-TRRQ0gxG)H8O z`)Mp^T<@wIwSZy0f~ao`e;$gQdp(A12Z5T=7ApV4RawKToPFp? zAI)ztXnv9O7eC!h57o`6$#DgTOKSR-S79HwTQSz_7YexdC#MOA;*(Drp)P7 zG+HOgQExS`8X9HcrB7F9Ii=P@XVE5$!M-B0{Y}`S#>2e9=5suQYao17zAxd?;G_@i zK!4%amcY}~z7`AbIyxfHyAgsj2fuf1p25kQwEJJ1JXlW^{*gDK+0s>ih-wZ}UWfK^ z$oUkelXVqu=uyvPSXNwUO#h0BowxM6eq>#$RwOVPXZB0IO)Gu1_=Rf*Md__Q!$Gq*s|M@cTT3&9+uxy@0LG`jt2KnIir=l=PI~izrqkFakWvK@ZXy{Gf}UWN zyWp;q2{V;mX7W?#MNzzSDV+=oH*Xv8YOy2RX+jb$xFcip4a~@I&prfPb<}j>uk_C)5?huN|d2Al1XQ7Ns`W9V|HN;@uqzNoXaI zyuQVjyF(3#1!{=N|Jfa}^MVfC5S$qCW>ji?Lp0Rnz5#qB)8*Mtf7W$%Xp8Mnn)DMS z5#Z)0`2$HTyg&F5+0N6UI+iQzAF3MhN7f(APq46470*q~-~)vLB+59C_;>=uzFTVd z&qffo#sP-Dm*ZACB4m6U(HdIl_OlQ79VXl`@YG~5y1?4n(Atrg&+^B>KyQ8#5<%yOdk!N zJ*UYoMcEvLth`8OZ8h{PLjEJoUTdgGwmq{%?B31Mn#KmR8r}(6_({9)=(lDMQaYghFboN3b60y^Z3_! z^lYoZfy#E^1vSt9kNJ^KGL(JyO6K$ht#$e){%`e_V#Z=dsG$ryY}r5J!&jLhX4?%W zS3L26uP*oPqHO_I=M2#=_au-REn0_ ze=mHO9YL_r>(rH>l{?!c7!C;~Khq=Fy7F9Y@U!fuDQa8$G;LVud7AO_RLh`uwkc50 z@>9JV86G~{lNwR-E_T~Uw^%&ofNb2GeujI~z*v4MLt1G^FYT$g9EQP%AkFwxnu6#? zs{2#A2DR$G>>_mQ)Z}U3lz`-%Gc{`Tfs~eIwGDL)Eu18pJMym9BZ)_Hr-3HSoR&Lx zCDHoU53cw^S(_z8Gl?^MnwpHvlgH&g%Y&N`!uA=Kp}ygD5o|h2WAIgR4er|chL2_y zIL;hT5VrpzrEdCSy>)Se6kWJZjXz%ma}LKdPvqGeC@}$3B8CchDi+PW742`H9Ap}2 z8f=^swOD5C+!7M?4Y*%-zF!FX1O2?MV?aks#>D&^3Y)yXYcM6wDgB9%jg*;zN$`dH zZcmFnR!L8N$DKnXvqJM(3ZvDIXD1pcREnrPlcD~&0T-4de7pMP1HA+@{;s-+jE!lp zQq4aW2{SUeGcoV~+P=U1Q<=u4peFUH5*2%x743NXR`gh|bOK+mSU>oA`0l0h z>fJ62`{K$JFBI%y^^p<7R{2$*SV-xyh!;&qE*Dh^!<$VjGGoce0&Bb7a<;SFmfFHi z%RC;KPy`~E)V?i5EF*lZY{GVs)wx^^9Gwq7g+i7n=7mH@H-xL|zR_crX9#^flN4;@ z(xj+*qBwwFJ*{vz@Ab)$@l^ z%-ugkXnrKzKLtl^i_))z-VA2cMLxV>*&ZPGeowR%H+kkRRV)Ij?nK3gfE5Z_i^8b& zzXb`hie}q-d~@)CI;Gt|)>;Z8Ehd-Xnj*ydY?$?Qo0kPYckS}oSxoDAKi~0`LlC4- z!5I#`F2vKlpRJs8JJ-v*2|W1i46@!47njucj53r4EA!uEpsa}(v_XqLHCf(h5V1rc zUj{*XZi84SmpDMA3FL_s6A7EPaf(DgdaKPQGBWf~AZ_BsyK+7!?Eo+(GWgM^xuW1RqUwFi572Z=f zM9QcCJa=?e)_u1^cA*AJ?AZIixXhN;5-wMQx4n^PL1`%Xh$}xk4OPhWTM4Zxy%^ir zE4^{g@>L83 z{186(Q{x~?jJ-m+m44KqYeZ*~{ngSPKOrf}e+!x-D-GSe9DDpO`f5Y&fMc z=r`xC^{`2HSfKLWg#wk*3MAJn3>%iWUv)p)odbNoIWYY8^t*mq*@ zAc{6jtW6Hp=eYXp6v4F*0})8?Naz|y(4^^2$Ybp+$GLzV(NTkvid*>Zx>sDU|5kq} zSB)lYu$s2(i}E9ezMa)pyV#hb7+ULo`8)Qs+JL^G6Xaw5N+mmg|hL`_8+m)%))LTrE+7ep|oG zSl)>SCO#{CLydpfzPHeOue`3id=x?F z_bS(B_Ulvqk|##y-6Py5fU}zfcKe}2CizJF%nT_zUGZ^0|69hcbKfsTR4aE}Wlf%c z@C+<>KfLAB4$c?}z@!^q0gCBh4TB#mc-Z;(p;~L8iwJG^V%3rwG!YGKmGj|DMDyw{ zZBsZCkyh_R`AT`epEOW8j82AEo?-rt!RwgT%N@2Zctk7PBJfJb>Sgo-uU!jsLPyS& z`I9#g5hHf>t`N(ZqY`mGDc@!xSo0r5*ioClwX=m=y7%#bZ^2qK^;UgrT+gCG)jzpN z80q$<)?k_PC=D(c`{wfBVC>sqFt+K*e+6T+d=QDD<9_0yJ(E!)+IieJp74}=Kq?54 z6$$XBv6th7c^1&~&n3FPs`8?@+74T=oKIx|DlkB?n)8PSC%}2FtcvL{p+awr{z)Xl;2Hb`>TS zTD0ligmwHDFGUwpB|+V$Mn>gJB^>mrsR+d-1d%oletAxeI^>}9+_xfpV95*+HCW$( z4u&*OXXyRY{P21!xQAlE(yx;sc%ahW(t3bt)e(ewp>OIW&~IHMp30 z0vnC0!d|R<=Us=IGfr&q%mB}pzZa|0^?o`+p?2;ZJ&zq^_gU9*9mAp08zVo4Q*)HS^Bi3 zb&qhr+X-ogq#nkuyJO)JOg%6RiLdD*sIK36xc?t>(f3o%>F{4SCZ3eFJ^o|UCXLnn zY+noxO+j-}8UQGE#r`jMQAbl~xDn1>)HDtKm~*z`*W@|p7Y7=QN@_kgzpP>?5}+$uXmiT#Dtad9uNr(e)HOEE+z+zd zw5J~@mdqXXpQyfiY|@_r+%13}N+IZCn<`;bpC|m*?w)b@J`l|=)aG}1 zSs2B1O&gO+cSj5KL=y@sw%Qmi990yKe>ky!?ddz}gR~}6!W@jrAiA~`QM#TXqYi=m z>vGI;(JZwm=RYwF$?hCAhtkN+(4zJkpXM><=VfI=1IFc|@+uIMUI?U5#Ni>#fjY#o+gEHFOW$slSJd{= zDgwpli?w^dr}+NWh@w}S3KlT85MaaHg7v$uqgF7-9%an!cmF(y#eDTpU``sjXJsI& zjNaTv$^;}YfyWyEMbocFO>Z`@vRSFpVW4#JBW-rwJ|me!vPUpK+sM1RvUzCfFq+Y= zwoLU#D!$;{o4r@q-!%P(Y`4m!ZDG0nXrs+D-4^Tp>e@HQ;Q@4z40xAtTGD!Y*Uyzx z(qNY{3Mo)T+-uO$7HTg1ip}Uyc@mQU5# z;&!vtx{#{-MEwn~o^~huMA-TC_WiH?{!uI$2QpX9i0YB=0Qt@J8=F7CYiaD=BXo;V z!gW*KkH?P;v8SZ*r=%ooPKOVtZB1fCGlCRZo05%8FAo3bQB&@O_*A+i6FU@b?TlfhLR^X$%O*`L&} z6PmGml~`i1{xymFRDUUno!SBTDohEAxpP0wQ zn#ji_I}syxQP)9Rl>hCSb4Xhu|Ct+krz6+^Q^XQX#ps?;h zdP}M|r+Kz<`Y58Ktv#Qk$ly{#h^%8Z&-#$wxWPXKGUn^pw+*-5xXUwt#Wj+1{@gD} zT$3F*Po<7ZiC=~>8@0G1Ig`)HTb}8%3E`*jP3B79Z8=CV3Tcgk!u*^z+eU>2p0X}} z6OdMkHyn|Za_l2SLi~PPDTfSGwN*JU?GL@&<5BcYHP^FZe?FI@85T}BYiU>`_*?mR z&wdA_&YVY+j{CZF=)!Wl;aC{@T7>`Fap@g^ykazfDU3}M*!97{xR8elb_z5S|`1UJmhZ?gC1%?+iNy`F0Xi|G{=wyhA?jaxmV^* zW20vpg|nXN-pG+u{*glrnQxz_q@~V?SFEo=WBl-YeZP5Zxd@pao!j`_+JVh6nh_9K zwzPg6w{R(MImOi3ioZh5(4C!g^Nh_w+Z_>G3>*CX-hGExI5&jN;d<(tq>JQ=n4?6= zy7!40b@6m!(r(FPzvuV3PbUuWD;?F;-3w=+)0~MuLHYT_nW?$YEt;uA_I1C)H%T=+ zpSPbKBWn$wKY9qDR}yG;YC3Byq5MGhHDflrU>EA%0luLFphh&kI-it>-FDMzAY!Q- zwLxe=ICs{(l#uBkO`jMoTf5C8MVRi0j8~!34ddXIIjf2%SOopoE}9$ViG^n@_9d(B zcK3gL8l8n0SrFKCy) zp)aI>+r|%lJvT}`N#ONzP8gx-dxu);ms>B)jqv!6=Y|%3r#PHQmHUS5P{!8tiP=?U zx+*2vyeg~VzSv#c9emqnV8}r1^%PEV!M_m{<>`+3=cCoigT1+h{nMMn&t%K%=CRXq zwa;#xyAB+VF?v=M@3PcRgl~W{?UVb9*l&0(P?5(#*`?y5p9uflGV<*)fv>)(ks>zn zmBH9MeTLy;htZ=<_e8I3{Ltc_x%ex_oj8>9V*6|Jxl=9`t~nl#g!iKJtv~mAQmdU! ztgm?O5z;=Zj~g`}GDa-I>aI7*Vw)hc4qFIW_ibR8(xffS{D#I_2_IO9=k;=#ZD^mU zBu|PWv>;m+yQkBiGWVAnG$g3%cHe1?i}_G8Ghj`U{F16ivW-*6E$w2w82@mlnA=;p9#=~4tprNDoAQ*Jb|Yi$hrifX z|3Pnk#!Sw|u5kH>G|t03SJd}@V*R|Y2WH7O7bS1FN5-r6v}?QX2|c*%Z{@mh`#JdG ztk1J-{)!)S+Ksm^|j8UX%|Rz-edIeUdt(K$u9f4WJ>l6}_%a1MD-q(VQ((@Do?g}(UlIomkDDi*>7t0oBSrqfiA)QU zqFYBvo>67mtP?|`26uQgB)AmEW@Vs1d1}`qqGOiN2&RBf=r6W}`rK8>15~m0)$h9w zuANACNq+&X^0IA`kG7d?GFL)P@)_NnX+_?Aq~m~p-}$UHQakC@8uvQqRPDL2#skPw z_U|Kg6{&?sn3dw!bh({CF4Xeyq7x}>u8?eN27hyGACspZW~}auS~&`XIyHy8!q#h@ zlQH&jzk%j)r2EHJ(yHngVyN?1VC%3gi4ht&kzUsQ9ItW$?|RIKeq=k&XzJ}34X zev5KLrC5@V^&B}$-0S9h)=_;Vi?|%75%5D^0G`c)uEYlRFv2p5xQ=h4ksIAe2(QD9 zn_{}eamsn$gxa-#H7!Hk?6JBdyISV7u#U#_wI0w||9UQreJi1ILBMOelW9P#lU=G< z{dYh~E0uWz2GNVgRH{5d-K#?o4!zaV^U=S1s3ba@#|$Hy-xrOx!-)HxaG(uI7B3wD zE9P<9zBQX+ZkEQoI}vW)*t2r%7kaawNB=D1c{UR%AQ#1tRFyFJL$m3liRSnA`j5Ue zlk};S_y`1!R9f(U-rASCk7Xr3c5IX!-$+>V74E+?oC>U=PhXCB%$n5@*}5RQzjs$V z&ika)re56{J^Hz<+^E)QT8Y#i?;vyqJ|lTb=nVbiiYex5RW5)JvFJCe=K2%zWug9M zRi2O+aP1qX+zKNClh(gyuUPg8l+G;bc&gkH?jHErs>aH zNAt`F;^*xTqW?&D6pol)y&86CLUx}pZ4K^i4l0YYT4ZqLv?Zv&8!nHYU)=&m}PkPf)0G)O$BMEtD`4vqDQ$y?HNaZAR1TZ*q6Kh zanIrs9m?c}zEr}zsNG_VJ&q5k2&0?K*Y{{x6xZmb$(y-vi-ZkM&Z_Q6v~-xNhT+?) z4Sgvc`U0mYv9Li@k@wSJ@`1%ASu4@-mIBfB{;R4$x@`^RBl+q?mo@+GtJdVvkFy=Y z3>;E*FvSmR>27juYsU;yvFr1Yn2bqRleM{vw-z=r%ye*VU2EN8pJ1b9=w1!B+PF|A zu`b)}uUo$Ap1~*C8zXLp!s@M)RVnKZ{Uct`vK+pq4&W-Wwk@}{YW70A z7Fzss`|!du$sqz!Gm{hnyLn=N=@wL!(OVRJU}llE`iINI<4M!Cv)!cdMwdge%w~h@ z=jH>qs-LxslD?VO)1pchLZ7LdV8yWx1BIJMh=Yavk7ncYZL}|nr~vQTh(gs8iI)V& zEw6|9V<>C3bfVV}N&q>^BoxbTodotPow2+76w2J`(v1sd9(x9VD&Uk(XIN0htFBeT z%E#GMOl0Wr1E$j#;lL~N>{~6*?posKcZQqSvP&l!C)_)oVLkbz$2KJn9hUeov6@g+ zb>B3%xyq}moY9k72Z|(Z_(H6s?dtXyQ{yi>`GyOpw)z7mWxztL?A#K=<*=NiJF3LM zz5Bx-LQ;f4;u=3mjBk9zsTx3?2~!(mh@0IJkX)Dg@kvtQ`Ve_Bz3|6(=Zd=Y;{Mh1 z4c{nMTffq5zXK`Z$4DjqqhDvnyAdA4RfmB07mT6r!ji!b{=ACfV`-)5z~VB33b z4Uf1=wn^37zz6h$@12+4TWAO?YIxuzXECL{C8J_iWg_BbDWM_G80BwjYCO2z>EI#X z|EfAKn=*4vWL43ZFM5-w-?qFJx%B7tT`prV3FcfVU@u1Z`YfhuPt1DprF&d@g+#5d zSPLLqe%mtt2;-ZXP1$*T(!G0`Qz9MQJ?5D~BvOo-i~-!jYlV+aA$=~Gos;FKbdJrG zp28vu26%PrYmM0lN+%8PLkLDwq-0su`*UZ;@hVkIcfnNd7lQ#JCmVfl&9qIHp$*cX zvd7Xnqg4!3vlpU@Ofo*+PbnWTb0+aGG;WvDN^1#@ zbPa&%cL81Gi%Yz3q2v1H*;nappDTuM^48jq22lXACB8Dd&Iwld!*Meff0u6VOvQNN ze)k62G2!TAp6|==ft+NsbMVz8Gwc1VJg*ekHCaxIWgIA_DN^tiPrTs zt#tZo?BQNjFxw#yh6&$i@S{)e&iSRzZ=d#Ios)&U9-0gg!Bgg`Kc-_KX=3oE_=V-! z)t)%alr?svRp>X5a8phWBHhf{Z!a<_`Hz{WPg%H)UkhzTt?HnJrG;-`Q`}Ci@|f2w zu-ds>b8r~jHfMkwdeUq%b$YPl>dxOTIRM&$G*dkkJZGD#)@yAYT3u^JCU+0V+R*4O zp`YwEus`0O;!A!s0=8%8Qa+nio~tzoKMO>xTM1r`O8-KN-kNiwy_vCg?$*_6$B~Po zDr#uErLEW-@5s(*E_viEiP9n=DzOGgla?wfUU8gktgarIo?Uj)mfCpDt1-lvW`<-W zM{}G;Rm09A*o3&|-ykj6!(BGf7`kNx8_p@?egS-;9x1M-KQO(PQu4K|g=OseLiU08 znRlzJrxPXknJP7?xzBF0H|Vg#b7H|9(@l1WWq;j+SkE@)C!6ZI{IqqHFyN?ps_&`6_%wy2e`CP28k^k($l(d=KmSe>)eHYVM%fwwI}$e%P%g( zBNi#_%QY`8Y)!s*KgsWAWopz*?TJAqJYdpuq1)>U6I(hy+ZtEE%0$UI{j6kqHEdXX z1wQIuws}roJ?bmL|2JIA06589L}zWNn@y!kINhmR)=Sf;n{FvOi@2N}juXAK?Sv+p zkXwb@KeTKt>#ST!kTS8gy~l&H**qjreUi>m8tNlaz-WJTB8HsV#I!Jt@}u_3C8Jhi z>|xa+?hfhpRLX{9`fHy9`FdOwC@sIgXfW`agl@KpXly0p{;VFDJSl z_UZI*h(D{J>#2|C8|=<)uCLkTDo*|r#pQc;=aD>Q8Yy+%6*kn~JDm8(YP48yNv9Re z_4rnYcY2JSaqBiHy`h7OHQxNplJ5XC!99*?!*-gGxn~Vc1ipzj7NzKHsEL{Q z3oymhB4>iPB#OOL{!U=RgquH#_UFge`1!d??=62@zdz+7+In=U<8vDaHnmE02FsT3 z-0K1ye|+*M?-7((T(G>#mzYbS#r`CH4rb2Y?3y{j=o&~Df#JKCOdtIo(=#7#PR0^u zzd$vakE*&gxQ+NLMSPFtu!t_&MQzp?{Jd%T8lrk&t<2wR*2|Onl&tJ5Xob)MnY=Wx z&$vwh{uEPXev}u#(X$oZP#U9&X*{hb0+`5%rk~mdc8M>iJLHm1G+(JTuL^IfvSiYg zR>xRdsLBh3iU3EnMYJnuCqP5YPr^@v{*TjP8Xo$VJAwNY1dOJdW079_9S=Stp zbA4xfQauB!QNY!?shjTXj=s)ayjHNG@Emw=&$ZALF#`PKYv})@-onV>OWseDIbwA? z$qqQP@zZ>J8MxVQ*}xktzx%6; z-JkWef29L?8g)(P-1+uXzs!@jcN!`hGX|%ur-OBk3>Njai^i7LD9&Q?7F&1&2V<~( z-5a@EX956)^xV@q*DrhxQg54P`QL6C*}!?uPSk|cKKs?q`hRuGJu@D1F-u==bmmi8 z%J%Inl$QP$AyUNS%QXkF=c;arI_O9%7c}XPgPZDArB(~_FAooon4HoKjpqNH+)K}k z;YaTbH$2)^me$1hMeqi=o8XNKyNg)NO)jFWJ^27J(G^}3R+D&oqlLj*>)MJiqq+gn zJ-1W7v*uo-`r4!*9><8XZKCNyb85Cx^-zZ_Cc+3`2i}bB;LmO zRWabWozyK}Qm6844F7!~#%NQ!Az3cpc?{6IHk-7< z_2{D2N&_`blG2b*N)|qv1N^B5vD}zsXdBQXfeKfBkNMH$5gsi?MKNMmY!jMQ))P0# zZ zkbLsfJp;2c`q{@wr`Cs3_aSA;yH2C#7eS%I+H`)_5j0Q{J4$$M)Rbn9^=W1_=>Q-| zk=g+HL$M9Zdo`eHCo8y0)cPc_&X&H$hKfSE=5mgE;*c!mRny}m;?8l2uy5{D(L+XZ zPo}?A*~TEY`YwKFPG0-dx4>!D?W6>(f|F?q&64d+b?4AgS;hicvpXWzQ%?(>8cIwl z4A2o-wOi}SvIPzT%+*N?7k2Z|-AjP8wrA z;^u>5$b{7|;uhA`|3IaF(C6tj`I||-mTv1Raq+hbhhA7OKprrKw8m<8TZvdyK5Alo zn{a>W{Q$8AB5b}vN6n$BByktBx!L@%;vRgh^jVo_6>pR6%X$(y5c(d9G>=-9nwo z$|`fP1gX;#Lj0n^$3kcGwe58Lb>oaDWvIumhMDR~Bp5QDcp+v@>qJh0A}P$lw(HJ$ z>3S~e2|_`aLv?}6Y|poEfGE=xTN>O_l4s4H9@Rgl$~iEdx1|?1rFj`?>j9l?d9N+* zbkvW})4x|{8EF+mt*WLQ0SxHI zjC7Hcx{aeB;B`41d(Z>#A_Hm%)Jh>j*Ts%YZ1qC~m3%`*yXej*knj!iDkpf^m;11- zD){0vat9wwfsoAn)*qp_Q$gopDO40U&W+_F{p6EUcjoZrx8;tEb+%WGsS;l{n|yVd zEBzsqh;{YE2x$u;^tU15lKo}=Hu*I<^8cr+E02b{d*h>!qL4~LNJ6%dbubt|icrbE zWFL&>wGL*+my{O6C`%>#zEv3eV6u+vA!FZV-L zBcC<8Jg6z&TXFE3eelY%MOMee*AOT29*Yutukb?d(qsR30YzCR!Vpu5Z_Zu*+htvv zynVvd0LtZg>$pkU!%d?JNYKbuMa;)u+T%@^`f#n=*@F%nhe#*-l^qda-|=Nka8+HM z#d}pTY#IgsEOh|I#9h(`#Hz(UUU+0}SYI@)E^ZOwU2s3)L$F#4IZ+HsMR)bbu+W03 z_dH+kCX0%HelDuEov+%rn-uC+@pU)Boy0mgsNmCFmBHU9Bbp7Dd3x!B|GayapMDbr zW?jcAEHtEIHUs;DT#W!)739q8{6r^tqV{A^<5zo$yF%`Y0vA!3PJ>H6AI)dbA_?C& z8ch|Fb@p)T*@BX@nySY3l~f~votp&`S@P711O4pig_stp9}3TZZfqR|d1GL(DxjGf z^~P0ND)>LC6o@?{xaC9CM}KKM(_NF)@8|o#G z`1%KBG77|ZH75(Hqw*VLx->~u8pIAQRXVXjY5+jRl?&x56Q(B&+*`A|4fSZs5VMrf~YWQqP za;0&Ki3I!lvlzdn)41*nsabUJ+ezHbqR>{({w~{-h9Sey~Gap?Ms4nMa?pE`} z60Lr4QDIz^%MzFCXmd>gd1GY;sKPZeRXzLsI#oiljI*6|803(9ke?>2;W7_ld~988 zoahHgYG$eZo=)1k=|r465`NML=muAP31)h7&{%&tam2JmE`|Mfj4|}&_9i|a@v@y; zlXmp*m`iYxab)#kZtP~Bz-ZOnz}Kx~lOJc%Fqd4rTno+HHW$qGI1jku0_7Mz9IW(* z+fqvkx$q?7Po^sW3wITX=k7KjxTI2*ZPqt{kxBiK3Sofhd{!nxHnwMU%LW;DMhdw{HSB!1(&*AOn954=l~H6_z5$FaW%N7dP+PpP=x?1aPmH-H&a!{elfb!*7ar_ zmmo8ntM0tWn~A~Vr$w4PoX5Xmd&lUMVCrVw zp-E!4fG@+UZ**u9H~#`-pj$Pa4`2}L@r%B#8g$)PG8ElO7vn%f@E)Z)P<=ar=tH3D z1C3|JYx@TA0OF1TqSrRu+=1vN!`Dk27UDqk^rSTlO|4~p;RBl6)Tb!KvgEgtHbAi7 zB@4+zh;u~Z+y+d%e#zky^gjyr*MwUharsNPnEQ-)XLe3XbM)$4} z=2m&)-YwD}CnjvWC=?sCLTa0i1>32k?5Np6H$)Q3ZH#|XbFx-AbW*nTs@Wb?RD-Ogm$NDh09bSSIwG`-S&e|6Mt=V)s7t8NLs&#b(0F-Vp6`m zWWeE~t6KD9XNJd2hY)0QH&{2wCSes8cAVyb*^tKPygJ->e-+-RrXk*6`6r@xz=qt5O*Fz-F=ufBz98Mv4JO?? zKEdHQWr>rEUu8$N2)Vjjopfu2I)8e<|K|FtrI_QSAa^UKu0i&|$+8*R2HR4C;;aUZ z)dJ972!hN{lO3ubwA}}mds*Vj@c!~npJcm~rcA#L&Vp0@SvIk!CfN*-&3vIxRc<(^ zPw}xl26zLzh0a;Gmu}e9ZU&njx&|s1LMo?s#_jtYgWDXzjnUgxWy|Dq*S+htwS^O5 zdbZ++PhvjX4<`!U6kMYUXlaiG9DI0Nakfvlz_7R=max_2=v5k2!~Fa$xO05po9y~Z z{X> z^SB$xST@2jy~rraF@s8G5C7@TY2%8X|1Xu@ zQR^)?^6yfWVWHNQf)!WP~v8i?fV4MiRxUm%Tu zq5PxX_dQ|0afU$R-prZCe!9U5K^KwPK2t>`lL}+VdU(oM9DGz~SK~-+p+SdE_ek#>tqn6!VQXnJfD_E0E-K=OtH*I$eLf9N%Y})IWKP)(c!~ zB4S#(qGYcTcVsiYkuJ3>?pMb9Yi_-I&zlM1Eav{wp*V8A=p{N3C&^|6)*kzjHPD+g zgwM1tyAQ99+*T$OzWQC;jYv*a!iA?2(&4pW8^5C!v~S(Lbpwy5oIbv~i>jq^Qpy>* zozmUEzLh-U3exb#MfSP13El;M%`c@r_oVpHFxy-DL+BCSPi4=aQmck@qb&SuznhhN zS-CT(Pq^T3SWlaZ$mAmoqAEs7n&i#^yOEk0>GvgS40vnHkmBAWv^UNbdA#@+u`@f0 z>AkZqRa|xzqN)|ZX33gsJ6})Uze5gq7fslptXE+C2$K{hHj-?*1<<8)pu0K2hGMhv zg+T4=Lm4#|H_zAqdPk5-QFdO7>S??A@B6`rYb$;h6~EQ%v04PudE(vxr_py`yZQN_u1gGb zM&fD`(2gI3O`hp7AfWk&X0Q8aN?h=7G~D%n{mTJZ5A!#11?^bd|$i^A%^7hNf0 zVmSLJm{L}@4viHq@dC`#Yh~nLtBPLOMZ0~6zpv2E|3$Hi18JVdzlS={ z_L+ZAw=m_GC~xJ*I)?&3hcWD#6c?I;b&~8_SH10)!kW8Caq!|;Fuwb08vwqDqkm5Ry7r_~41$Oz zh52(ex_9i}s_X`*RJI5d+HQ}|De)ycFEPx&xF5N?vxst-lTk=@=+1x81jq&oE{rkb z`i#6mBl%aZJY5^QDo#44?#DHb68e{+;Er3tdr~O`^$eMg-IsF{9}NAO=XD1sm4s(K z2%V)g%>6Z$FFc}RLugN;7_OOOcJ4$qx>FLM|7jmvS}OCm#tWrttbb&qDV+^ge+L4o zd}oQhi<4CQgZ-mECw2P&xDp2fX}Q|LeC#|P*adm{`~90f4aa@zetc{lj{EN{qd0EG z-9!+`$jQ&o)f49XukgPs?0G`}?ZlY=pXS6{{S+%%c17jSc1l}5=Oa#_S0~2&kD#O^ z5D4n!>FMO)=j!G8Z}!;GGVW7m$+5|0`uzVpA#@;+_KC6mgU2Bw_FVZ5#?8z>=zjrq CM+|fT diff --git a/powerbi/README.md b/powerbi/README.md index 5102fb77a52..c5fb0576704 100644 --- a/powerbi/README.md +++ b/powerbi/README.md @@ -40,7 +40,6 @@ The function supports two parameters of which the second is optional: A table that contains a file/folder listing of your Delta Lake table. PowerBI supports a wide set of storage services which you can use for this. There are however some mandatory things this file/folder listing has to cotain: - a sub-folder `_delta_log` (which holds the Delta Log files and also ensures that the parent folder is the root of the Delta Lake table) - mandatory columns `Name`, `Folder Path`, `Content`, `Extension` -- a column called `file_name` These are all returned by default for common Storage connectors like Azure Data Lake Storage Gen2 or Azure Blob Storaage ## Parameter DeltaTableOptions @@ -57,7 +56,10 @@ Assuming your Delta Lake table is partitioned by Year and Month and you want to If you are lazy you can also use this shorter version without explicit type definitions: ``` (x) => Record.Field(x, "Year") = 2021 and Record.Field(x, "Month") = "Jan" - +``` +or even more lightweight +``` +(x) => x[Year] = 2021 and x[Month] = "Jan" ``` It supports all possible variations that are supported by Power Query/M so you can also build complex partition filters. @@ -99,9 +101,9 @@ in # FAQ **Q:** The Power Query UI does not show the second parameter. How can I use it? -**A:** To use the second parameter of the function you need to use the advanced editor. +**A:** To use the second parameter of the function you need to use the advanced editor. Power Query does currently not support parameters of type record in the UI -------------------- **Q:** How can I use [Delta Lake Time Travel](https://databricks.com/blog/2019/02/04/introducing-delta-time-travel-for-large-scale-data-lakes.html)? -**A:** The function supports an optional second parameter to supply generic parameters. To query specific version of the Delta Lake table, you can provide a record with the field `Version` and the value of the version you want to query. +**A:** The function supports an optional second parameter to supply generic parameters. To query specific version of the Delta Lake table, you can provide a record with the field `Version` and the value of the version you want to query. For example: `fn_ReadDeltaTable(#"Filtered Rows", [Version=123])` diff --git a/powerbi/fn_ReadDeltaTable.pq b/powerbi/fn_ReadDeltaTable.pq index 0b4f6d5384f..79b120a1edc 100644 --- a/powerbi/fn_ReadDeltaTable.pq +++ b/powerbi/fn_ReadDeltaTable.pq @@ -1,222 +1,244 @@ -( - DeltaTableFolderContent as table, - optional DeltaTableOptions as record -) as table => - let + fn_ReadDeltaTable = ( + DeltaTableFolderContent as table, + optional DeltaTableOptions as record + ) as table => - DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), - PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", (x) => true), - UseFileBuffer = if DeltaTableOptions = null then false else Record.FieldOrDefault(DeltaTableOptions, "UseFileBuffer", false), - - DeltaTableFolderContent_wFullPath = let - Source = DeltaTableFolderContent, - #"Added Full_Path" = Table.AddColumn(DeltaTableFolderContent, "Full_Path", each Text.Replace([Folder Path] & [Name], "=", "%3D"), Text.Type), - #"Added File_Name" = Table.AddColumn(#"Added Full_Path", "File_Name", each if Text.Length([Extension]) > 0 then List.Last(Text.Split([Full_Path], Delimiter)) else null, type text), - Buffered = Table.Buffer(#"Added File_Name") - in - Buffered, - - #"Delimiter" = - let - Delimiter = if Text.Contains(DeltaTableFolderContent{0}[Folder Path], "//") then "/" else "\" - in - Delimiter, -PQ_DataTypes = - let - Source = [ - Any.Type = Any.Type, - None.Type = None.Type, - Day.Type = Day.Type, - Duration.Type = Duration.Type, - Record.Type = Record.Type, - Precision.Type = Precision.Type, - Number.Type = Number.Type, - Binary.Type = Binary.Type, - Byte.Type = Byte.Type, - Character.Type = Character.Type, - Text.Type = Text.Type, - Function.Type = Function.Type, - Null.Type = Null.Type, - List.Type = List.Type, - Type.Type = Type.Type, - Logical.Type = Logical.Type, - Int8.Type = Int8.Type, - Int16.Type = Int16.Type, - Int32.Type = Int32.Type, - Int64.Type = Int64.Type, - Single.Type = Single.Type, - Double.Type = Double.Type, - Decimal.Type = Decimal.Type, - Currency.Type = Currency.Type, - Percentage.Type = Percentage.Type, - Guid.Type = Guid.Type, - Date.Type = Date.Type, - DateTime.Type = DateTime.Type, - DateTimeZone.Type = DateTimeZone.Type, - Time.Type = Time.Type, - Table.Type = Table.Type - ] - in - Source, - - #"TableSchema" = - let - ExpressionText = "type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", - BufferedExpression = List.Buffer({ExpressionText}){0}, - TableSchema = Expression.Evaluate(BufferedExpression, PQ_DataTypes) - in - TableSchema, + DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), + PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", (x) => true), + UseFileBuffer = if DeltaTableOptions = null then false else Record.FieldOrDefault(DeltaTableOptions, "UseFileBuffer", false), - #"_delta_log Folder" = - let - Source = DeltaTableFolderContent_wFullPath, - #"Filtered Rows" = Table.SelectRows(Source, each Text.Contains([Full_Path], Delimiter & "_delta_log" & Delimiter)), - #"Added Version" = Table.AddColumn(#"Filtered Rows", "Version", each try Int64.From(Text.BeforeDelimiter([File_Name], ".")) otherwise -1, Int64.Type), - #"Filtered RequestedVersion" = if DeltaTableVersion = null then #"Added Version" else Table.SelectRows(#"Added Version", each [Version] <= DeltaTableVersion), - BufferedTable = Table.Buffer(#"Filtered RequestedVersion"), - BufferedContent = Table.TransformColumns(BufferedTable,{{"Content", Binary.Buffer}}) - in - BufferedContent, - - #"DeltaTablePath" = - let - DeltaTablePath = Text.Combine(List.RemoveLastN(Text.Split(#"_delta_log Folder"{0}[Full_Path], Delimiter), 2), Delimiter) & Delimiter - in - DeltaTablePath, + DeltaTableFolderContent_wFullPath = + let + Source = DeltaTableFolderContent, + #"Added Full_Path" = Table.AddColumn(DeltaTableFolderContent, "Full_Path", each Text.Replace([Folder Path] & [Name], "=", "%3D"), Text.Type), + #"Added File_Name" = Table.AddColumn(#"Added Full_Path", "File_Name", each if Text.Length([Extension]) > 0 then List.Last(Text.Split([Full_Path], Delimiter)) else null, type text), + Buffered = Table.Buffer(#"Added File_Name") + in + Buffered, + + #"Delimiter" = + let + Delimiter = if Text.Contains(DeltaTableFolderContent{0}[Folder Path], "//") then "/" else "\" + in + Delimiter, - #"_last_checkpoint" = - let - #"_delta_log" = #"_delta_log Folder", - #"Filtered Rows" = Table.SelectRows(_delta_log, each Text.EndsWith([Name], "_last_checkpoint")), - #"Added Custom" = Table.AddColumn(#"Filtered Rows", "JsonContent", each Json.Document([Content])), - JsonContent = #"Added Custom"{0}[JsonContent], - CheckEmpty = if Table.RowCount(#"Filtered Rows") = 0 then [Size=-1, version=-1] else JsonContent, - LatestCheckPointWithParts = if Record.HasFields(CheckEmpty, "parts") then CheckEmpty else Record.AddField(CheckEmpty, "parts", 1), - - #"Filtered Rows Version" = Table.SelectRows(#"_delta_log", each Text.EndsWith([Name], ".checkpoint.parquet")), - MaxVersion = try Table.Group(#"Filtered Rows Version", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion] otherwise -1, - #"Filtered Rows MaxVersion" = Table.SelectRows(#"Filtered Rows Version", each [Version] = MaxVersion), - CheckpointFromVersion = [version=try MaxVersion otherwise -1, size=-1, parts = Table.RowCount(#"Filtered Rows MaxVersion")], - - LastCheckpoint = Table.Buffer(Table.FromRecords({if DeltaTableVersion = null then LatestCheckPointWithParts else CheckpointFromVersion})){0} - in - LastCheckpoint, - - #"Checkpoint Files" = - let - LastCheckpointFile = {1..Record.Field(_last_checkpoint, "parts")}, - #"Converted to Table" = Table.FromList(LastCheckpointFile, Splitter.SplitByNothing(), {"part"}, null, ExtraValues.Error), - #"Add Version" = Table.AddColumn(#"Converted to Table", "version", each Record.Field(_last_checkpoint, "version")), - #"Add SingleFile" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint.parquet", Text.Type), - #"Add MultipleFiles" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint." & Text.PadStart(Text.From([part]), 10, "0") & "." & Text.PadStart(Text.From(Record.Field(_last_checkpoint, "parts")), 10, "0") & ".parquet", Text.Type), - AllFiles = Table.SelectColumns(if Record.Field(_last_checkpoint, "parts") = 1 then #"Add SingleFile" else #"Add MultipleFiles", "file_name"), - AllFiles_BufferedList = List.Buffer(Table.ToList(AllFiles)), - Content = Table.SelectRows(#"_delta_log Folder", each List.Count(List.Select(AllFiles_BufferedList, (inner) => Text.EndsWith([Name], inner))) > 0) - in - Content, - - #"Logs Checkpoint" = - let - Source = #"Checkpoint Files", - #"Parsed Logs" = Table.AddColumn(Source, "Custom", each Parquet.Document([Content])), - #"Expanded Logs" = Table.ExpandTableColumn(#"Parsed Logs", "Custom", {"add", "remove", "metaData", "commitInfo", "protocol"}, {"add", "remove", "metaData", "commitInfo", "protocol"}), - #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) - in - #"Removed Other Columns", - - #"Latest Log Files" = - let - Source = #"_delta_log Folder", - #"Filtered Rows" = Table.SelectRows(Source, each ([Extension] = ".json")), - #"Filtered Rows1" = Table.SelectRows(#"Filtered Rows", each [Version] > Record.Field(_last_checkpoint, "version")) - in - #"Filtered Rows1", + PQ_DataTypes = + let + Source = [ + Any.Type = Any.Type, + None.Type = None.Type, + Day.Type = Day.Type, + Duration.Type = Duration.Type, + Record.Type = Record.Type, + Precision.Type = Precision.Type, + Number.Type = Number.Type, + Binary.Type = Binary.Type, + Byte.Type = Byte.Type, + Character.Type = Character.Type, + Text.Type = Text.Type, + Function.Type = Function.Type, + Null.Type = Null.Type, + List.Type = List.Type, + Type.Type = Type.Type, + Logical.Type = Logical.Type, + Int8.Type = Int8.Type, + Int16.Type = Int16.Type, + Int32.Type = Int32.Type, + Int64.Type = Int64.Type, + Single.Type = Single.Type, + Double.Type = Double.Type, + Decimal.Type = Decimal.Type, + Currency.Type = Currency.Type, + Percentage.Type = Percentage.Type, + Guid.Type = Guid.Type, + Date.Type = Date.Type, + DateTime.Type = DateTime.Type, + DateTimeZone.Type = DateTimeZone.Type, + Time.Type = Time.Type, + Table.Type = Table.Type + ] + in + Source, + + #"TableSchema" = + let + ExpressionText = "type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", + BufferedExpression = List.Buffer({ExpressionText}){0}, + TableSchema = Expression.Evaluate(BufferedExpression, PQ_DataTypes) + in + TableSchema, - #"Logs JSON" = - let - Source = #"Latest Log Files", - #"Added Custom" = Table.AddColumn(Source, "JsonContent", each Lines.FromBinary([Content])), - #"Expanded JsonContent" = Table.ExpandListColumn(#"Added Custom", "JsonContent"), - #"Parsed Logs" = Table.TransformColumns(#"Expanded JsonContent",{{"JsonContent", Json.Document}}), - #"Expanded Logs" = Table.ExpandRecordColumn(#"Parsed Logs", "JsonContent", {"add", "remove", "metaData", "commitInfo", "protocol"}), - #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) - in - #"Removed Other Columns", - - #"Logs ALL" = - let - Source = Table.Combine({#"Logs Checkpoint", #"Logs JSON"}), - #"Added timestamp" = Table.AddColumn(Source, "log_timestamp", each if [add] <> null then Record.Field([add], "modificationTime") else - if [remove] <> null then Record.Field([remove], "deletionTimestamp") else - if [commitInfo] <> null then Record.Field([commitInfo], "timestamp") else - if [metaData] <> null then Record.Field([metaData], "createdTime") else null, Int64.Type), - #"Added datetime" = Table.AddColumn(#"Added timestamp", "log_datetime", each try #datetime(1970,1,1,0,0,0)+#duration(0,0,0,[log_timestamp]/1000) otherwise null, DateTime.Type) - in - #"Added datetime", - - #"metadata_columns" = - let - Source = #"Logs ALL", - #"Filtered Rows1" = Table.SelectRows(Source, each ([metaData] <> null)), - MaxVersion = Table.Group(#"Filtered Rows1", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion], - #"Filtered Rows2" = Table.SelectRows(#"Filtered Rows1", each [Version] = MaxVersion), - #"Kept First Rows" = Table.FirstN(#"Filtered Rows2",1), - #"Removed Other Columns" = Table.SelectColumns(#"Kept First Rows",{"metaData"}), - #"Expanded metaData" = Table.ExpandRecordColumn(#"Removed Other Columns", "metaData", {"schemaString", "partitionColumns"}, {"schemaString", "partitionColumns"}), - #"Filtered Rows" = Table.SelectRows(#"Expanded metaData", each ([schemaString] <> null)), - JSON = Table.TransformColumns(#"Filtered Rows",{{"schemaString", Json.Document}}), - #"Expanded schemaString" = Table.ExpandRecordColumn(JSON, "schemaString", {"fields"}, {"fields"}), - #"Expanded fields" = Table.ExpandListColumn(#"Expanded schemaString", "fields"), - #"Expanded fields1" = Table.ExpandRecordColumn(#"Expanded fields", "fields", {"name", "type", "nullable", "metadata"}, {"name", "type", "nullable", "metadata"}), - #"Added Custom" = Table.Buffer(Table.AddColumn(#"Expanded fields1", "isPartitionedBy", each List.Contains([partitionColumns], [name]), Logical.Type)), - #"Added Custom1" = Table.AddColumn(#"Added Custom", "PBI_DataType", - each if [type] = "long" then [PBI_DataType=Int64.Type, PBI_Text="Int64.Type", PBI_Transformation=Int64.From] - else if [type] = "integer" then [PBI_DataType=Int32.Type, PBI_Text="Int32.Type", PBI_Transformation=Int32.From] - else if [type] = "short" then [PBI_DataType=Int16.Type, PBI_Text="Int16.Type", PBI_Transformation=Int16.From] - else if [type] = "byte" then [PBI_DataType=Int8.Type, PBI_Text="Int8.Type", PBI_Transformation=Int8.From] - else if [type] = "float" then [PBI_DataType=Single.Type, PBI_Text="Single.Type", PBI_Transformation=Single.From] - else if [type] = "double" then [PBI_DataType=Double.Type, PBI_Text="Double.Type", PBI_Transformation=Double.From] - else if [type] = "string" then [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From] - else if [type] = "timestamp" then [PBI_DataType=DateTime.Type, PBI_Text="DateTime.Type", PBI_Transformation=DateTime.From] - else if [type] = "boolean" then [PBI_DataType=Logical.Type, PBI_Text="Logical.Type", PBI_Transformation=Logical.From] - else [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From]), - #"Expanded PBI_DataType" = Table.ExpandRecordColumn(#"Added Custom1", "PBI_DataType", {"PBI_DataType", "PBI_Text", "PBI_Transformation"}, {"PBI_DataType", "PBI_Text", "PBI_Transformation"}), - #"Added Custom2" = Table.AddColumn(#"Expanded PBI_DataType", "ChangeDataType", each {[name], [PBI_DataType]}, type list), - #"Added Custom3" = Table.AddColumn(#"Added Custom2", "TableDataType", each [name] & "=" & (if [nullable] then "nullable " else "") & Text.From([PBI_Text]), type text), - #"Added Custom4" = Table.AddColumn(#"Added Custom3", "ColumnTransformation", each {[name], [PBI_Transformation]}, type list), - #"Buffered Fields" = Table.Buffer(#"Added Custom4") - in - #"Buffered Fields", - - #"Data" = + #"_delta_log Folder" = + let + Source = DeltaTableFolderContent_wFullPath, + #"Filtered Rows" = Table.SelectRows(Source, each Text.Contains([Full_Path], Delimiter & "_delta_log" & Delimiter)), + #"Added Version" = Table.AddColumn(#"Filtered Rows", "Version", each try Int64.From(Text.BeforeDelimiter([File_Name], ".")) otherwise -1, Int64.Type), + #"Filtered RequestedVersion" = if DeltaTableVersion = null then #"Added Version" else Table.SelectRows(#"Added Version", each [Version] <= DeltaTableVersion), + BufferedTable = Table.Buffer(#"Filtered RequestedVersion"), + BufferedContent = Table.TransformColumns(BufferedTable,{{"Content", Binary.Buffer}}) + in + BufferedContent, + + #"DeltaTablePath" = + let + DeltaTablePath = Text.Combine(List.RemoveLastN(Text.Split(#"_delta_log Folder"{0}[Full_Path], Delimiter), 2), Delimiter) & Delimiter + in + DeltaTablePath, + + #"_last_checkpoint" = + let + #"_delta_log" = #"_delta_log Folder", + #"Filtered Rows" = Table.SelectRows(_delta_log, each Text.EndsWith([Name], "_last_checkpoint")), + #"Added Custom" = Table.AddColumn(#"Filtered Rows", "JsonContent", each Json.Document([Content])), + JsonContent = #"Added Custom"{0}[JsonContent], + CheckEmpty = if Table.RowCount(#"Filtered Rows") = 0 then [Size=-1, version=-1] else JsonContent, + LatestCheckPointWithParts = if Record.HasFields(CheckEmpty, "parts") then CheckEmpty else Record.AddField(CheckEmpty, "parts", 1), + + #"Filtered Rows Version" = Table.SelectRows(#"_delta_log", each Text.EndsWith([Name], ".checkpoint.parquet")), + MaxVersion = try Table.Group(#"Filtered Rows Version", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion] otherwise -1, + #"Filtered Rows MaxVersion" = Table.SelectRows(#"Filtered Rows Version", each [Version] = MaxVersion), + CheckpointFromVersion = [version=try MaxVersion otherwise -1, size=-1, parts = Table.RowCount(#"Filtered Rows MaxVersion")], + + LastCheckpoint = Table.Buffer(Table.FromRecords({if DeltaTableVersion = null then LatestCheckPointWithParts else CheckpointFromVersion})){0} + in + LastCheckpoint, + + #"Checkpoint Files" = + let + LastCheckpointFile = {1..Record.Field(_last_checkpoint, "parts")}, + #"Converted to Table" = Table.FromList(LastCheckpointFile, Splitter.SplitByNothing(), {"part"}, null, ExtraValues.Error), + #"Add Version" = Table.AddColumn(#"Converted to Table", "version", each Record.Field(_last_checkpoint, "version")), + #"Add SingleFile" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint.parquet", Text.Type), + #"Add MultipleFiles" = Table.AddColumn(#"Add Version", "file_name", each Text.PadStart(Text.From([version]), 20, "0") & ".checkpoint." & Text.PadStart(Text.From([part]), 10, "0") & "." & Text.PadStart(Text.From(Record.Field(_last_checkpoint, "parts")), 10, "0") & ".parquet", Text.Type), + AllFiles = Table.SelectColumns(if Record.Field(_last_checkpoint, "parts") = 1 then #"Add SingleFile" else #"Add MultipleFiles", "file_name"), + AllFiles_BufferedList = List.Buffer(Table.ToList(AllFiles)), + Content = Table.SelectRows(#"_delta_log Folder", each List.Count(List.Select(AllFiles_BufferedList, (inner) => Text.EndsWith([Name], inner))) > 0) + in + Content, + + #"Logs Checkpoint" = + let + Source = #"Checkpoint Files", + #"Parsed Logs" = Table.AddColumn(Source, "Custom", each Parquet.Document([Content])), + #"Expanded Logs" = Table.ExpandTableColumn(#"Parsed Logs", "Custom", {"add", "remove", "metaData", "commitInfo", "protocol"}, {"add", "remove", "metaData", "commitInfo", "protocol"}), + #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) + in + #"Removed Other Columns", + + #"Latest Log Files" = + let + Source = #"_delta_log Folder", + #"Filtered Rows" = Table.SelectRows(Source, each ([Extension] = ".json")), + #"Filtered Rows1" = Table.SelectRows(#"Filtered Rows", each [Version] > Record.Field(_last_checkpoint, "version")) + in + #"Filtered Rows1", + + #"Logs JSON" = + let + Source = #"Latest Log Files", + #"Added Custom" = Table.AddColumn(Source, "JsonContent", each Lines.FromBinary([Content])), + #"Expanded JsonContent" = Table.ExpandListColumn(#"Added Custom", "JsonContent"), + #"Parsed Logs" = Table.TransformColumns(#"Expanded JsonContent",{{"JsonContent", Json.Document}}), + #"Expanded Logs" = Table.ExpandRecordColumn(#"Parsed Logs", "JsonContent", {"add", "remove", "metaData", "commitInfo", "protocol"}), + #"Removed Other Columns" = Table.SelectColumns(#"Expanded Logs",{"Version", "add", "remove", "metaData", "commitInfo", "protocol"}) + in + #"Removed Other Columns", + + #"Logs ALL" = + let + Source = Table.Combine({#"Logs Checkpoint", #"Logs JSON"}), + #"Added timestamp" = Table.AddColumn(Source, "log_timestamp", each if [add] <> null then Record.Field([add], "modificationTime") else + if [remove] <> null then Record.Field([remove], "deletionTimestamp") else + if [commitInfo] <> null then Record.Field([commitInfo], "timestamp") else + if [metaData] <> null then Record.Field([metaData], "createdTime") else null, Int64.Type), + #"Added datetime" = Table.AddColumn(#"Added timestamp", "log_datetime", each try #datetime(1970,1,1,0,0,0)+#duration(0,0,0,[log_timestamp]/1000) otherwise null, DateTime.Type) + in + #"Added datetime", + + #"metadata_columns" = + let + Source = #"Logs ALL", + #"Filtered Rows1" = Table.SelectRows(Source, each ([metaData] <> null)), + MaxVersion = Table.Group(#"Filtered Rows1", {}, {{"MaxVersion", each List.Max([Version]), type number}}){0}[MaxVersion], + #"Filtered Rows2" = Table.SelectRows(#"Filtered Rows1", each [Version] = MaxVersion), + #"Kept First Rows" = Table.FirstN(#"Filtered Rows2",1), + #"Removed Other Columns" = Table.SelectColumns(#"Kept First Rows",{"metaData"}), + #"Expanded metaData" = Table.ExpandRecordColumn(#"Removed Other Columns", "metaData", {"schemaString", "partitionColumns"}, {"schemaString", "partitionColumns"}), + #"Filtered Rows" = Table.SelectRows(#"Expanded metaData", each ([schemaString] <> null)), + JSON = Table.TransformColumns(#"Filtered Rows",{{"schemaString", Json.Document}}), + #"Expanded schemaString" = Table.ExpandRecordColumn(JSON, "schemaString", {"fields"}, {"fields"}), + #"Expanded fields" = Table.ExpandListColumn(#"Expanded schemaString", "fields"), + #"Expanded fields1" = Table.ExpandRecordColumn(#"Expanded fields", "fields", {"name", "type", "nullable", "metadata"}, {"name", "type", "nullable", "metadata"}), + #"Added Custom" = Table.Buffer(Table.AddColumn(#"Expanded fields1", "isPartitionedBy", each List.Contains([partitionColumns], [name]), Logical.Type)), + #"Added Custom1" = Table.AddColumn(#"Added Custom", "PBI_DataType", + each if [type] = "long" then [PBI_DataType=Int64.Type, PBI_Text="Int64.Type", PBI_Transformation=Int64.From] + else if [type] = "integer" then [PBI_DataType=Int32.Type, PBI_Text="Int32.Type", PBI_Transformation=Int32.From] + else if [type] = "short" then [PBI_DataType=Int16.Type, PBI_Text="Int16.Type", PBI_Transformation=Int16.From] + else if [type] = "byte" then [PBI_DataType=Int8.Type, PBI_Text="Int8.Type", PBI_Transformation=Int8.From] + else if [type] = "float" then [PBI_DataType=Single.Type, PBI_Text="Single.Type", PBI_Transformation=Single.From] + else if [type] = "double" then [PBI_DataType=Double.Type, PBI_Text="Double.Type", PBI_Transformation=Double.From] + else if [type] = "string" then [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From] + else if [type] = "timestamp" then [PBI_DataType=DateTime.Type, PBI_Text="DateTime.Type", PBI_Transformation=DateTime.From] + else if [type] = "boolean" then [PBI_DataType=Logical.Type, PBI_Text="Logical.Type", PBI_Transformation=Logical.From] + else [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From]), + #"Expanded PBI_DataType" = Table.ExpandRecordColumn(#"Added Custom1", "PBI_DataType", {"PBI_DataType", "PBI_Text", "PBI_Transformation"}, {"PBI_DataType", "PBI_Text", "PBI_Transformation"}), + #"Added Custom2" = Table.AddColumn(#"Expanded PBI_DataType", "ChangeDataType", each {[name], [PBI_DataType]}, type list), + #"Added Custom3" = Table.AddColumn(#"Added Custom2", "TableDataType", each [name] & "=" & (if [nullable] then "nullable " else "") & Text.From([PBI_Text]), type text), + #"Added Custom4" = Table.AddColumn(#"Added Custom3", "ColumnTransformation", each {[name], [PBI_Transformation]}, type list), + #"Buffered Fields" = Table.Buffer(#"Added Custom4") + in + #"Buffered Fields", + + #"Data" = let - Source = #"Logs ALL", - #"Added Counter" = Table.AddColumn(Source, "Counter", each if [remove] <> null then -1 else if [add] <> null then 1 else null, Int8.Type), - #"Added file_name" = Table.AddColumn(#"Added Counter", "file_name", each if [add] <> null then Record.Field([add], "path") else if [remove] <> null then Record.Field([remove], "path") else null, Text.Type), - #"Filtered Rows" = Table.SelectRows(#"Added file_name", each ([file_name] <> null)), - #"Added partitionValuesTable" = Table.AddColumn(#"Filtered Rows", "partitionValuesTable", each if [add] <> null then if Value.Is(Record.Field([add], "partitionValues"), Record.Type) then Record.ToTable(Record.Field([add], "partitionValues")) else Table.RenameColumns(Record.Field([add], "partitionValues"), {"Key", "Name"}) else null, type nullable table), - #"Added partitionValuesJSON" = Table.AddColumn(#"Added partitionValuesTable", "partitionValuesJSON", each Text.FromBinary(Json.FromValue([partitionValuesTable]))), - #"Grouped Rows1" = Table.Group(#"Added partitionValuesJSON", {"file_name"}, {{"partitionValuesJSON", each List.Max([partitionValuesJSON]), type nullable text}, {"isRelevant", each List.Sum([Counter]), type nullable text}}), - #"Relevant Files" = Table.SelectRows(#"Grouped Rows1", each ([isRelevant] > 0)), - #"Added partitionValuesTable2" = Table.AddColumn(#"Relevant Files", "partitionValuesTable", each try Table.FromRecords(Json.Document([partitionValuesJSON])) otherwise null), - #"Added partitionValuesRecord" = Table.AddColumn(#"Added partitionValuesTable2", "partitionValuesRecord", each Record.TransformFields(Record.FromTable([partitionValuesTable]), Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[ColumnTransformation]), Expression.Evaluate("type [" & Text.Combine(Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[TableDataType], ", ") & "]", PQ_DataTypes)), - #"Filtered Rows1" = Table.SelectRows(#"Added partitionValuesRecord", each PartitionFilterFunction([partitionValuesRecord])), - #"Expanded partitionValuesRecord" = Table.ExpandRecordColumn(#"Filtered Rows1", "partitionValuesRecord", Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[name]), - #"Added Full_Path" = Table.AddColumn(#"Expanded partitionValuesRecord", "Full_Path", each Text.Replace(DeltaTablePath & Text.Replace([file_name], "=", "%3D"), "/", Delimiter), Text.Type), - #"Removed Columns3" = Table.RemoveColumns(#"Added Full_Path",{"file_name", "partitionValuesJSON", "isRelevant", "partitionValuesTable"}), - #"Buffered RelevantFiles" = Table.Buffer(#"Removed Columns3"), - #"Merged Queries" = Table.NestedJoin(#"Buffered RelevantFiles", {"Full_Path"}, DeltaTableFolderContent_wFullPath, {"Full_Path"}, "DeltaTable Folder", JoinKind.Inner), - #"Removed Columns" = Table.RemoveColumns(#"Merged Queries",{"Full_Path"}), - #"Expanded DeltaTable Folder" = Table.ExpandTableColumn(#"Removed Columns", "DeltaTable Folder", {"Content"}, {"Content"}), - BufferFile = if UseFileBuffer then Table.TransformColumns(#"Expanded DeltaTable Folder",{{"Content", Binary.Buffer}}) else #"Expanded DeltaTable Folder", - #"Added Custom1" = Table.AddColumn(BufferFile, "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", PQ_DataTypes)), - #"Removed Columns1" = Table.RemoveColumns(#"Added Custom1",{"Content"}), - #"Expanded Data" = Table.ExpandTableColumn(#"Removed Columns1", "Data", Table.SelectRows(metadata_columns, each not [isPartitionedBy])[name]), - #"Reordered Columns" = Table.ReorderColumns(#"Expanded Data", metadata_columns[name]) - in - #"Reordered Columns" - -in #"Data" \ No newline at end of file + Source = #"Logs ALL", + #"Added Counter" = Table.AddColumn(Source, "Counter", each if [remove] <> null then -1 else if [add] <> null then 1 else null, Int8.Type), + #"Added file_name" = Table.AddColumn(#"Added Counter", "file_name", each if [add] <> null then Record.Field([add], "path") else if [remove] <> null then Record.Field([remove], "path") else null, Text.Type), + #"Filtered Rows" = Table.SelectRows(#"Added file_name", each ([file_name] <> null)), + #"Added partitionValuesTable" = Table.AddColumn(#"Filtered Rows", "partitionValuesTable", each if [add] <> null then if Value.Is(Record.Field([add], "partitionValues"), Record.Type) then Record.ToTable(Record.Field([add], "partitionValues")) else Table.RenameColumns(Record.Field([add], "partitionValues"), {"Key", "Name"}) else null, type nullable table), + #"Added partitionValuesJSON" = Table.AddColumn(#"Added partitionValuesTable", "partitionValuesJSON", each Text.FromBinary(Json.FromValue([partitionValuesTable]))), + #"Grouped Rows1" = Table.Group(#"Added partitionValuesJSON", {"file_name"}, {{"partitionValuesJSON", each List.Max([partitionValuesJSON]), type nullable text}, {"isRelevant", each List.Sum([Counter]), type nullable text}}), + #"Relevant Files" = Table.SelectRows(#"Grouped Rows1", each ([isRelevant] > 0)), + #"Added partitionValuesTable2" = Table.AddColumn(#"Relevant Files", "partitionValuesTable", each try Table.FromRecords(Json.Document([partitionValuesJSON])) otherwise null), + #"Added partitionValuesRecord" = Table.AddColumn(#"Added partitionValuesTable2", "partitionValuesRecord", each Record.TransformFields(Record.FromTable([partitionValuesTable]), Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[ColumnTransformation]), Expression.Evaluate("type [" & Text.Combine(Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[TableDataType], ", ") & "]", PQ_DataTypes)), + #"Filtered Rows1" = Table.SelectRows(#"Added partitionValuesRecord", each PartitionFilterFunction([partitionValuesRecord])), + #"Expanded partitionValuesRecord" = Table.ExpandRecordColumn(#"Filtered Rows1", "partitionValuesRecord", Table.SelectRows(#"metadata_columns", each [isPartitionedBy] = true)[name]), + #"Added Full_Path" = Table.AddColumn(#"Expanded partitionValuesRecord", "Full_Path", each Text.Replace(DeltaTablePath & Text.Replace([file_name], "=", "%3D"), "/", Delimiter), Text.Type), + #"Removed Columns3" = Table.RemoveColumns(#"Added Full_Path",{"file_name", "partitionValuesJSON", "isRelevant", "partitionValuesTable"}), + #"Buffered RelevantFiles" = Table.Buffer(#"Removed Columns3"), + #"Merged Queries" = Table.NestedJoin(#"Buffered RelevantFiles", {"Full_Path"}, DeltaTableFolderContent_wFullPath, {"Full_Path"}, "DeltaTable Folder", JoinKind.Inner), + #"Removed Columns" = Table.RemoveColumns(#"Merged Queries",{"Full_Path"}), + #"Expanded DeltaTable Folder" = Table.ExpandTableColumn(#"Removed Columns", "DeltaTable Folder", {"Content"}, {"Content"}), + BufferFile = if UseFileBuffer then Table.TransformColumns(#"Expanded DeltaTable Folder",{{"Content", Binary.Buffer}}) else #"Expanded DeltaTable Folder", + #"Added Custom1" = Table.AddColumn(BufferFile, "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", PQ_DataTypes)), + #"Removed Columns1" = Table.RemoveColumns(#"Added Custom1",{"Content"}), + #"Expanded Data" = Table.ExpandTableColumn(#"Removed Columns1", "Data", Table.SelectRows(metadata_columns, each not [isPartitionedBy])[name]), + #"Reordered Columns" = Table.ReorderColumns(#"Expanded Data", metadata_columns[name]) + in + #"Reordered Columns" + + in + #"Data", + + documentation = [ + Documentation.Name = "fn_ReadDeltaTable", + Documentation.Description = "Takes the file/folder list of a Delta Lake table and returns the content as a table object in Power Query.", + Documentation.LongDescription = "Takes the file/folder list of a Delta Lake table and returns the content as a table object in Power Query. An optional 2nd parameter can be used to for special features like Time Travel, Partition Elimination, etc.", + Documentation.Category = "Table", + Documentation.Source = "https://github.com/delta-io/connectors/blob/master/powerbi/fn_ReadDeltaTable.pq", + Documentation.Version = "1.0", + Documentation.Author = "Gerhard Brueckl, paiqo GmbH", + Documentation.Examples = {[Description = " ", + Code = "let + Source = AzureStorage.Blobs(""https://gbadls01.blob.core.windows.net/public""), + #""Filtered Rows"" = Table.SelectRows(Source, each Text.StartsWith([Name], ""powerbi_delta/FactInternetSales_part.delta/"")), + DeltaTable = fn_ReadDeltaTable(#""Filtered Rows"", [Version=7]) +in + DeltaTable", + Result = "#table( {""ProductKey"", ""OrderDateKey"", ""Value""}, { {""A"", ""2020-01-01"", 123} ,{""B"", ""2020-04-02"", 45} } )"]}] + +in + Value.ReplaceType(fn_ReadDeltaTable, Value.ReplaceMetadata(Value.Type(fn_ReadDeltaTable), documentation)) \ No newline at end of file From 4b7b176aee8b173416c22d1d21d0786346337883 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Thu, 29 Jul 2021 18:06:42 -0700 Subject: [PATCH 063/291] [SC-80783] Change OSS Connectors repo to use Sonatype for the release (#105) Update build.sbt to use sonatype and not bintray, as bintray is sunset. Made changes similar to those made to delta-io/delta when the same thing was done (https://github.com/delta-io/delta/commit/926e30da6b6e86448cdb4739a082e0b566eec129) Tested by staging a sonatype release. Changed the `version.sbt` to be `0.2.1-SNAPSHOT-sonatype-test` and was able to see that new package version in the sonatype staging repository. The pgp keys were present, too. --- build.sbt | 35 ++++++++++++++--------------------- project/plugins.sbt | 21 +++++++-------------- 2 files changed, 21 insertions(+), 35 deletions(-) diff --git a/build.sbt b/build.sbt index d5400f2dae8..2d7a4138c54 100644 --- a/build.sbt +++ b/build.sbt @@ -53,6 +53,15 @@ lazy val commonSettings = Seq( lazy val releaseSettings = Seq( publishMavenStyle := true, releaseCrossBuild := true, + publishTo := { + val nexus = "https://oss.sonatype.org/" + if (isSnapshot.value) { + Some("snapshots" at nexus + "content/repositories/snapshots") + } else { + Some("releases" at nexus + "service/local/staging/deploy/maven2") + } + }, + releasePublishArtifactsAction := PgpKeys.publishSigned.value, licenses += ("Apache-2.0", url("http://www.apache.org/licenses/LICENSE-2.0")), pomExtra := https://github.com/delta-io/connectors @@ -82,8 +91,6 @@ lazy val releaseSettings = Seq( https://github.com/zsxwing , - bintrayOrganization := Some("delta-io"), - bintrayRepository := "delta", releaseProcess := Seq[ReleaseStep]( checkSnapshotDependencies, inquireVersions, @@ -91,7 +98,7 @@ lazy val releaseSettings = Seq( setReleaseVersion, commitReleaseVersion, tagRelease, - publishArtifacts, + releaseStepCommandAndRemaining("+publishLocalSigned"), setNextVersion, commitNextVersion ) @@ -102,25 +109,11 @@ lazy val skipReleaseSettings = Seq( publish := () ) -// Don't release the root project -publishArtifact := false - -publish := () - // Looks some of release settings should be set for the root project as well. -releaseCrossBuild := true - -releaseProcess := Seq[ReleaseStep]( - checkSnapshotDependencies, - inquireVersions, - runTest, - setReleaseVersion, - commitReleaseVersion, - tagRelease, - publishArtifacts, - setNextVersion, - commitNextVersion -) +publishArtifact := false // Don't release the root project +publish := {} +publishTo := Some("snapshots" at "https://oss.sonatype.org/content/repositories/snapshots") +releaseCrossBuild := false lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( name := "delta-hive", diff --git a/project/plugins.sbt b/project/plugins.sbt index ad705fd5d68..9af949c1a14 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -14,20 +14,7 @@ * limitations under the License. */ - -resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) - -resolvers += "bintray-spark-packages" at "https://dl.bintray.com/spark-packages/maven/" - -resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" - -resolvers += Resolver.url( - "typesafe sbt-plugins", - url("https://dl.bintray.com/typesafe/sbt-plugins"))(Resolver.ivyStylePatterns) - -addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.13") - -addSbtPlugin("org.foundweekends" % "sbt-bintray" % "0.5.6") +addSbtPlugin("com.github.sbt" % "sbt-release" % "1.0.15") addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.9") @@ -39,3 +26,9 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.10.0-RC1") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.4.2") +// changing this to addSbtPlugin("com.github.sbt" % "sbt-pgp" % "2.1.2") +// will cause issues with release +addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") + +addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") + From 338ccde72a46acf9546ae0e9baf64e5c63fee625 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 2 Aug 2021 11:24:36 -0700 Subject: [PATCH 064/291] [#101] add incremental changes api (#102) - resolves #101 - adds `DeltaLog::getChanges` API. Exposes an iterator of `VersionDelta`s, which is a java wrapper class containing the version and actions. - includes `DeltaLog.java` public interface - includes `DeltaLogImpl.scala` internal implementation - adds remaining `Actions` missing from Delta OSS (RemoveFile, AddCDCFIle, Protocol, SetTransaction) as both internal scala representations and public java interfaces - includes tests in `DeltaLogSuite.scala` - increases goldenTables project delta oss version to 0.8.0 (to get access to AddCDCFile) --- README.md | 3 + build.sbt | 10 +- .../_delta_log/00000000000000000000.json | 4 + .../_delta_log/00000000000000000001.json | 3 + .../_delta_log/00000000000000000002.json | 3 + .../scala/io/delta/golden/GoldenTables.scala | 23 +++- .../java/io/delta/standalone/DeltaLog.java | 14 +++ .../java/io/delta/standalone/VersionLog.java | 34 ++++++ .../io/delta/standalone/actions/Action.java | 21 ++++ .../delta/standalone/actions/AddCDCFile.java | 39 ++++++ .../io/delta/standalone/actions/AddFile.java | 6 +- .../delta/standalone/actions/CommitInfo.java | 2 +- .../delta/standalone/actions/FileAction.java | 8 ++ .../io/delta/standalone/actions/Format.java | 2 +- .../io/delta/standalone/actions/JobInfo.java | 2 +- .../io/delta/standalone/actions/Metadata.java | 2 +- .../standalone/actions/NotebookInfo.java | 2 +- .../io/delta/standalone/actions/Protocol.java | 20 ++++ .../delta/standalone/actions/RemoveFile.java | 57 +++++++++ .../standalone/actions/SetTransaction.java | 27 +++++ .../io/delta/standalone/types/StructType.java | 4 - .../standalone/internal/DeltaLogImpl.scala | 32 ++++- .../standalone/internal/SnapshotImpl.scala | 11 +- .../standalone/internal/actions/actions.scala | 113 +++++++++++++++++- .../internal/exception/DeltaErrors.scala | 13 ++ .../internal/util/ConversionUtils.scala | 48 +++++++- .../standalone/internal/DeltaLogSuite.scala | 68 +++++++++++ .../internal/util/GoldenTableUtils.scala | 10 +- 28 files changed, 547 insertions(+), 34 deletions(-) create mode 100644 golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000001.json create mode 100644 golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000002.json create mode 100644 standalone/src/main/java/io/delta/standalone/VersionLog.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/Action.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/FileAction.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/Protocol.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java create mode 100644 standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java diff --git a/README.md b/README.md index 353399aed3a..c7b5e7e0a37 100644 --- a/README.md +++ b/README.md @@ -188,3 +188,6 @@ There are two mediums of communication within the Delta Lake community. - [Login here](https://delta-users.slack.com/) - Public [Mailing list](https://groups.google.com/forum/#!forum/delta-users) + +# Local Development & Testing +- Before local debugging of `standalone` tests in IntelliJ, run all `standalone` tests using SBT. This helps IntelliJ recognize the golden tables as class resources. diff --git a/build.sbt b/build.sbt index 2d7a4138c54..24dc33a122a 100644 --- a/build.sbt +++ b/build.sbt @@ -26,7 +26,7 @@ lazy val testScalastyle = taskKey[Unit]("testScalastyle") val sparkVersion = "2.4.3" val hadoopVersion = "2.7.2" val hiveVersion = "2.3.7" -val deltaVersion = "0.5.0" +val hiveDeltaVersion = "0.5.0" lazy val commonSettings = Seq( organization := "io.delta", @@ -145,7 +145,7 @@ lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( ExclusionRule(organization = "com.google.protobuf") ), "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % deltaVersion % "test", + "io.delta" %% "delta-core" % hiveDeltaVersion % "test", "org.apache.spark" %% "spark-sql" % sparkVersion % "test", "org.apache.spark" %% "spark-catalyst" % sparkVersion % "test" classifier "tests", "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", @@ -191,7 +191,7 @@ lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") se // TODO Figure out how this fixes some bad dependency "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % deltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") + "io.delta" %% "delta-core" % hiveDeltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") ) ) @@ -239,7 +239,7 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") // TODO Figure out how this fixes some bad dependency "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % deltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") + "io.delta" %% "delta-core" % hiveDeltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") ) ) @@ -298,7 +298,7 @@ lazy val goldenTables = (project in file("golden-tables")) settings ( // Test Dependencies "org.scalatest" %% "scalatest" % "3.0.5" % "test", "org.apache.spark" % "spark-sql_2.12" % "3.0.0" % "test", - "io.delta" % "delta-core_2.12" % "0.7.0" % "test", + "io.delta" % "delta-core_2.12" % "0.8.0" % "test", "commons-io" % "commons-io" % "2.8.0" % "test", "org.apache.spark" % "spark-catalyst_2.12" % "3.0.0" % "test" classifier "tests", "org.apache.spark" % "spark-core_2.12" % "3.0.0" % "test" classifier "tests", diff --git a/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..16b327da299 --- /dev/null +++ b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1626806331480,"operation":"Manual Update","operationParameters":{},"isBlindAppend":true,"operationMetrics":{}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"partitionColumns":[],"configuration":{},"createdTime":1626806331460}} +{"add":{"path":"fake/path/1","partitionValues":{},"size":1,"modificationTime":1,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000001.json b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..ab4e3d6146c --- /dev/null +++ b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1626806336805,"operation":"Manual Update","operationParameters":{},"readVersion":0,"isBlindAppend":false,"operationMetrics":{}}} +{"cdc":{"path":"fake/path/2","partitionValues":{"partition_foo":"partition_bar"},"size":1,"tags":{"tag_foo":"tag_bar"},"dataChange":false}} +{"remove":{"path":"fake/path/1","deletionTimestamp":100,"dataChange":true,"extendedFileMetadata":false,"size":0}} diff --git a/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000002.json b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000..3a16fe9975f --- /dev/null +++ b/golden-tables/src/test/resources/golden/deltalog-getChanges/_delta_log/00000000000000000002.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1626806337545,"operation":"Manual Update","operationParameters":{},"readVersion":1,"isBlindAppend":true,"operationMetrics":{}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":3}} +{"txn":{"appId":"fakeAppId","version":3,"lastUpdated":200}} diff --git a/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala b/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala index f73e8e6811c..e11b141f233 100644 --- a/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala +++ b/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala @@ -32,7 +32,7 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransaction} import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate -import org.apache.spark.sql.delta.actions.{Action, AddFile, CommitInfo, JobInfo, Metadata, NotebookInfo, Protocol, RemoveFile, SingleAction} +import org.apache.spark.sql.delta.actions.{Action, AddCDCFile, AddFile, CommitInfo, JobInfo, Metadata, NotebookInfo, Protocol, RemoveFile, SetTransaction, SingleAction} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.{FileNames, JsonUtils} import org.apache.spark.sql.test.SharedSparkSession @@ -51,7 +51,7 @@ import org.apache.spark.SparkConf * GENERATE_GOLDEN_TABLES=1 build/sbt 'goldenTables/test-only *GoldenTables -- -z tbl_name' * ``` * - * After generating golden tables, ensure to package or test project standalone`, otherwise the + * After generating golden tables, ensure to package or test project standalone, otherwise the * test resources won't be available when running tests with IntelliJ. */ class GoldenTables extends QueryTest with SharedSparkSession { @@ -374,6 +374,25 @@ class GoldenTables extends QueryTest with SharedSparkSession { Iterator(Metadata(), Protocol(), commitInfoFile, addFile).map(a => JsonUtils.toJson(a.wrap))) } + /** TEST: DeltaLogSuite > getChanges - no data loss */ + generateGoldenTable("deltalog-getChanges") { tablePath => + val log = DeltaLog.forTable(spark, new Path(tablePath)) + + val add1 = AddFile("fake/path/1", Map.empty, 1, 1, dataChange = true) + val txn1 = log.startTransaction() + txn1.commitManually(Metadata() :: add1 :: Nil: _*) + + val addCDC2 = AddCDCFile("fake/path/2", Map("partition_foo" -> "partition_bar"), 1, + Map("tag_foo" -> "tag_bar")) + val remove2 = RemoveFile("fake/path/1", Some(100), dataChange = true) + val txn2 = log.startTransaction() + txn2.commitManually(addCDC2 :: remove2 :: Nil: _*) + + val setTransaction3 = SetTransaction("fakeAppId", 3L, Some(200)) + val txn3 = log.startTransaction() + txn3.commitManually(Protocol() :: setTransaction3 :: Nil: _*) + } + /////////////////////////////////////////////////////////////////////////// // io.delta.standalone.internal.ReadOnlyLogStoreSuite /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index a87535b0396..6c15846dadb 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -21,6 +21,8 @@ import io.delta.standalone.internal.DeltaLogImpl; +import java.util.Iterator; + /** * {@link DeltaLog} is the representation of the transaction logs of a Delta table. It provides APIs * to access the states of a Delta table. @@ -75,6 +77,18 @@ public interface DeltaLog { /** @return the path of the Delta table. */ Path getPath(); + /** + * Get all actions starting from "startVersion" (inclusive). + * If `startVersion` doesn't exist, return an empty {@code Iterator}. + * + * @param startVersion the table version to begin retrieving actions from (inclusive) + * @param failOnDataLoss whether to throw when data loss detected + * @return an {@code Iterator} of {@link VersionLog}s + * @throws IllegalArgumentException if {@code startVersion} is negative + * @throws IllegalStateException if data loss detected and {@code failOnDataLoss} is true + */ + Iterator getChanges(long startVersion, boolean failOnDataLoss); + /** * Create a {@link DeltaLog} instance representing the table located at the provided {@code path}. * diff --git a/standalone/src/main/java/io/delta/standalone/VersionLog.java b/standalone/src/main/java/io/delta/standalone/VersionLog.java new file mode 100644 index 00000000000..3234616d33e --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/VersionLog.java @@ -0,0 +1,34 @@ +package io.delta.standalone; + +import io.delta.standalone.actions.Action; + +import java.util.Collections; +import java.util.List; + +/** + * {@link VersionLog} is the representation of all actions (changes) to the Delta Table + * at a specific table version. + */ +public class VersionLog { + private final long version; + private final List actions; + + public VersionLog(long version, List actions) { + this.version = version; + this.actions = actions; + } + + /** + * @return the table version at which these actions occured + */ + public long getVersion() { + return version; + } + + /** + * @return an unmodifiable {@code List} of the actions for this table version + */ + public List getActions() { + return Collections.unmodifiableList(actions); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/Action.java b/standalone/src/main/java/io/delta/standalone/actions/Action.java new file mode 100644 index 00000000000..d1e427bcc9d --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/Action.java @@ -0,0 +1,21 @@ +package io.delta.standalone.actions; + +/** + * A marker interface for all Actions that can be applied to a Delta Table. + * Each action represents a single change to the state of a Delta table. + * + * You can use the following code to extract the concrete type of an {@link Action}. + *
{@code
+ *   List actions = ... // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
+ *   actions.forEach(x -> {
+ *       if (x instanceof AddFile) {
+ *          AddFile addFile = (AddFile) x;
+ *          ...
+ *       } else if (x instanceof AddCDCFile) {
+ *          AddCDCFile addCDCFile = (AddCDCFile)x;
+ *          ...
+ *       } else if ...
+ *   });
+ * }
+ */ +public interface Action { } diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java new file mode 100644 index 00000000000..5016237e501 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java @@ -0,0 +1,39 @@ +package io.delta.standalone.actions; + +import java.util.Map; + +public class AddCDCFile implements FileAction { + private final String path; + private final Map partitionValues; + private final long size; + private final Map tags; + + public AddCDCFile(String path, Map partitionValues, long size, Map tags) { + this.path = path; + this.partitionValues = partitionValues; + this.size = size; + this.tags = tags; + } + + @Override + public String getPath() { + return path; + } + + public Map getPartitionValues() { + return partitionValues; + } + + public long getSize() { + return size; + } + + public Map getTags() { + return tags; + } + + @Override + public boolean isDataChange() { + return false; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java index 0cea60e53d1..751f7156018 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java @@ -15,8 +15,6 @@ */ package io.delta.standalone.actions; -import java.net.URI; -import java.net.URISyntaxException; import java.util.Collections; import java.util.Map; import java.util.Objects; @@ -30,7 +28,7 @@ * * @see
Delta Transaction Log Protocol */ -public final class AddFile { +public final class AddFile implements FileAction { private final String path; private final Map partitionValues; private final long size; @@ -56,6 +54,7 @@ public AddFile(String path, Map partitionValues, long size, * relative path, it's relative to the root of the table. Note: the path is encoded and * should be decoded by {@code new java.net.URI(path)} when using it. */ + @Override public String getPath() { return path; } @@ -90,6 +89,7 @@ public long getModificationTime() { * {@code false} the file must already be present in the table or the records in the * added file must be contained in one or more remove actions in the same version */ + @Override public boolean isDataChange() { return dataChange; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index a760bc5a5de..d96a9678001 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -28,7 +28,7 @@ * * @see Delta Transaction Log Protocol */ -public class CommitInfo { +public class CommitInfo implements Action { private final Optional version; private final Timestamp timestamp; private final Optional userId; diff --git a/standalone/src/main/java/io/delta/standalone/actions/FileAction.java b/standalone/src/main/java/io/delta/standalone/actions/FileAction.java new file mode 100644 index 00000000000..221384e6d04 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/FileAction.java @@ -0,0 +1,8 @@ +package io.delta.standalone.actions; + +public interface FileAction extends Action { + + String getPath(); + + boolean isDataChange(); +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/Format.java b/standalone/src/main/java/io/delta/standalone/actions/Format.java index 39c8e120957..98244ab59b7 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Format.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Format.java @@ -24,7 +24,7 @@ * * @see Delta Transaction Log Protocol */ -public final class Format { +public final class Format implements Action { private final String provider; private final Map options; diff --git a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java index 617d273148d..64b02c440e7 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java @@ -18,7 +18,7 @@ import java.util.Objects; /** Represents the Databricks Job information that committed to the Delta table. */ -public class JobInfo { +public class JobInfo implements Action { private final String jobId; private final String jobName; private final String runId; diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index dd03eda3350..928434b7230 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -29,7 +29,7 @@ * * @see Delta Transaction Log Protocol */ -public final class Metadata { +public final class Metadata implements Action { private final String id; private final String name; private final String description; diff --git a/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java b/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java index 507a3df7658..4d1962ad6b4 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java @@ -18,7 +18,7 @@ import java.util.Objects; /** Represents the Databricks Notebook information that committed to the Delta table. */ -public class NotebookInfo { +public class NotebookInfo implements Action { private final String notebookId; public NotebookInfo(String notebookId) { diff --git a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java new file mode 100644 index 00000000000..1a421ccede9 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java @@ -0,0 +1,20 @@ +package io.delta.standalone.actions; + +public class Protocol implements Action { + private final int minReaderVersion; + private final int minWriterVersion; + + public Protocol(int minReaderVersion, int minWriterVersion) { + this.minReaderVersion = minReaderVersion; + this.minWriterVersion = minWriterVersion; + } + + public int getMinReaderVersion() { + return minReaderVersion; + } + + public int getMinWriterVersion() { + return minWriterVersion; + } +} + diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java new file mode 100644 index 00000000000..f3ad9d1fff9 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java @@ -0,0 +1,57 @@ +package io.delta.standalone.actions; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; + +public class RemoveFile implements FileAction { + private final String path; + private final Optional deletionTimestamp; + private final boolean dataChange; + private final boolean extendedFileMetadata; + private final Map partitionValues; + private final long size; + private final Map tags; + + public RemoveFile(String path, Optional deletionTimestamp, boolean dataChange, + boolean extendedFileMetadata, Map partitionValues, long size, + Map tags) { + this.path = path; + this.deletionTimestamp = deletionTimestamp; + this.dataChange = dataChange; + this.extendedFileMetadata = extendedFileMetadata; + this.partitionValues = partitionValues; + this.size = size; + this.tags = tags; + } + + @Override + public String getPath() { + return path; + } + + public Optional getDeletionTimestamp() { + return deletionTimestamp; + } + + @Override + public boolean isDataChange() { + return dataChange; + } + + public boolean isExtendedFileMetadata() { + return extendedFileMetadata; + } + + public Map getPartitionValues() { + return Collections.unmodifiableMap(partitionValues); + } + + public long getSize() { + return size; + } + + public Map getTags() { + return Collections.unmodifiableMap(tags); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java new file mode 100644 index 00000000000..64ef81a9c8d --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java @@ -0,0 +1,27 @@ +package io.delta.standalone.actions; + +import java.util.Optional; + +public class SetTransaction implements Action { + private final String appId; + private final long verion; + private final Optional lastUpdated; + + public SetTransaction(String appId, long verion, Optional lastUpdated) { + this.appId = appId; + this.verion = verion; + this.lastUpdated = lastUpdated; + } + + public String getAppId() { + return appId; + } + + public long getVerion() { + return verion; + } + + public Optional getLastUpdated() { + return lastUpdated; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index de8d0afb4a9..544c2b77740 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -52,10 +52,6 @@ public final class StructType extends DataType { private final HashMap nameToField; public StructType(StructField[] fields) { - if (fields.length == 0) { - throw new IllegalArgumentException("a StructType must have at least one field"); - } - this.fields = fields; this.nameToField = new HashMap<>(); Arrays.stream(fields).forEach(field -> nameToField.put(field.getName(), field)); diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index f1c79f285b7..1a29aef6e33 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -16,15 +16,18 @@ package io.delta.standalone.internal -import java.io.File import java.util.concurrent.locks.ReentrantLock +import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import io.delta.standalone.DeltaLog +import io.delta.standalone.{DeltaLog, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} +import io.delta.standalone.internal.actions.Action +import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore -import io.delta.standalone.internal.util.ConversionUtils +import io.delta.standalone.internal.util.{ConversionUtils, FileNames} /** * Scala implementation of Java interface [[DeltaLog]]. @@ -53,6 +56,29 @@ private[internal] class DeltaLogImpl private( ConversionUtils.convertCommitInfo(history.getCommitInfo(version)) } + override def getChanges( + startVersion: Long, + failOnDataLoss: Boolean): java.util.Iterator[VersionLog] = { + if (startVersion < 0) throw new IllegalArgumentException(s"Invalid startVersion: $startVersion") + + val deltaPaths = store.listFrom(FileNames.deltaFile(logPath, startVersion)) + .filter(f => FileNames.isDeltaFile(f.getPath)) + + // Subtract 1 to ensure that we have the same check for the inclusive startVersion + var lastSeenVersion = startVersion - 1 + deltaPaths.map { status => + val p = status.getPath + val version = FileNames.deltaVersion(p) + if (failOnDataLoss && version > lastSeenVersion + 1) { + throw DeltaErrors.failOnDataLossException(lastSeenVersion + 1, version) + } + lastSeenVersion = version + + new VersionLog(version, + store.read(p).map(x => ConversionUtils.convertAction(Action.fromJson(x))).toList.asJava) + }.asJava + } + /** * Run `body` inside `deltaLogLock` lock using `lockInterruptibly` so that the thread can be * interrupted when waiting for the lock. diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index d0160c7dedb..14afd9552c1 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -24,11 +24,10 @@ import scala.collection.JavaConverters._ import com.github.mjakubowski84.parquet4s.ParquetReader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} - -import io.delta.standalone.{DeltaLog, Snapshot} +import io.delta.standalone.Snapshot import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} -import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Protocol, SingleAction} +import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.sources.StandaloneHadoopConf @@ -95,8 +94,10 @@ private[internal] class SnapshotImpl( JsonUtils.mapper.readValue[SingleAction](line) } } else if (path.endsWith("parquet")) { - ParquetReader.read[SingleAction](path, ParquetReader.Options( - timeZone = readTimeZone, hadoopConf = hadoopConf)).toSeq + ParquetReader.read[Parquet4sSingleActionWrapper]( + path, ParquetReader.Options( + timeZone = readTimeZone, hadoopConf = hadoopConf) + ).toSeq.map(_.unwrap) } else Seq.empty[SingleAction] }.toList } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala index d2d8fca9ca2..b765906f1ed 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala @@ -67,6 +67,18 @@ private[internal] case class Protocol( def simpleString: String = s"($minReaderVersion,$minWriterVersion)" } +/** +* Sets the committed version for a given application. Used to make operations +* like streaming append idempotent. +*/ +case class SetTransaction( + appId: String, + version: Long, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + lastUpdated: Option[Long]) extends Action { + override def wrap: SingleAction = SingleAction(txn = this) +} + /** Actions pertaining to the addition and removal of files. */ private[internal] sealed trait FileAction extends Action { val path: String @@ -113,13 +125,32 @@ private[internal] case class RemoveFile( path: String, @JsonDeserialize(contentAs = classOf[java.lang.Long]) deletionTimestamp: Option[Long], - dataChange: Boolean = true) extends FileAction { + dataChange: Boolean = true, + extendedFileMetadata: Boolean = false, + partitionValues: Map[String, String] = null, + size: Long = 0, + tags: Map[String, String] = null) extends FileAction { override def wrap: SingleAction = SingleAction(remove = this) @JsonIgnore val delTimestamp: Long = deletionTimestamp.getOrElse(0L) } +/** + * A change file containing CDC data for the Delta version it's within. Non-CDC readers should + * ignore this, CDC readers should scan all ChangeFiles in a version rather than computing + * changes from AddFile and RemoveFile actions. + */ +case class AddCDCFile( + path: String, + partitionValues: Map[String, String], + size: Long, + tags: Map[String, String] = null) extends FileAction { + override val dataChange = false + + override def wrap: SingleAction = SingleAction(cdc = this) +} + private[internal] case class Format( provider: String = "parquet", options: Map[String, String] = Map.empty) @@ -219,10 +250,12 @@ private[internal] case class NotebookInfo(notebookId: String) /** A serialization helper to create a common action envelope. */ private[internal] case class SingleAction( + txn: SetTransaction = null, add: AddFile = null, remove: RemoveFile = null, metaData: Metadata = null, protocol: Protocol = null, + cdc: AddCDCFile = null, commitInfo: CommitInfo = null) { def unwrap: Action = { @@ -232,8 +265,12 @@ private[internal] case class SingleAction( remove } else if (metaData != null) { metaData + } else if (txn != null) { + txn } else if (protocol != null) { protocol + } else if (cdc != null) { + cdc } else if (commitInfo != null) { commitInfo } else { @@ -261,3 +298,77 @@ private[internal] class JsonMapSerializer extends JsonSerializer[Map[String, Str jgen.writeEndObject() } } + +/** + * Parquet4s Wrapper Classes + * + * With the inclusion of RemoveFile as an exposed Java API, and since it was upgraded to match the + * latest Delta OSS release, we now had a case class inside of [[SingleAction]] that had "primitive" + * default paramaters. They are primitive in the sense that Parquet4s would try to decode them using + * the [[PrimitiveValueCodecs]] trait. But since these parameters have default values, there is no + * guarantee that they will exist in the underlying parquet checkpoint files. Thus (without these + * classes), parquet4s would throw errors like this: + * + * Cause: java.lang.IllegalArgumentException: NullValue cannot be decoded to required type + * at com.github.mjakubowski84.parquet4s.RequiredValueCodec.decode(ValueCodec.scala:61) + * at com.github.mjakubowski84.parquet4s.RequiredValueCodec.decode$(ValueCodec.scala:58) + * at com.github.mjakubowski84.parquet4s.PrimitiveValueCodecs$$anon$5.decode(ValueCodec.scala:137) + * + * Note this only happens with "primitive" parameters with default arguments, and not with "complex" + * or optional constructor parameters. + * + * We solve this issue by creating wrapper classes that wrap these primitive constructor parameters + * in [[Option]]s, and then un-wrapping them as needed, performing the appropriate Option[T] => T + * parameter conversions. + */ + +private[internal] trait Parquet4sWrapper[T] { + def unwrap: T +} + +private[internal] case class Parquet4sRemoveFileWrapper( + path: String, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + deletionTimestamp: Option[Long], + dataChangeOpt: Option[Boolean] = Some(true), + extendedFileMetadataOpt: Option[Boolean] = Some(false), + partitionValues: Map[String, String] = null, + size: Option[Long] = Some(0), + tags: Map[String, String] = null) extends Parquet4sWrapper[RemoveFile] { + + override def unwrap: RemoveFile = RemoveFile( + path, + deletionTimestamp, + dataChangeOpt.contains(true), + extendedFileMetadataOpt.contains(true), + partitionValues, + size match { + case Some(x) => x; + case _ => 0 + }, + tags + ) +} + +private[internal] case class Parquet4sSingleActionWrapper( + txn: SetTransaction = null, + add: AddFile = null, + remove: Parquet4sRemoveFileWrapper = null, + metaData: Metadata = null, + protocol: Protocol = null, + cdc: AddCDCFile = null, + commitInfo: CommitInfo = null) extends Parquet4sWrapper[SingleAction] { + + override def unwrap: SingleAction = SingleAction( + txn, + add, + remove match { + case x: Parquet4sRemoveFileWrapper if x != null => x.unwrap + case _ => null + }, + metaData, + protocol, + cdc, + commitInfo + ) +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 7c029176491..cdc5090c408 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -99,4 +99,17 @@ private[internal] object DeltaErrors { new NullPointerException(s"Read a null value for field $fieldName, yet schema indicates " + s"that this field can't be null. Schema: ${schema.getTreeString}") } + + def failOnDataLossException(expectedVersion: Long, seenVersion: Long): Throwable = { + new IllegalStateException( + s"""The stream from your Delta table was expecting process data from version $expectedVersion, + |but the earliest available version in the _delta_log directory is $seenVersion. The files + |in the transaction log may have been deleted due to log cleanup. + | + |If you would like to ignore the missed data and continue your stream from where it left + |off, you can set the .option("failOnDataLoss", "false") as part + |of your readStream statement. + """.stripMargin + ) + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index ddf52fc96a6..ceefe3ebb67 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -21,8 +21,8 @@ import java.util.{Optional => OptionalJ} import collection.JavaConverters._ -import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ} -import io.delta.standalone.internal.actions.{AddFile, CommitInfo, Format, JobInfo, Metadata, NotebookInfo} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, AddCDCFile => AddCDCFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} +import io.delta.standalone.internal.actions.{Action, AddCDCFile, AddFile, CommitInfo, Format, JobInfo, Metadata, NotebookInfo, Protocol, RemoveFile, SetTransaction} /** * Provide helper methods to convert from Scala to Java types. @@ -33,7 +33,7 @@ private[internal] object ConversionUtils { * This is a workaround for a known issue in Scala 2.11: `asJava` doesn't handle `null`. * See https://github.com/scala/scala/pull/4343 */ - private def mapAsJava[K, V](map: Map[K, V]): java.util.Map[K, V] = { + private def nullableMapAsJava[K, V](map: Map[K, V]): java.util.Map[K, V] = { if (map == null) { null } else { @@ -74,7 +74,26 @@ private[internal] object ConversionUtils { internal.modificationTime, internal.dataChange, internal.stats, - mapAsJava(internal.tags)) + nullableMapAsJava(internal.tags)) + } + + def convertAddCDCFile(internal: AddCDCFile): AddCDCFileJ = { + new AddCDCFileJ( + internal.path, + internal.partitionValues.asJava, + internal.size, + nullableMapAsJava(internal.tags)) + } + + def convertRemoveFile(internal: RemoveFile): RemoveFileJ = { + new RemoveFileJ( + internal.path, + toJavaLongOptional(internal.deletionTimestamp), + internal.dataChange, + internal.extendedFileMetadata, + nullableMapAsJava(internal.partitionValues), + internal.size, + nullableMapAsJava(internal.tags)) } /** @@ -151,4 +170,25 @@ private[internal] object ConversionUtils { def convertNotebookInfo(internal: NotebookInfo): NotebookInfoJ = { new NotebookInfoJ(internal.notebookId) } + + def convertSetTransaction(internal: SetTransaction): SetTransactionJ = { + new SetTransactionJ(internal.appId, internal.version, toJavaLongOptional(internal.lastUpdated)) + } + + def convertProtocol(internal: Protocol): ProtocolJ = { + new ProtocolJ(internal.minReaderVersion, internal.minWriterVersion) + } + + def convertAction(internal: Action): ActionJ = internal match { + case x: AddFile => convertAddFile(x) + case a: AddCDCFile => convertAddCDCFile(a) + case x: RemoveFile => convertRemoveFile(x) + case x: CommitInfo => convertCommitInfo(x) + case x: Format => convertFormat(x) + case x: JobInfo => convertJobInfo(x) + case x: Metadata => convertMetadata(x) + case x: NotebookInfo => convertNotebookInfo(x) + case x: SetTransaction => convertSetTransaction(x) + case x: Protocol => convertProtocol(x) + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index 7eadb38ce18..06ce09cffc6 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -279,4 +279,72 @@ class DeltaLogSuite extends FunSuite { } } + test("getChanges - no data loss") { + withLogForGoldenTable("deltalog-getChanges") { log => + val versionToActionsMap = Map( + 0L -> Seq("CommitInfo", "Protocol", "Metadata", "AddFile"), + 1L -> Seq("CommitInfo", "AddCDCFile", "RemoveFile"), + 2L -> Seq("CommitInfo", "Protocol", "SetTransaction") + ) + + def verifyChanges(startVersion: Int): Unit = { + val versionLogs = log.getChanges(startVersion, false).asScala.toSeq + + assert(versionLogs.length == 3 - startVersion, + s"getChanges($startVersion) skipped some versions") + + for (versionLog <- versionLogs) { + val version = versionLog.getVersion + val actions = versionLog.getActions.asScala.map(_.getClass.getSimpleName) + val expectedActions = versionToActionsMap(version) + assert(expectedActions == actions, + s"getChanges($startVersion) had incorrect actions at version $version.") + } + } + + // standard cases + verifyChanges(0) + verifyChanges(1) + verifyChanges(2) + + // non-existant start version + val versionLogsIter = log.getChanges(3, false) + assert(!versionLogsIter.hasNext, + "getChanges with a non-existant start version did not return an empty iterator") + + // negative start version + assertThrows[IllegalArgumentException] { + log.getChanges(-1, false) + } + } + } + + test("getChanges - data loss") { + withGoldenTable("deltalog-getChanges") { tablePath => + val tempDir = Files.createTempDirectory(UUID.randomUUID().toString).toFile + try { + FileUtils.copyDirectory(new File(tablePath), tempDir) + val log = DeltaLog.forTable(new Configuration(), tempDir.getCanonicalPath) + + // we delete 2 files so that the `DeltaErrors.failOnDataLossException` is thrown + val logPath = new Path(log.getPath, "_delta_log") + new File(new Path(logPath, "00000000000000000000.json").toUri).delete() + new File(new Path(logPath, "00000000000000000001.json").toUri).delete() + + val versionLogs = log.getChanges(0, false).asScala.toSeq + assert(versionLogs.length == 1) + + assertThrows[IllegalStateException] { + val versionLogsIter = log.getChanges(0, true) + while (versionLogsIter.hasNext) { + versionLogsIter.next() + } + } + } finally { + // just in case + FileUtils.deleteDirectory(tempDir) + } + } + } + } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/util/GoldenTableUtils.scala b/standalone/src/test/scala/io/delta/standalone/internal/util/GoldenTableUtils.scala index 554bf59085f..88a816a6d9c 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/util/GoldenTableUtils.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/util/GoldenTableUtils.scala @@ -24,11 +24,17 @@ import org.apache.hadoop.conf.Configuration object GoldenTableUtils { - /** Load the golden table as a class resource so that it works in IntelliJ and SBT tests */ + /** + * Load the golden table as a class resource so that it works in IntelliJ and SBT tests. + * + * If this is causing a `java.lang.NullPointerException` while debugging in IntelliJ, you + * probably just need to SBT test that specific test first. + */ val goldenTable = new File(getClass.getResource("/golden").toURI) /** - * Create a [[DeltaLog]] for the given golden table and execute the test function. + * Create a [[DeltaLog]] (with Java interface) for the given golden table and execute the test + * function. * * @param name The name of the golden table to load. * @param testFunc The test to execute which takes the [[DeltaLog]] as input arg. From 0bf07df709be2a93cb582eb0cf220f81e791fa9e Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 10 Aug 2021 11:47:00 -0700 Subject: [PATCH 065/291] add MiMa settings (#113) * add MiMa settings and test them * responded to PR comments --- build.sbt | 46 +++++++++++++++++++++++++++- project/StandaloneMimaExcludes.scala | 31 +++++++++++++++++++ project/plugins.sbt | 1 + 3 files changed, 77 insertions(+), 1 deletion(-) create mode 100644 project/StandaloneMimaExcludes.scala diff --git a/build.sbt b/build.sbt index 24dc33a122a..5a0cf86ac53 100644 --- a/build.sbt +++ b/build.sbt @@ -247,6 +247,8 @@ lazy val standalone = (project in file("standalone")) .settings( name := "delta-standalone", commonSettings, + releaseSettings, + mimaSettings, unmanagedResourceDirectories in Test += file("golden-tables/src/test/resources"), libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", @@ -288,7 +290,49 @@ lazy val standalone = (project in file("standalone")) // Ensure unidoc is run with tests. Must be cleaned before test for unidoc to be generated. (test in Test) := ((test in Test) dependsOn unidoc.in(Compile)).value ) - .settings(releaseSettings) + +/* + ******************** + * MIMA settings * + ******************** + */ +def getPrevVersion(currentVersion: String): String = { + implicit def extractInt(str: String): Int = { + """\d+""".r.findFirstIn(str).map(java.lang.Integer.parseInt).getOrElse { + throw new Exception(s"Could not extract version number from $str in $version") + } + } + + val (major, minor, patch): (Int, Int, Int) = { + currentVersion.split("\\.").toList match { + case majorStr :: minorStr :: patchStr :: _ => + (majorStr, minorStr, patchStr) + case _ => throw new Exception(s"Could not find previous version for $version.") + } + } + + val majorToLastMinorVersions: Map[Int, Int] = Map( + // TODO add mapping when required + // e.g. 0 -> 8 + ) + if (minor == 0) { // 1.0.0 + val prevMinor = majorToLastMinorVersions.getOrElse(major - 1, { + throw new Exception(s"Last minor version of ${major - 1}.x.x not configured.") + }) + s"${major - 1}.$prevMinor.0" // 1.0.0 -> 0.8.0 + } else if (patch == 0) { + s"$major.${minor - 1}.0" // 1.1.0 -> 1.0.0 + } else { + s"$major.$minor.${patch - 1}" // 1.1.1 -> 1.1.0 + } +} + +lazy val mimaSettings = Seq( + (test in Test) := ((test in Test) dependsOn mimaReportBinaryIssues).value, + mimaPreviousArtifacts := Set("io.delta" %% "delta-standalone" % getPrevVersion(version.value)), + mimaBinaryIssueFilters ++= StandaloneMimaExcludes.ignoredABIProblems +) + lazy val goldenTables = (project in file("golden-tables")) settings ( name := "golden-tables", diff --git a/project/StandaloneMimaExcludes.scala b/project/StandaloneMimaExcludes.scala new file mode 100644 index 00000000000..9100372250b --- /dev/null +++ b/project/StandaloneMimaExcludes.scala @@ -0,0 +1,31 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import com.typesafe.tools.mima.core._ +import com.typesafe.tools.mima.core.ProblemFilters._ + +/** + * The list of Mima errors to exclude in the Standalone project. + */ +object StandaloneMimaExcludes { + val ignoredABIProblems = Seq( + // Ignore changes to internal Scala codes + ProblemFilters.exclude[Problem]("io.delta.standalone.internal.*"), + + // Public API changes in 0.2.0 -> 0.3.0 + ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.getChanges") + ) +} diff --git a/project/plugins.sbt b/project/plugins.sbt index 9af949c1a14..f69f9e9fd8b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -32,3 +32,4 @@ addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.3.0") From 43bb0093a26de0bfbea06ded4191cb5bb633914c Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Wed, 25 Aug 2021 15:05:35 -0700 Subject: [PATCH 066/291] Add toJson method to DataType.java (#119) This PR adds support for conversion to json strings in `DataTypeParser.scala` and exposure via a `String toJson()` method in `DataType.java.` Includes added tests in `DeltaDataReaderSuite.scala.` --- .../io/delta/standalone/types/DataType.java | 7 ++++ .../internal/util/DataTypeParser.scala | 36 +++++++++++++++++- .../internal/DeltaDataReaderSuite.scala | 38 ++++++++++++++++++- 3 files changed, 79 insertions(+), 2 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/types/DataType.java b/standalone/src/main/java/io/delta/standalone/types/DataType.java index 5bc25ccc12c..6e0cd566a9a 100644 --- a/standalone/src/main/java/io/delta/standalone/types/DataType.java +++ b/standalone/src/main/java/io/delta/standalone/types/DataType.java @@ -41,6 +41,8 @@ import java.util.Locale; import java.util.Objects; +import io.delta.standalone.internal.util.DataTypeParser; + /** * The base type of all {@code io.delta.standalone} data types. * Represents a bare-bones Java implementation of the Spark SQL @@ -74,6 +76,11 @@ public String getCatalogString() { return getSimpleString(); } + /** + * @return a JSON (@code String} representation of the type + */ + public String toJson() { return DataTypeParser.toJson(this); } + /** * Builds a readable {@code String} representation of the {@code ArrayType} */ diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala index 6bf1326e393..88a13727b24 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala @@ -42,9 +42,10 @@ import io.delta.standalone.types._ import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.json4s.JsonDSL._ import org.json4s.JsonAST.JValue -private[internal] object DataTypeParser { +private[standalone] object DataTypeParser { private val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r @@ -83,6 +84,39 @@ private[internal] object DataTypeParser { s"Failed to convert the JSON string '${compact(render(other))}' to a data type.") } + def toJson(value : DataType): String = { + compact(render(dataTypeToJValue(value))) + } + + def dataTypeToJValue(dataType: DataType): JValue = dataType match { + case array: ArrayType => + ("type" -> "array") ~ + ("elementType" -> dataTypeToJValue(array.getElementType)) ~ + ("containsNull" -> array.containsNull()) + case map: MapType => + ("type" -> "map") ~ + ("keyType" -> dataTypeToJValue(map.getKeyType())) ~ + ("valueType" -> dataTypeToJValue(map.getValueType())) ~ + ("valueContainsNull" -> map.valueContainsNull()) + case struct: StructType => + ("type" -> "struct") ~ + ("fields" -> struct.getFields().map(structFieldToJValue).toList) + case decimal: DecimalType => + s"decimal(${decimal.getPrecision()},${decimal.getScale()})" + case _: DataType => + dataType.getTypeName() + } + + def structFieldToJValue(field: StructField): JValue = { + val name = field.getName() + val dataType = field.getDataType() + val nullable = field.isNullable() + + ("name" -> name) ~ + ("type" -> dataTypeToJValue(dataType)) ~ + ("nullable" -> nullable) + } + /** Given the string representation of a type, return its DataType */ private def nameToType(name: String): DataType = { name match { diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala index fdc6b5143ba..be5cd66a449 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala @@ -26,8 +26,9 @@ import scala.collection.JavaConverters._ import io.delta.standalone.data.{RowRecord => JRowRecord} import io.delta.standalone.DeltaLog import io.delta.standalone.internal.sources.StandaloneHadoopConf +import io.delta.standalone.internal.util.DataTypeParser import io.delta.standalone.internal.util.GoldenTableUtils._ -import io.delta.standalone.types.{DateType, StructField, StructType, TimestampType} +import io.delta.standalone.types._ import org.apache.hadoop.conf.Configuration // scalastyle:off funsuite import org.scalatest.FunSuite @@ -300,4 +301,39 @@ class DeltaDataReaderSuite extends FunSuite { assert(row.getSchema == expectedSchema) } } + + def checkDataTypeToJsonFromJson(dataType: DataType): Unit = { + test(s"DataType to Json and from Json - $dataType") { + assert(DataTypeParser.fromJson(dataType.toJson()) === dataType) + } + + test(s"DataType inside StructType to Json and from Json - $dataType") { + val field1 = new StructField("foo", dataType, true) + val field2 = new StructField("bar", dataType, true) + val struct = new StructType(Array(field1, field2)) + assert(DataTypeParser.fromJson(struct.toJson()) === struct) + } + } + + checkDataTypeToJsonFromJson(new BooleanType) + checkDataTypeToJsonFromJson(new ByteType) + checkDataTypeToJsonFromJson(new ShortType) + checkDataTypeToJsonFromJson(new IntegerType) + checkDataTypeToJsonFromJson(new LongType) + checkDataTypeToJsonFromJson(new FloatType) + checkDataTypeToJsonFromJson(new DoubleType) + checkDataTypeToJsonFromJson(new DecimalType(10, 5)) + checkDataTypeToJsonFromJson(DecimalType.USER_DEFAULT) + checkDataTypeToJsonFromJson(new DateType) + checkDataTypeToJsonFromJson(new TimestampType) + checkDataTypeToJsonFromJson(new StringType) + checkDataTypeToJsonFromJson(new BinaryType) + checkDataTypeToJsonFromJson(new ArrayType(new DoubleType, true)) + checkDataTypeToJsonFromJson(new ArrayType(new StringType, false)) + checkDataTypeToJsonFromJson(new MapType(new IntegerType, new StringType, true)) + checkDataTypeToJsonFromJson( + new MapType( + new IntegerType, + new ArrayType(new DoubleType, true), + false)) } From da6de9781484e0ce90e9d2f37fcfd1d85ee42dd3 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Fri, 27 Aug 2021 02:20:48 +0800 Subject: [PATCH 067/291] remove spark dependencies for hive subproject (#116) * remove spark dependencies for hive-subproject (#115) * responded to PR comments Co-authored-by: Yann --- build.sbt | 20 +- ...-a89e-1c4468eff784.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9a0f-44d143761f88.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4374-a89e-1c4468eff784.c000.snappy.parquet | Bin 0 -> 448 bytes ...4021-9a0f-44d143761f88.c000.snappy.parquet | Bin 0 -> 443 bytes ...-92eb-04549d3f220e.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a708-bec69fc562a2.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...435a-92eb-04549d3f220e.c000.snappy.parquet | Bin 0 -> 443 bytes ...4868-a708-bec69fc562a2.c000.snappy.parquet | Bin 0 -> 448 bytes .../_delta_log/00000000000000000000.json | 7 + ...-bab2-d565a7ca5bfb-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-9ad9-f6d087ae7cb8-c000.snappy.parquet.crc | Bin 0 -> 16 bytes .../_delta_log/00000000000000000000.json | 5 + ...41e6-bab2-d565a7ca5bfb-c000.snappy.parquet | Bin 0 -> 659 bytes ...430d-9ad9-f6d087ae7cb8-c000.snappy.parquet | Bin 0 -> 659 bytes ...-9d99-e98cfde281cf-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-a508-fc046c1c57cf-c000.snappy.parquet.crc | Bin 0 -> 16 bytes .../_delta_log/00000000000000000000.json | 5 + ...4a4a-9d99-e98cfde281cf-c000.snappy.parquet | Bin 0 -> 653 bytes ...4697-a508-fc046c1c57cf-c000.snappy.parquet | Bin 0 -> 653 bytes ...-8d76-768bd813c3ff-c000.snappy.parquet.crc | Bin 0 -> 44 bytes .../_delta_log/00000000000000000000.json | 4 + ...45df-8d76-768bd813c3ff-c000.snappy.parquet | Bin 0 -> 4156 bytes ...-94eb-fea5209018d0-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-9c89-c4c2481bfd93-c000.snappy.parquet.crc | Bin 0 -> 16 bytes .../_delta_log/00000000000000000000.json | 5 + ...46bd-94eb-fea5209018d0-c000.snappy.parquet | Bin 0 -> 659 bytes ...451b-9c89-c4c2481bfd93-c000.snappy.parquet | Bin 0 -> 659 bytes ...-ac35-249c0f5750ff-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-9b5a-d064e8c5be2d-c000.snappy.parquet.crc | Bin 0 -> 16 bytes .../_delta_log/00000000000000000000.json | 5 + ...46ea-ac35-249c0f5750ff-c000.snappy.parquet | Bin 0 -> 653 bytes ...49e7-9b5a-d064e8c5be2d-c000.snappy.parquet | Bin 0 -> 653 bytes .../_delta_log/00000000000000000000.json | 8 + ...-b8b3-71d02178554f.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...4511-b8b3-71d02178554f.c000.snappy.parquet | Bin 0 -> 628 bytes ...-8ba3-1499f599b8f1.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...400a-8ba3-1499f599b8f1.c000.snappy.parquet | Bin 0 -> 650 bytes ...-ba9b-31fb8aeb8093.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...4b88-ba9b-31fb8aeb8093.c000.snappy.parquet | Bin 0 -> 628 bytes ...-bba8-3dbf43b4eb86.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...43fe-bba8-3dbf43b4eb86.c000.snappy.parquet | Bin 0 -> 639 bytes ...-a6cc-04759d5f692b.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...4405-a6cc-04759d5f692b.c000.snappy.parquet | Bin 0 -> 650 bytes .../_delta_log/00000000000000000000.json | 7 + ...-bd22-d361a60627e3.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-bf02-5d1872324d6f.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4401-bd22-d361a60627e3.c000.snappy.parquet | Bin 0 -> 436 bytes ...4d70-bf02-5d1872324d6f.c000.snappy.parquet | Bin 0 -> 431 bytes ...-9a4c-de0b22b62bbd.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9c6a-25f7559cd150.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...454f-9a4c-de0b22b62bbd.c000.snappy.parquet | Bin 0 -> 431 bytes ...465a-9c6a-25f7559cd150.c000.snappy.parquet | Bin 0 -> 436 bytes .../_delta_log/00000000000000000000.json | 7 + ...-8c35-7ed2416c5ff5.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-8da9-622005e49f2c.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...44c9-8c35-7ed2416c5ff5.c000.snappy.parquet | Bin 0 -> 629 bytes ...4cb9-8da9-622005e49f2c.c000.snappy.parquet | Bin 0 -> 618 bytes ...-992e-35c2fb4d9c09.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-8a22-5bb45a598dcf.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...46c1-992e-35c2fb4d9c09.c000.snappy.parquet | Bin 0 -> 618 bytes ...4f4d-8a22-5bb45a598dcf.c000.snappy.parquet | Bin 0 -> 629 bytes .../_delta_log/00000000000000000000.json | 7 + ...-b474-1748ba085060.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a968-acfa96d6f1e9.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4e66-b474-1748ba085060.c000.snappy.parquet | Bin 0 -> 436 bytes ...4e51-a968-acfa96d6f1e9.c000.snappy.parquet | Bin 0 -> 431 bytes ...-8145-1e0d73406b02.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-8b7e-29038d983f83.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4321-8145-1e0d73406b02.c000.snappy.parquet | Bin 0 -> 431 bytes ...4781-8b7e-29038d983f83.c000.snappy.parquet | Bin 0 -> 436 bytes .../_delta_log/00000000000000000000.json | 7 + ...-91b2-6568396e2ec3.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-b8a6-c6e9b78a1448.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4d47-91b2-6568396e2ec3.c000.snappy.parquet | Bin 0 -> 436 bytes ...41f5-b8a6-c6e9b78a1448.c000.snappy.parquet | Bin 0 -> 431 bytes ...-ae73-9baf5a8a08d0.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a530-a8b1360ca9f6.c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...4c49-ae73-9baf5a8a08d0.c000.snappy.parquet | Bin 0 -> 431 bytes ...4e93-a530-a8b1360ca9f6.c000.snappy.parquet | Bin 0 -> 436 bytes .../scala/io/delta/golden/GoldenTables.scala | 114 +++++- .../java/io/delta/hive/util/JavaUtils.java | 134 +++++++ .../scala/io/delta/hive/DeltaHelperTest.scala | 5 +- .../io/delta/hive/HiveConnectorTest.scala | 360 +++++++----------- .../scala/io/delta/hive/test/HiveTest.scala | 15 +- .../spark/sql/delta/SparkSessionHelper.scala | 44 --- scalastyle-config.xml | 6 - .../internal/DeltaDataReaderSuite.scala | 2 +- .../standalone/internal/DeltaLogSuite.scala | 1 - .../internal/DeltaTimeTravelSuite.scala | 1 - .../internal/ReadOnlyLogStoreSuite.scala | 1 - 91 files changed, 458 insertions(+), 312 deletions(-) create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00001-27f5c1f6-2393-4021-9a0f-44d143761f88.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/part-00001-27f5c1f6-2393-4021-9a0f-44d143761f88.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00001-b6134dd2-aa40-4868-a708-bec69fc562a2.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/part-00001-b6134dd2-aa40-4868-a708-bec69fc562a2.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-incorrect-format-config/.part-00000-7b3124df-d8a4-4a4a-9d99-e98cfde281cf-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-incorrect-format-config/.part-00001-e8582398-602e-4697-a508-fc046c1c57cf-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-incorrect-format-config/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-incorrect-format-config/part-00000-7b3124df-d8a4-4a4a-9d99-e98cfde281cf-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-incorrect-format-config/part-00001-e8582398-602e-4697-a508-fc046c1c57cf-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/.part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/.part-00000-e24c5388-1621-46bd-94eb-fea5209018d0-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/.part-00001-f2126b8d-1594-451b-9c89-c4c2481bfd93-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/part-00000-e24c5388-1621-46bd-94eb-fea5209018d0-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/part-00001-f2126b8d-1594-451b-9c89-c4c2481bfd93-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00001-6eb569ba-9300-49e7-9b5a-d064e8c5be2d-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/part-00001-6eb569ba-9300-49e7-9b5a-d064e8c5be2d-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180512/city=sh/.part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180512/city=sh/part-00001-c87aeb63-6d9c-4511-b8b3-71d02178554f.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=hz/.part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=hz/part-00000-de1d5bcd-ad7e-4b88-ba9b-31fb8aeb8093.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20181212/city=sz/.part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20181212/city=sz/part-00001-529ff89b-55c6-4405-a6cc-04759d5f692b.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/.part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/.part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/.part-00000-531fe778-e359-44c9-8c35-7ed2416c5ff5.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/.part-00001-923b258c-b34c-4cb9-8da9-622005e49f2c.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/part-00000-531fe778-e359-44c9-8c35-7ed2416c5ff5.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/part-00001-923b258c-b34c-4cb9-8da9-622005e49f2c.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00001-e44bca08-b26b-4f4d-8a22-5bb45a598dcf.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/part-00001-e44bca08-b26b-4f4d-8a22-5bb45a598dcf.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/.part-00000-180d1a36-4ba9-4321-8145-1e0d73406b02.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/.part-00001-3379bbbf-1ab8-4781-8b7e-29038d983f83.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/part-00000-180d1a36-4ba9-4321-8145-1e0d73406b02.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/part-00001-3379bbbf-1ab8-4781-8b7e-29038d983f83.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00001-e7f40ed6-fefa-41f5-b8a6-c6e9b78a1448.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/part-00001-e7f40ed6-fefa-41f5-b8a6-c6e9b78a1448.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/.part-00000-1bb7c99b-be0e-4c49-ae73-9baf5a8a08d0.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/.part-00001-c357f264-a317-4e93-a530-a8b1360ca9f6.c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/part-00000-1bb7c99b-be0e-4c49-ae73-9baf5a8a08d0.c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/part-00001-c357f264-a317-4e93-a530-a8b1360ca9f6.c000.snappy.parquet create mode 100644 hive/src/test/java/io/delta/hive/util/JavaUtils.java delete mode 100644 hive/src/test/scala/org/apache/spark/sql/delta/SparkSessionHelper.scala diff --git a/build.sbt b/build.sbt index 5a0cf86ac53..ddfd1945ff5 100644 --- a/build.sbt +++ b/build.sbt @@ -144,12 +144,7 @@ lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( ExclusionRule("org.apache.hive", "hive-exec"), ExclusionRule(organization = "com.google.protobuf") ), - "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % hiveDeltaVersion % "test", - "org.apache.spark" %% "spark-sql" % sparkVersion % "test", - "org.apache.spark" %% "spark-catalyst" % sparkVersion % "test" classifier "tests", - "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", - "org.apache.spark" %% "spark-sql" % sparkVersion % "test" classifier "tests" + "org.scalatest" %% "scalatest" % "3.0.5" % "test" ), /** Hive assembly jar. Build with `assembly` command */ @@ -171,6 +166,7 @@ lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") se name := "hive-mr", commonSettings, skipReleaseSettings, + unmanagedResourceDirectories in Test += file("golden-tables/src/test/resources"), libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( @@ -188,10 +184,7 @@ lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") se ExclusionRule("ch.qos.logback", "logback-classic"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") ), - // TODO Figure out how this fixes some bad dependency - "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", - "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % hiveDeltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") + "org.scalatest" %% "scalatest" % "3.0.5" % "test" ) ) @@ -199,6 +192,7 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") name := "hive-tez", commonSettings, skipReleaseSettings, + unmanagedResourceDirectories in Test += file("golden-tables/src/test/resources"), libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" excludeAll ( ExclusionRule(organization = "com.google.protobuf") @@ -236,10 +230,8 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") "org.apache.tez" % "tez-mapreduce" % "0.8.4" % "test", "org.apache.tez" % "tez-dag" % "0.8.4" % "test", "org.apache.tez" % "tez-tests" % "0.8.4" % "test" classifier "tests", - // TODO Figure out how this fixes some bad dependency - "org.apache.spark" %% "spark-core" % sparkVersion % "test" classifier "tests", - "org.scalatest" %% "scalatest" % "3.0.5" % "test", - "io.delta" %% "delta-core" % hiveDeltaVersion % "test" excludeAll ExclusionRule("org.apache.hadoop") + "com.esotericsoftware" % "kryo-shaded" % "4.0.2" % "test", + "org.scalatest" %% "scalatest" % "3.0.5" % "test" ) ) diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..4110a89a7f8b5cc9cda0b07b5b034af5ababf22c GIT binary patch literal 12 TcmYc;N@ieSU}AV#XZ8mG6b1v- literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00001-27f5c1f6-2393-4021-9a0f-44d143761f88.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/.part-00001-27f5c1f6-2393-4021-9a0f-44d143761f88.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b1be927703982dd6e5f9b4f30291c62c2a208b39 GIT binary patch literal 12 TcmYc;N@ieSU}C6by*(KK5xfH@ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo0/part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..33ab70582e810406e109cb2c6e826382c88cd09d GIT binary patch literal 448 zcmaJ;&r8EF7|rUeJ$MpQQplkM9UIipTGp+_iy|VPhF2MquIViH$CkE=jCt@c_p5Fo z2wpzEy!Z0G_a(Ed2M;5BiiZx?q0zBHNxzQ}B3(fy8vZ%~aB`zTJvi#fuIp4-vz)u* zdL`7$Q@wbWg}@qsXByFOtLj=PWa<{R=)j;L9UDlu_6*ic-TW{=djsS0A76_?HW&$J zpw^M)5K^fn7b^jN{M(wpN>yi7Ne+GPau<=&A4R>+Iiy^-vf-+F?&WhaMrGb zdV6UXud))@0P(^g)_v35356Wfr7rE;6wr~4EcfVO!!ezW^Vzp={)u!cj0GJb3t$tu zJ|T@-aS{}AoEZEvrSfw|=ey)&VVQEtgV(PzNdN!< literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..e0a0ba2b465328cc5c030a2dda92edf7d6c5e788 GIT binary patch literal 12 TcmYc;N@ieSU}CuQ=>2{G7Bd8= literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00001-b6134dd2-aa40-4868-a708-bec69fc562a2.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/.part-00001-b6134dd2-aa40-4868-a708-bec69fc562a2.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..5b36873c00b33196f7f1b909e84dc408656e6126 GIT binary patch literal 12 TcmYc;N@ieSU}ET9&t?q(5(omg literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/BarFoo=foo1/part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e47afea9bb6ebb6291a4de140e673dde14563671 GIT binary patch literal 443 zcmaJ;&r8EF7>!*WJ&3oI6mlp5#|AaD)^%&~q6ZOA!>bHQ*K`*9V@uPCj{Wa`(+vc{ z%SZCw%lF>*Wp?`%V1y}lJ#50Ek1a~f9zuwiCs1uP{B;B1{Mvxp>+Q+C3>%L3aMrGb zdV6UXud))@0Pw;f{rjf56AIa;LmfJ>C`iW^>N`gU8@6e0?9YL9^N*!NAr^E5FA$r^ z@d#``kxl^heRIOAaB|ozz@4@4S*)WI_uv-!lENk}P4Z zR^37~kL86dvz8^#TULu|PyLJn)InO*S}vsmpzg|2q$P;pwe;ArFH5$&-F%100vINY_{9k z7LkX&>qQ}*eN4w^zQ=s0A2(T^ID8x>={Sv|;4%(H8iqlTc%c_YV(87IcodDiSj34x PuxuM%LA4N?!ry%XOIl`{ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..70b4ebf77d3 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-column-names-case-insensitive/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629874535433,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"BarFoo\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"1782","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"FooBar\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"BarFoo\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["BarFoo"],"configuration":{},"createdTime":1629874533636}} +{"add":{"path":"BarFoo=foo0/part-00000-36c1f69c-21dc-4374-a89e-1c4468eff784.c000.snappy.parquet","partitionValues":{"BarFoo":"foo0"},"size":448,"modificationTime":1629874535000,"dataChange":true}} +{"add":{"path":"BarFoo=foo1/part-00000-5c80a439-70eb-435a-92eb-04549d3f220e.c000.snappy.parquet","partitionValues":{"BarFoo":"foo1"},"size":443,"modificationTime":1629874535000,"dataChange":true}} +{"add":{"path":"BarFoo=foo0/part-00001-27f5c1f6-2393-4021-9a0f-44d143761f88.c000.snappy.parquet","partitionValues":{"BarFoo":"foo0"},"size":443,"modificationTime":1629874535000,"dataChange":true}} +{"add":{"path":"BarFoo=foo1/part-00001-b6134dd2-aa40-4868-a708-bec69fc562a2.c000.snappy.parquet","partitionValues":{"BarFoo":"foo1"},"size":448,"modificationTime":1629874535000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..84a501d3e0d26223c4a12e1da45612264bc6b2a9 GIT binary patch literal 16 XcmYc;N@ieSU}E^KoXUL6-fklRBB%tl literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/.part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..13df0f26ed00ac3499fee839076e0e97c9aebaa2 GIT binary patch literal 16 XcmYc;N@ieSU}Ctp=ij+w_I4WqEqn%* literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..c2cda841b35 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1629874421524,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputBytes":"1318","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1629874419356}} +{"add":{"path":"part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet","partitionValues":{},"size":659,"modificationTime":1629874421000,"dataChange":true}} +{"add":{"path":"part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet","partitionValues":{},"size":659,"modificationTime":1629874421000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00000-377b2930-7ed7-41e6-bab2-d565a7ca5bfb-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..15b5ceb5badbef2f1867abd78a729080fde36fe5 GIT binary patch literal 659 zcmah{!D`z;5M3#%QH>9wuwu6eDwf71pe!ppRxIO7Z>98LdMzQWq?IEgOSUAZ6ysm$ zA&?$Q|FS>OS=Yg#g$AUZotgLEyn&s)`xIb=FYwsMUAT0x?GnpE2oZfC2c162pzu01 z;XHbyn+E*545CJ33w!wOM+lG|PjxA)DzT4x&MWNbIo57Y2-?sOh&}~?UwVJEjiW9$ z>GXZCS<3qMx=C)-LV_kXPfTJa0m8lAWpDxpyVRwA-*$m9LLHlG05+Yj^UZMXjlMno z&pOlvPaq4k`#%g9_vol&TUui9DDmBZ&h#&<)Ym%#56C)zjh`?QQJ;{io^!dB;F<62 z^E*{Hc~z1ZLoT>L~E2*VPxkP_0a5E6IbgVeQwYViVT1^*U*p&$3(< zX~QO;H>{Kk-AV!|?_qh_s=2D6P_BzY#svU?seJY*$|jrb)<2L32Qtja%lUKMt1TiQ z_IA$~wYT5T3wR)S;GM?nyhuGcj?!$LMbU5&hbjxh;V>1U2qQHTNfcj3mm*ei8k|wz OMF>9h3!c*qp5t%xor+8V literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-deleted-path/part-00001-6537e97d-662a-430d-9ad9-f6d087ae7cb8-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..747a873e655457d57fe78d42c3f9f02586a7ab5c GIT binary patch literal 659 zcmah{QH#?+5T4|6HI_bzuwhpMhQ%v)5SLABb4}@sZz4V%zKX~u+0gJ?9A-W%=gVVuvbrCMg+(cG7rfRF9YKG)VT`))Hy}(!TE15OgzYK z^d}EZae#IQ%|$a1;*fwmdV#?KxVJVsD!x8-AwoZn` zX3J-zZll_FZ@TQQUMu7z_L)uXECMn*Tqb8IaL9ZXj$I!SGZ?tcG!WYxdY>%svo&^? z|5%6ls0nFt_V}CSlM$N@T*pvMZYaK5SXurdm4#+U#39%Ol<^%$0t+c^+Lcfng_^}d zzId`^hkd;V z`ur&O@wyF;`vsARq=ISuhyDl0|LBYd^MC9T zgfIk63?6@RLOQYv9u)MA!eUh7t^~Hc-k4sr%{#F&y~;Ddq)da1c$*Z#4OAyW7P-x%w9u6Uut|KkCaJVNVZcAFkTy zuw9ghM2aLj)tj=)BQ?$PVp?R`CNb}~sc>}sQeH#-XM`Rq4KECvb8&Kyd08nQavj_VRzOeBk zvoY_yG++RJDip`hMU?LpQ+rW48xB!o>Fa-dD@}W)0ox zKW^~@G=!SCczA6EWMC8=R8aQ{KP&{Mk8JvpxesZ`H3d!F;70HxO6z9MVez(@F`?Az*!rHD`%{msbywGLV zvdOzOtK`B!61&@}qU!WqH`u6FWhqsOI1+Wl?lswDz1c*+x#|xWHx$+Uf7JIIpl4gN zuNO_Yy)KM-EO;F5t5s2Ep&X}KKF-s0bf^-YC&_4(i9{r+J{Gf7ouns1X_dtXeiQ)U M13xW*3;Yp(0n3JoPXGV_ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/.part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/.part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..7cce9dc935ece8d26cd384e301feb42f1aea3205 GIT binary patch literal 44 zcmYc;N@ieSU}9Jz+ajtTR{Czw_0$gkN9QB8H%RnwomRh5FJ7S$Z?3dxvFuwe0AvFY A@Bjb+ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..628cf997d11 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1629873175558,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"4156","numOutputRows":"1"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"byte\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"binary\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c3\",\"type\":\"boolean\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c4\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c5\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c6\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c7\",\"type\":\"float\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c8\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c9\",\"type\":\"short\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c10\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c11\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c12\",\"type\":\"decimal(38,18)\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c13\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"c14\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"long\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"c15\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"f1\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"f2\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1629873173115}} +{"add":{"path":"part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet","partitionValues":{},"size":4156,"modificationTime":1629873175000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-map-types-correctly/part-00000-c9259a22-ce39-45df-8d76-768bd813c3ff-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0f2dccb28adf045797ac375fc54358716e6c0e3c GIT binary patch literal 4156 zcmb7IL2nyH6rNpg94B?t6m^z0tSZZ@S1B@TOX_ultr z-h1n0>;2DcO6VDCl&A_JO*5vUIfRhtsgpHn1@-bd70;ByOki(NQv#~(j)5P5VhKjs zQUqYnb_b9Z`i2Bi6hlz}2!F$5(TL2IX;vDx4UIh+E3txtvtmjBO`4ZNWEP-hUq&Lq zK_Uw+Ojb6c@EihXZ_%@ovs7<1GS~wy5*fggl?a%EjNNJWgdAQp!0+%PV=q>)=j@qs znmr8gWnMHgTw?6GEVR&Y_Iz`i1;ZH{domVc1qElpl+!>c0c6jJK3_q>fozj%tRya> zmSCGLk;7mA{I*WWH}4gh&sBPc9idhk3;%g4SuHgiKh=q|G`nhPLCw^zEpK)s&KM zog#5buwU2)VvUwK-?SMP({UAb=293tgr`*Q1@L!hS%PQwLvF0PxrFo*S!iKKwK9!P+l)C({uC^sfDX5sy0n(nxT39aX{55%%bbL19uo8R%EsTGnJVw zU3d4c+;qHM7kvQQ#7IHIIBAdYheEWn+SK>oNtf-X#R`eH&b^AnX zdVEi75ObEXFDLji%POac?rR`sHEe(0_DELK1O2A;GTW;2LCIbKvV-R*Wd|E#FtC9T z9@j1XYwP*RSqkyl3Tq8cM)DO6l+fGHivB>co~@v)_;?o*4&WH=1m8QWNvo_O;~J(? z=$Fl$JFBy>Jc5U~fnyKRRei7MKPuJ{d>r#WV9fY?KEyPcp=D)joU*Buuwe7Rp3Is>-Vv7wI5+IxYE)o|DIoVJHuz}!-Sdn{boKR74} z9D8@* z=i|Ybx65{=Z0Fb7yM3>lcN)!ZuhDBZtLyE$+pE{B)o!I;sW;tPrPFM0Ha9D6x81eR VCtBU=R! literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/.part-00001-f2126b8d-1594-451b-9c89-c4c2481bfd93-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/.part-00001-f2126b8d-1594-451b-9c89-c4c2481bfd93-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..44e7f12cd439d4caaa416067ddc997385850632e GIT binary patch literal 16 XcmYc;N@ieSU}7lN-C1?a-fklRBKZXZ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..3a137a97cb6 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1629872975334,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputBytes":"1318","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1629872972259}} +{"add":{"path":"part-00000-e24c5388-1621-46bd-94eb-fea5209018d0-c000.snappy.parquet","partitionValues":{},"size":659,"modificationTime":1629872975000,"dataChange":true}} +{"add":{"path":"part-00001-f2126b8d-1594-451b-9c89-c4c2481bfd93-c000.snappy.parquet","partitionValues":{},"size":659,"modificationTime":1629872975000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/part-00000-e24c5388-1621-46bd-94eb-fea5209018d0-c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-non-partitioned/part-00000-e24c5388-1621-46bd-94eb-fea5209018d0-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6d12d13f2733b48d1be1bbe4ade538726de30383 GIT binary patch literal 659 zcmah{!Ak2u6rIEqF$RwHc1=FM^sUoGn;6l*F z?7P>T)Mkpuw^^8*D)Pn9IlzjRsgJ9E$czE9<`1+{AL~7 z0Z$+cvpat^2e(1OFqK62UE*1T&hV2}+fzFN56CKjjejr_*&ZQfHDY2Sz%yIeXScGd zv$7zoJ;oVFWd94IzR4MEdNY*`tI9hzmW@b6Bgm6Bqs{$9(gCfTY8p4xqiH7dq^5(* z87;(EwPGL2OITJkawID#6w^ExQ4RoLN{_ya(!p##_ddwo4>HWiiqTKp-*ZGxmv(=R zD|fk{>ocD--`$I*S)RC}A0}x(4Z~hH3S=4tyOS;rwIfd+PFc4y|jc>{ZSb`=sJZ^0%G<8JdlzA;cm+d3}Mw0(db)2YFqGyC{(UC2{l|Yda@GZe6F^3(eu*avOVn zV!E?;gLWyK&mY_LlUhjRB<7(>%`^lu*j`45D6sE(ZZNbxM4a?&aCHMQoxbzkaBhsD zJ^Rl(u7{eC7H1EC86FuRg=K4s(H+H411qx!Qr$rBh&Tl6fHHpLNN@v6>t@d7Qle(Q zmCwJZrY-7Qjs+|) zRGGDG`hLwSxzMFF#PqJMs5&)Q4JN8pS<0lu21KfW-I22Cdb0`c!J~UH&M2z+Q`iq1 zpyxZeZx)Te+s_YqD0t`}C99&$d^w4;e3HlUc$7pckD~E76Oo8wbt2L@IgL+6qLM5; Qb^{Lp5BxCzF7Z451%Hf+jQ{`u literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f6fda0f0d2795ac9996fbebbd6063d65b330a2ca GIT binary patch literal 16 XcmYc;N@ieSU}DJLF~dYk{hkT{A{_*6 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00001-6eb569ba-9300-49e7-9b5a-d064e8c5be2d-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/.part-00001-6eb569ba-9300-49e7-9b5a-d064e8c5be2d-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..9ac8f4f5fef6bf44c10d87a0d2c028d25fe5e57b GIT binary patch literal 16 XcmYc;N@ieSU}9J;vu%x(`aKl@BIN|9 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..4b60de03f85 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/_delta_log/00000000000000000000.json @@ -0,0 +1,5 @@ +{"commitInfo":{"timestamp":1629872770300,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputBytes":"1306","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1629872768383}} +{"add":{"path":"part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet","partitionValues":{},"size":653,"modificationTime":1629872770000,"dataChange":true}} +{"add":{"path":"part-00001-6eb569ba-9300-49e7-9b5a-d064e8c5be2d-c000.snappy.parquet","partitionValues":{},"size":653,"modificationTime":1629872770000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-not-allow-write/part-00000-fab61bc4-5175-46ea-ac35-249c0f5750ff-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4b75e6d8a35fa56e30561f815ac7415daa27f44a GIT binary patch literal 653 zcmah{!D`z;5M3#Xag7fluwu6eDwf71pe(C6RxIO7Zz1#$dJQ4$S}R9HmTXB*DaOCh zLkm5W{$l^3vyPLL77~zlcIM4{^9J_v!>14=}3~0f>$sn3;>|h_i{|EuHy|F2!Whu7N%-P4Tnd5BcM4%0LNX#jq`HlaeZS8ch zO~>cmdM>NGn|gAqXA(5AeP9zyBHT4m9e}+K^=Qy*d(b~A-0j1@)-#!vbLZk4+kw{eT8s6TrkTn22aVNLfv}oJ;V_H`e*R zuIj8T$ZnqtE)Y5TucKb%9JX96b;GOjGoR^3CbE&_yR~A?m$_ymRyWmh(y)M~na-1% zjjmR#kTU~GsBXKmqR~@bL7`aYxl}peK-2+y)MTU8dL2A*)iW0+WX1F~>h&6ti>=w? zS>U&*s7R+#)bA%E5>c!NViK#f_)I9RlJJNI9zyV; L-|&zw;UWG3K=9bS{(o0fBO8>#lC!zPREeMm#DEokB^ZUh!n+O#X7HVXZUYwO~z zX}hwi>G4EJA%UL#HPKyF0k+> zpTF4gTCFXluzQ#)>)xzPi;a3)RZ3TgBhkeCMw8EX`+fYAtABBELs_r>M}4^m`t)G- z;ku0u*F}j)q)4Juy)COeQnM^CW<{1w&UI>vG@VRxnaVUXQ@PCaMRp;z(Rp&l;t&8I L_~rmy;*a6ht06)j% zNLQ`G1t}sZk9Kf;dR1z72$r*=-KS%`inkyYjmg-&7LY$~+b2Tj8{_6QCf%;KV zC8U{z%+e|2a}K6i;h(*VGz(`5Iq5KqS%|cM5ZNMP(CT6-a+c2CSS)hx^PH3S%7*6a zxu64@<>_*gQ+>@f;C2PN(Brj^((b bYfW6Q=k_d5c)s1%OdTN@=(B>*82;NA_kWHR literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/.part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..8c26ac11ed50250792f542f378645076718d0f78 GIT binary patch literal 16 XcmYc;N@ieSU}AVaEA(H5%olb5DDefJ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180520/city=bj/part-00001-4c732f0f-a473-400a-8ba3-1499f599b8f1.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4787f1ca94e6c869008f1894585bc272ade859b4 GIT binary patch literal 650 zcmah{O-sW-6x^gA6e)Od!v=zs#g-Q0l31EHr5A7Fsfafb*)-c4Owx9fC{hX@Jo*Ft zmHq}_lPVDfA-v7Yo0&J0WO{yOV}y?}HL(geb=)vW?OJBhYJX^``=dt)9gb9}6{Q9> zhXU~klafMJff30o@Jk&$J~%1ZI|x?w$wf0+artnU%J7cXU{zDK7e#wjI*rLc)^!7%f+TG2yeS53FNAjbxlXJ}7)%Q1 zfBG*;XdjBEBHbipDHn{dI0RsM7WOD)5-ej<>obd4i1dCTl6A-+_BsX!2(DZpF=$Iy{%w{Py={yjjpV0BmhVEVqjPk=Q0fTq}z&Hy-?uCE=sxax7DjjdO zTl1R@aRI#F62z%k2>BoTc8kd6-tlQ9Tl;LchgFu0J2mUAKScIb!ZO j{l0HGmg9(*u(dOtxv$VD+mgse%T_>l^@fErPLLt7Ak6!uKfK z;58N(tcu9osPK5_@S97CKDcZZnam&Wi{XPxBy9j~9TC<@WulG%D8ZsWOiR#+Zs=yG zY0#G5h3%%k#89FUE!MJ5bMS{<-9UdRguOd22DV(X7ort8)$-2EVCnW(&l>{Tdrc#t zn~aU~k&u~0)1vY(o>X2$$Il{Br3_>#*;{SK%hyct zJ}=6AIxM-#!$`$J!TYx}zIY0}sy1^27^NcwrqeiO7Qw47 zQCg~z%Ku~E&4FF59G@n6do^O)7Pckq_OU;W;-D>iUJ&*|&vVXvSB0+YIDzF_uBW=z a(DTo|bIVtLV4vuw0e}lWY5>>x-#!7KP>tvS literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/.part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..811ff4f5a12edcf0b328b386e0995dedc8155da1 GIT binary patch literal 16 XcmYc;N@ieSU}Ct#`qfk6!N%PHCD#SU literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-partition-prune/date=20180718/city=hz/part-00000-f888e95b-c831-43fe-bba8-3dbf43b4eb86.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..019d84d3fc80a8c86cdc8e6d02b28bdbc9b23902 GIT binary patch literal 639 zcmah{O-sW-5Z$Cvixf|8*gy_pv89E$B*vyq>BXD$RJ@AFrrFkDKH4O$Xz9gY;Scn0 z2u|8+i-N*3yR-A&n>Wi2uWoFNu#OEA%WycvRh=kVhIDz zDJJ{mBd8Fij3s%O!Nu{}hFo4^S?gcd@+r?A@AJ`vh&fgeuGM9tBs>=N5&=nNloUvn zHC;0sRUOx~Hae_o8w{3JSzV|vjcWf7&zcS%K@55}UL_s+ZX`eNDNJz(v3S27vfe#Cds4*)3nV)ky3Es(hu+} z{RVH-DiMW3xRaZE&bj9#xjw(KFv7=}8d!##CRTLPxQdd1?+z99e)R~U!;uWNCN-dz zP#_**Qc+{0A}PxQ=kEBa?cRXSd+ z*TxSU;sSWLB}#HJ73n_q-5Qb0o#WG4+TKNMTg)<<)$Vyk6bEfSa)WRbx~_fZIU;l% i+YU^}bX+koeb*bhL(>ypVD&XaM+h!dgRgN7f8iJ8K$Nxs literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..df728b31bb4 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629873032991,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"c2\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"1734","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["c2"],"configuration":{},"createdTime":1629873029858}} +{"add":{"path":"c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":436,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00000-786c7455-9587-454f-9a4c-de0b22b62bbd.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":431,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":431,"modificationTime":1629873032000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":436,"modificationTime":1629873032000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..36aeb8d3d89c8dd92b8c7136e2e8e7cdb70e573a GIT binary patch literal 12 TcmYc;N@ieSU}7-UyO{<65cUGy literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/.part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..dc43e92d33a435c0e5f1a18f4ffa4732b494ad45 GIT binary patch literal 12 TcmYc;N@ieSU}8vAp5Orh5Y+;g literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00000-2bcc9ff6-0551-4401-bd22-d361a60627e3.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c7bda6510defa907862eb01a0f8e3dbf95a80b08 GIT binary patch literal 436 zcmaKp!Arw16vneUYY(1;loWa>LB|F)B$jpS@FI8+Ps5vtNZO{e*mhf+DpKabzuZ@u zAP8O_FW>v+eILp4>P}#UPx08tCNw6tTrwOYgh*eIjmE!DAe>wqP!Ep!Vc+!>HXQ%v zxY@}1<*7-YRVlFn;+a8=L?GLAT*n<*E?`sBLQbz@!#3>?`*Sd|F8}d!T*w5gzyxR$ zIX)rldd1{Mf)D$)WUs1j@^wWH1I8IgWb%h-wgrQ9+g52-uisdyv`nRzsz@8Uc<5*)%YKvykoQ<#X|+-{2vu7YGA;mso64s@JL#g^?H+fC-0mIE z%i7yV^aK-}32z*?d69Z@9;Mkli=yBn4pkP0L6Gv0hmo4{B#LLzjK?ZY#l&%4c!AXb Ip(Xs+7klDiTmS$7 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo0/part-00001-ca647ee7-f1ad-4d70-bf02-5d1872324d6f.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..2297174763f7a134b1a7dc197a504f8c1abb3473 GIT binary patch literal 431 zcmaKp!Arw16vks$M-SpHC50SHz_CFMiFMsNym%8&!<&dm+NQJEc3av`bnJikRVE06 zmq+rwU*7levb=qe7-5S20NZdFVuupjK?o6h2-QX7UpElWuPvy9LEr2#YKAY^S; zTx}HC@NY-H)21obH8~Br-~y4!AEMco9Kvlo-Ey;j=ap_%rdmZF_pJTcXtrQYYr3>$ z0m}{% zC6|Irf1GqhnfYoSXZbvj0gqN35t@z0wARx+_bSlmNg>?em|Re6c?qo)3uJA000%;~gV< zfe54sym8VMW#*}QoaOU8j>D@Y(s>kxVJ0IP#d<2!IGM#Wndl@7CX_MQKo09(!dHC( D_~u~{ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-partitioned/c2=foo1/part-00001-1c702e73-89b5-465a-9c6a-25f7559cd150.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..8143adb67f742316adca1a5e99c6bf523667d455 GIT binary patch literal 436 zcmaKp!Arw16vneUy9%C!loWC(LB|F)w3c=2@FI8+Ps5vtNZO{e*mhf*Dl+E5zuZ@u zAP64vNWS;W`#zGz)m?xQKE)#so3JslMM?h%Aw>FzP;E5&bpqk!+JJg6=$Zpq9f}Rd zyE$&wLcKgS>9ec^Hb6Wxh>-@!HXZ8Fp+y0km=_F>pF<%Ni~G!o{ zbJl3prkZ&yFJzfDZ2quiwWzvQ8i2gVidxI1R3OxCS&F0t0B%Z;{mf+Z?QZwDL*#bv zcwQ-YAJGkX;Pb#8C2dh=u9(GHKFi}cyhtLMM^PANe&k27ocL*+OyjAa$RrEK4yEt{ K%N{}t_^&U>J7JRm literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..936ea867cd5 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629872936115,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"b\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"2494","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["b"],"configuration":{},"createdTime":1629872933338}} +{"add":{"path":"b=foo0/part-00000-531fe778-e359-44c9-8c35-7ed2416c5ff5.c000.snappy.parquet","partitionValues":{"b":"foo0"},"size":629,"modificationTime":1629872935000,"dataChange":true}} +{"add":{"path":"b=foo1/part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet","partitionValues":{"b":"foo1"},"size":618,"modificationTime":1629872936000,"dataChange":true}} +{"add":{"path":"b=foo0/part-00001-923b258c-b34c-4cb9-8da9-622005e49f2c.c000.snappy.parquet","partitionValues":{"b":"foo0"},"size":618,"modificationTime":1629872935000,"dataChange":true}} +{"add":{"path":"b=foo1/part-00001-e44bca08-b26b-4f4d-8a22-5bb45a598dcf.c000.snappy.parquet","partitionValues":{"b":"foo1"},"size":629,"modificationTime":1629872936000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/.part-00000-531fe778-e359-44c9-8c35-7ed2416c5ff5.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/.part-00000-531fe778-e359-44c9-8c35-7ed2416c5ff5.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..1edf32c3c25612f65c6557442ce6c29283d8499c GIT binary patch literal 16 XcmYc;N@ieSU}88~Xt?KpP~v6)CjzzlwHq$-520Z>2ouw`>|6WJXg7wR1b18viIzCN{b;zPxiN@W z1`kH%@MsJ_M1Lo=?{OA59;MS_%;ijCQoah!|EQu&r#Y?nxZncl{s%^RlW~0C&0J|- zO#kwU(lU`+(*MSS>3=iD2CUS@Jl4!(qf}){$p%*omdi;6N$jp!X|B~+71+q1k!Q4sWci3mgxszVWn(NTCLA{8b6zT>$7xZueExWEth4E>afC;$Ke literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/part-00001-923b258c-b34c-4cb9-8da9-622005e49f2c.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo0/part-00001-923b258c-b34c-4cb9-8da9-622005e49f2c.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..7db5cabb7e414578559d91526179c97deafea74f GIT binary patch literal 618 zcmaiyK}*9h7>2*rZuBs`rKHe92{O$1eT_|Ds2K zgkN=2K~MH)<-01;pGjsUd-Mnl;_Q z5SHt?LD%-se*kvdMVXk6<=i_@U3>ILUf08Xcp~21eKtJO+3@KX|7?tYW&VLVZ-!%m z8&Eo{CS1-X#^fJ~`Ms*@d{)xdfD0~w?*4MrcLm4C-7S>n)$D;!m6n;*lD-&ArXS~u z4Oy+LMXFiA#IkZUZO&xFOiD% Yau^?qL?v0c>joYG4tO;H&hVGL1E9)@*8l(j literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..3e7d4c163ea08b635b73be2b392cc2a67879f2b2 GIT binary patch literal 16 XcmYc;N@ieSU}D(vKh2xD^JX^yCd37g literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00001-e44bca08-b26b-4f4d-8a22-5bb45a598dcf.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/.part-00001-e44bca08-b26b-4f4d-8a22-5bb45a598dcf.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..203c537db659872c692d97361e11dc058f802059 GIT binary patch literal 16 XcmYc;N@ieSU}C5h-u~c!P~v6)CF%vo literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-schema-match/b=foo1/part-00000-7dad1d59-f42c-46c1-992e-35c2fb4d9c09.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..66cd4ea63ff3fa1ce735b2f00a48a435c40e0e6a GIT binary patch literal 618 zcmaiyPfNov7{=4ClOBe*loWa>0mlY4q;Bi%*u|T860a(fw%IIp-PUFk9lQ7y{E8m^ z5WbuH2LpwWym_DJmpp;oUfuZ^VF$Y&HlR4crbCPhLWmd|%ob`K?4WVl)?uzxR^rMA z49D#~9H2&3$x``by>PIQhhL-$^;+5%9P4)fx>im^@$yv0&vGQNfv|q26HOpoTdnaK zgfMK!_Ufhs{thxvY>=^T8P?o-ubcfp^4bpMgA?KA`iJJ=+Ln)Z{D;;*%6x!X-@LG>;tRG-L3%=}anCj9=MEDv<~!$fvfT>TM!v zkCv*K#fo}#kjgA6Y43hPb1_;;0=gSlnkzY!1vK(mmWe0>9YE>P?U}TkM)6_H6Yt-&;{$<$>j_25Co z>;6dphHpx>f}k*;@AL6K19SH4+$TWxNXH`<9xT#uX>A(-XswQ`4V@1+EbRUTAfw;zmlWB5gI$HkCni5=U|hD(}n2Jy<^ z$*3Hjjp3*0AB6TJ_5#PFbXttLoJmZ|SE2cxD#~=4(|V5!E`T0YA12T8&kKg?yf6GRhD~ zq4L<9CmSr5OYdD9Z?xDU&BxzKFPA`1*I^GQMQi=Kmd|~`{Z=QMr&-dH{V*BzM`76O mMu8dyL9dsHKm?&W5^)$EhsPpPQQ{vso(q5jUJQUU{BW;r(28FG literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..1460c7c2e56 --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629873142667,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"c2\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"1734","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["c2"],"configuration":{},"createdTime":1629873139851}} +{"add":{"path":"c2=+%20%253D%25250/part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet","partitionValues":{"c2":"+ =%0"},"size":436,"modificationTime":1629873142000,"dataChange":true}} +{"add":{"path":"c2=+%20%253D%25251/part-00000-180d1a36-4ba9-4321-8145-1e0d73406b02.c000.snappy.parquet","partitionValues":{"c2":"+ =%1"},"size":431,"modificationTime":1629873142000,"dataChange":true}} +{"add":{"path":"c2=+%20%253D%25250/part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet","partitionValues":{"c2":"+ =%0"},"size":431,"modificationTime":1629873142000,"dataChange":true}} +{"add":{"path":"c2=+%20%253D%25251/part-00001-3379bbbf-1ab8-4781-8b7e-29038d983f83.c000.snappy.parquet","partitionValues":{"c2":"+ =%1"},"size":436,"modificationTime":1629873142000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..e96edb1565d34560a009ea1b5fe739ca204d69e5 GIT binary patch literal 12 TcmYc;N@ieSU}9jIw7wAl5hViw literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/.part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..db681c79214da2b6fb021005ca2affd571152b8d GIT binary patch literal 12 TcmYc;N@ieSU}D&Od&V;W6!Qc} literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00000-88ad45a3-9b80-4e66-b474-1748ba085060.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..23ff419188eb7757ba5938fcb18c4d833a369ee7 GIT binary patch literal 436 zcmaKp!Arw16vneUYY(1;loWa>LB|F)B$jpS@FI8+Ps5vtNZO{e*mhf+DpKabzuZ@u zAP8O_FW>v+eILp4>P}#UPx08tCNw6tTrwOYgh*eIjmE!DAe>wqP!Ep!Vc+!>HXQ%v zxY@}1<*7-YRVlFn;+a8=L?GLAT*n<*E?`q+nNF`_!#3>?`*Sd|F8}d!T*w5gzyxR$ zIX)rldd1{Mf)D$)WUs1j@^wWH1I8IgWb%h-wgrQ9+g52-uisdyv`nRzsz@8Uc<5*)%YKvykoQ<#X|+-{2vu7YGA;mso64s@JL#g^?H+fC-0mIE z%i7yV^aK-}32z*?d69Z@9;Mkli=yBn4pkP0L6Gv0hmo4{B#LLzjK?ZY#l&%4c!AXb Ip(Xs+7kk!WTmS$7 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%250/part-00001-aff2b410-c566-4e51-a968-acfa96d6f1e9.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d7d2cb77b7358543ea61811e2c156694d16b547b GIT binary patch literal 431 zcmaKp!Arw16vks$M-SpHC50SHz_CFMiFMsNym%8&!<&dm+NQJEc3av`bnJikRVE06 zmq+rwU*7levb=qe7-5S20NZdFVuupjK?o6h2-QX7UpElWuPvy9LEr2#YKAY^S; zTx}HC@NY-H)21obH8~Br-~y4!AEMco9Kvlo-Ey;j=ap_%rdmZF_pJTcXtrQYYr3>$ z0m}{% zC6|Irf1GqhnfYoSXZbvj0gqN35t@z0wARx+_bSlmNg>?em|Re6c?qo)3uJA000%;~gV< zfe54sym8VMW#*}QoaOU8j>D@Y(s>kxVJ0IP#d<2!IGM#Wndl@7CX_MQKo09(!dHC( D_}XC* literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/part-00001-3379bbbf-1ab8-4781-8b7e-29038d983f83.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-special-chars-in-partition-column/c2=+ %3D%251/part-00001-3379bbbf-1ab8-4781-8b7e-29038d983f83.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ac6100bbdf481023d7663eea241309c33ad9ecea GIT binary patch literal 436 zcmaKp!Arw16vneUy9%C!loWC(LB|F)w3c=2@FI8+Ps5vtNZO{e*mhf*Dl+E5zuZ@u zAP64vNWS;W`#zGz)m?xQKE)#so3JslMM?h%Aw>FzP;E5&bpqk!+JJg6=$Zpq9f}Rd zyE$&wLcKgS>9ec^Hb6Wxh>-@!HXZ8Fp+y0kAj@<*6&to`f7qYBp>_F>pF<%Ni~G!o{ zbJl3prkZ&yFJzfDZ2quiwWzvQ8i2gVidxI1R3OxCS&F0t0B%Z;{mf+Z?QZwDL*#bv zcwQ-YAJGkX;Pb#8C2dh=u9(GHKFi}cyhtLMM^PANe&k27ocL*+OyjAa$RrEK4yEt{ K%N{}t_^&U>FJY4a literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..4f77994c33e --- /dev/null +++ b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/_delta_log/00000000000000000000.json @@ -0,0 +1,7 @@ +{"commitInfo":{"timestamp":1629873109640,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"c2\"]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"4","numOutputBytes":"1734","numOutputRows":"10"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["c2"],"configuration":{},"createdTime":1629873107868}} +{"add":{"path":"c2=foo0/part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":436,"modificationTime":1629873109000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00000-1bb7c99b-be0e-4c49-ae73-9baf5a8a08d0.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":431,"modificationTime":1629873109000,"dataChange":true}} +{"add":{"path":"c2=foo0/part-00001-e7f40ed6-fefa-41f5-b8a6-c6e9b78a1448.c000.snappy.parquet","partitionValues":{"c2":"foo0"},"size":431,"modificationTime":1629873109000,"dataChange":true}} +{"add":{"path":"c2=foo1/part-00001-c357f264-a317-4e93-a530-a8b1360ca9f6.c000.snappy.parquet","partitionValues":{"c2":"foo1"},"size":436,"modificationTime":1629873109000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..5030f64f4ba8fb45c2c2d73d398b88c7d3bb2b2a GIT binary patch literal 12 TcmYc;N@ieSU}8w!$Fd6m5)cC% literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00001-e7f40ed6-fefa-41f5-b8a6-c6e9b78a1448.c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/.part-00001-e7f40ed6-fefa-41f5-b8a6-c6e9b78a1448.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c836919055ab0028eff0107b1c5cadb2756ac23e GIT binary patch literal 12 TcmYc;N@ieSU}E^vUtj?M6cqzz literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo0/part-00000-f1acd078-4e44-4d47-91b2-6568396e2ec3.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1f2aab702184bac94e9eed1e07f125ea571a7224 GIT binary patch literal 436 zcmaKpK}*9x5QVp~#vD8e8#b_quxM!^E?Gm~%YXbF3Om6lFaa8R zJwn!c#pOnV5C690uS(bXx*~@G_qmVAKAY^S; zTx}HC@NY-H)21obH8~Br-~y4!AEMco9Kvlo-Ey;j=ap_%rdmZF_pJTcXtrQYYr3>$ z0m}{% zC6|Irf1GqhnfYoSXZbvj0gqN35t@z0wARx+_bSlmNg>?em|Re6c?qo)3uJA000%;~gV< zfe54sym8VMW#*}QoaOU8j>D@Y(s>kxVJ0IP#d<2!IGM#Wndl@7CX_MQKo09(!dHC( D_~>B} literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/part-00001-c357f264-a317-4e93-a530-a8b1360ca9f6.c000.snappy.parquet b/golden-tables/src/test/resources/golden/hive/deltatbl-touch-files-needed-for-partitioned/c2=foo1/part-00001-c357f264-a317-4e93-a530-a8b1360ca9f6.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..41d81d09d84e7fdc30964e742618f644939c5415 GIT binary patch literal 436 zcmaKp!Arw16vneUy9%C!loWC(LB|F)q?UE-@FI8+Ps5vtNZO{e*mhgmDl+E5zuZ@u zAP64v$ot;!z3-7MuI_w{@F^a7*o2FTElT=F2qDrxglMDDuMC8fYXjoJpeqjGbSO3) z@8-B(Oa1axC(o*s*Z}d&AV%UN+jOWyhZY5FVwuS4RBYI${b7IhhSudjeh!75U=)}D z4ZR*AE4}1$Ey0I>Tk=<>>wHy_!+;Ae5E=g=>P^96yG^Sau2*loRE5U0S+0t-X7h(Ft7O@g5+CF}mRF5hDh)!_7KMxp0H9NO>}MvMZ+E-L9U`}T z$MaIV`xRZE`-1!KC~os2b>%Edvso5J!9^UZEDVDn6`=?tH4#Y^Pot@bRh;@`hf;Wf KWe=eR{MQ%AJz Unit): Unit = { + private def createHiveGoldenTableFile(name: String): File = + new File(createGoldenTableFile("hive"), name) + + private def generateGoldenTable(name: String, + createTableFile: String => File = createGoldenTableFile) (generator: String => Unit): Unit = { if (shouldGenerateGoldenTables) { test(name) { - val tablePath = createGoldenTableFile(name) + val tablePath = createTableFile(name) JavaUtils.deleteRecursively(tablePath) generator(tablePath.getCanonicalPath) } @@ -663,4 +667,110 @@ class GoldenTables extends QueryTest with SharedSparkSession { Seq(row).toDF().write.format("delta").mode("append").partitionBy("_2").save(tablePath) } } + generateGoldenTable("deltatbl-not-allow-write", createHiveGoldenTableFile) { tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("a", "b").write.format("delta").save(tablePath) + } + + generateGoldenTable("deltatbl-schema-match", createHiveGoldenTableFile) { tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}", s"test${x % 3}")) + data.toDF("a", "b", "c").write.format("delta").partitionBy("b").save(tablePath) + } + + generateGoldenTable("deltatbl-non-partitioned", createHiveGoldenTableFile) { tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("c1", "c2").write.format("delta").save(tablePath) + } + + generateGoldenTable("deltatbl-partitioned", createHiveGoldenTableFile) { tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("c1", "c2").write.format("delta").partitionBy("c2").save(tablePath) + } + + generateGoldenTable("deltatbl-partition-prune", createHiveGoldenTableFile) { tablePath => + val data = Seq( + ("hz", "20180520", "Jim", 3), + ("hz", "20180718", "Jone", 7), + ("bj", "20180520", "Trump", 1), + ("sh", "20180512", "Jay", 4), + ("sz", "20181212", "Linda", 8) + ) + data.toDF("city", "date", "name", "cnt") + .write.format("delta").partitionBy("date", "city").save(tablePath) + } + + generateGoldenTable("deltatbl-touch-files-needed-for-partitioned", createHiveGoldenTableFile) { + tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("c1", "c2").write.format("delta").partitionBy("c2").save(tablePath) + } + + generateGoldenTable("deltatbl-special-chars-in-partition-column", createHiveGoldenTableFile) { + tablePath => + val data = (0 until 10).map(x => (x, s"+ =%${x % 2}")) + data.toDF("c1", "c2").write.format("delta").partitionBy("c2").save(tablePath) + } + + generateGoldenTable("deltatbl-map-types-correctly", createHiveGoldenTableFile) { tablePath => + val data = Seq( + TestClass( + 97.toByte, + Array(98.toByte, 99.toByte), + true, + 4, + 5L, + "foo", + 6.0f, + 7.0, + 8.toShort, + new java.sql.Date(60000000L), + new java.sql.Timestamp(60000000L), + new java.math.BigDecimal(12345.6789), + Array("foo", "bar"), + Map("foo" -> 123L), + TestStruct("foo", 456L) + ) + ) + data.toDF.write.format("delta").save(tablePath) + } + + generateGoldenTable("deltatbl-column-names-case-insensitive", createHiveGoldenTableFile) { + tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("FooBar", "BarFoo").write.format("delta").partitionBy("BarFoo").save(tablePath) + } + + generateGoldenTable("deltatbl-deleted-path", createHiveGoldenTableFile) { + tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("c1", "c2").write.format("delta").save(tablePath) + } + + generateGoldenTable("deltatbl-incorrect-format-config", createHiveGoldenTableFile) { tablePath => + val data = (0 until 10).map(x => (x, s"foo${x % 2}")) + data.toDF("a", "b").write.format("delta").save(tablePath) + } } + +case class TestStruct(f1: String, f2: Long) + +/** A special test class that covers all Spark types we support in the Hive connector. */ +case class TestClass( + c1: Byte, + c2: Array[Byte], + c3: Boolean, + c4: Int, + c5: Long, + c6: String, + c7: Float, + c8: Double, + c9: Short, + c10: java.sql.Date, + c11: java.sql.Timestamp, + c12: BigDecimal, + c13: Array[String], + c14: Map[String, Long], + c15: TestStruct +) + +case class OneItem[T](t: T) diff --git a/hive/src/test/java/io/delta/hive/util/JavaUtils.java b/hive/src/test/java/io/delta/hive/util/JavaUtils.java new file mode 100644 index 00000000000..ae02e04d679 --- /dev/null +++ b/hive/src/test/java/io/delta/hive/util/JavaUtils.java @@ -0,0 +1,134 @@ +package io.delta.hive.util; + +import java.io.*; + +import com.google.common.base.Preconditions; + +import org.apache.commons.lang3.SystemUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * General utilities available in the network package. Many of these are sourced from Spark's + * own Utils, just accessible within this package. + */ +public class JavaUtils { + private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); + + + /** + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * + * @param file Input file / dir to be deleted + * @throws IOException if deletion is unsuccessful + */ + public static void deleteRecursively(File file) throws IOException { + deleteRecursively(file, null); + } + + /** + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * + * @param file Input file / dir to be deleted + * @param filter A filename filter that make sure only files / dirs with the satisfied filenames + * are deleted. + * @throws IOException if deletion is unsuccessful + */ + public static void deleteRecursively(File file, FilenameFilter filter) throws IOException { + if (file == null) { return; } + + // On Unix systems, use operating system command to run faster + // If that does not work out, fallback to the Java IO way + if (SystemUtils.IS_OS_UNIX && filter == null) { + try { + deleteRecursivelyUsingUnixNative(file); + return; + } catch (IOException e) { + logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + + "Falling back to Java IO way", file.getAbsolutePath(), e); + } + } + + deleteRecursivelyUsingJavaIO(file, filter); + } + + private static void deleteRecursivelyUsingJavaIO( + File file, + FilenameFilter filter) throws IOException { + if (file.isDirectory() && !isSymlink(file)) { + IOException savedIOException = null; + for (File child : listFilesSafely(file, filter)) { + try { + deleteRecursively(child, filter); + } catch (IOException e) { + // In case of multiple exceptions, only last one will be thrown + savedIOException = e; + } + } + if (savedIOException != null) { + throw savedIOException; + } + } + + // Delete file only when it's a normal file or an empty directory. + if (file.isFile() || (file.isDirectory() && listFilesSafely(file, null).length == 0)) { + boolean deleted = file.delete(); + // Delete can also fail if the file simply did not exist. + if (!deleted && file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + } + + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { + ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); + Process process = null; + int exitCode = -1; + + try { + // In order to avoid deadlocks, consume the stdout (and stderr) of the process + builder.redirectErrorStream(true); + builder.redirectOutput(new File("/dev/null")); + + process = builder.start(); + + exitCode = process.waitFor(); + } catch (Exception e) { + throw new IOException("Failed to delete: " + file.getAbsolutePath(), e); + } finally { + if (process != null) { + process.destroy(); + } + } + + if (exitCode != 0 || file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + + private static File[] listFilesSafely(File file, FilenameFilter filter) throws IOException { + if (file.exists()) { + File[] files = file.listFiles(filter); + if (files == null) { + throw new IOException("Failed to list files for dir: " + file); + } + return files; + } else { + return new File[0]; + } + } + + private static boolean isSymlink(File file) throws IOException { + Preconditions.checkNotNull(file); + File fileInCanonicalDir = null; + if (file.getParent() == null) { + fileInCanonicalDir = file; + } else { + fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); + } + return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); + } +} diff --git a/hive/src/test/scala/io/delta/hive/DeltaHelperTest.scala b/hive/src/test/scala/io/delta/hive/DeltaHelperTest.scala index ac46ea812b6..eb5fe19da54 100644 --- a/hive/src/test/scala/io/delta/hive/DeltaHelperTest.scala +++ b/hive/src/test/scala/io/delta/hive/DeltaHelperTest.scala @@ -1,15 +1,14 @@ package io.delta.hive import scala.collection.JavaConverters._ - import io.delta.standalone.types._ import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoFactory} -import org.apache.spark.SparkFunSuite +import org.scalatest.FunSuite -class DeltaHelperTest extends SparkFunSuite { +class DeltaHelperTest extends FunSuite { test("DeltaHelper checkTableSchema correct") { // scalastyle:off diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala index 71baae5328a..f4f41505408 100644 --- a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala +++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala @@ -19,13 +19,25 @@ package io.delta.hive import java.io.File import io.delta.hive.test.HiveTest -import io.delta.tables.DeltaTable +import io.delta.hive.util.JavaUtils -import org.apache.spark.network.util.JavaUtils import org.scalatest.BeforeAndAfterEach abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { + val hiveGoldenTable = new File(getClass.getResource("/golden/hive").toURI) + + /** + * Create the full table path for the given golden table and execute the test function. + * + * @param name The name of the golden table to load. + * @param testFunc The test to execute which takes the full table path as input arg. + */ + def withHiveGoldenTable(name: String)(testFunc: String => Unit): Unit = { + val tablePath = new File(hiveGoldenTable, name).getCanonicalPath + testFunc(tablePath) + } + test("should not allow to create a non external Delta table") { val e = intercept[Exception] { runQuery( @@ -68,18 +80,13 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("should not allow to write to a Delta table") { withTable("deltaTbl") { - withTempDir { dir => - withSparkSession { spark => - import spark.implicits._ - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - testData.toDS.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) - } + withHiveGoldenTable("deltatbl-not-allow-write") { tablePath => runQuery( s""" |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) |STORED BY 'io.delta.hive.DeltaStorageHandler' - |LOCATION '${dir.getCanonicalPath}'""".stripMargin) + |LOCATION '${tablePath}'""".stripMargin) val e = intercept[Exception] { runQuery("INSERT INTO deltaTbl(a, b) VALUES(123, 'foo')") } @@ -129,21 +136,13 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("Hive schema should match delta's schema") { withTable("deltaTbl") { - withTempDir { dir => - val testData = (0 until 10).map(x => (x, s"foo${x % 2}", s"test${x % 3}")) - - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("a", "b", "c").write.format("delta") - .partitionBy("b").save(dir.getCanonicalPath) - } - + withHiveGoldenTable("deltatbl-schema-match") { tablePath => // column number mismatch var e = intercept[Exception] { runQuery( s""" |create external table deltaTbl(a string, b string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) } @@ -154,7 +153,7 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { runQuery( s""" |create external table deltaTbl(e int, c string, b string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) } @@ -165,7 +164,7 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { runQuery( s""" |create external table deltaTbl(a int, c string, b string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) } @@ -174,75 +173,70 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { } } - test("detect schema changes outside Hive") { - withTable("deltaTbl") { - withTempDir { dir => - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - - withSparkSession { spark => - import spark.implicits._ - testData.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) - } - - runQuery( - s""" - |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) - |STORED BY 'io.delta.hive.DeltaStorageHandler' - |LOCATION '${dir.getCanonicalPath}'""".stripMargin - ) - - checkAnswer("SELECT * FROM deltaTbl", testData) - - // Change the underlying Delta table to a different schema - val testData2 = testData.map(_.swap) - - withSparkSession { spark => - import spark.implicits._ - testData2.toDF("a", "b") - .write - .format("delta") - .mode("overwrite") - .option("overwriteSchema", "true") - .save(dir.getCanonicalPath) - } - - // Should detect the underlying schema change and fail the query - val e = intercept[Exception] { - runQuery("SELECT * FROM deltaTbl") - } - assert(e.getMessage.contains(s"schema is not the same")) - - // Re-create the table because Hive doesn't allow `ALTER TABLE` on a non-native table. - // TODO Investigate whether there is a more convenient way to update the table schema. - runQuery("DROP TABLE deltaTbl") - runQuery( - s""" - |CREATE EXTERNAL TABLE deltaTbl(a STRING, b INT) - |STORED BY 'io.delta.hive.DeltaStorageHandler' - |LOCATION '${dir.getCanonicalPath}'""".stripMargin - ) - - // After fixing the schema, the query should work again. - checkAnswer("SELECT * FROM deltaTbl", testData2) - } - } - } +// test("detect schema changes outside Hive") { +// withTable("deltaTbl") { +// withTempDir { dir => +// val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) +// +// withSparkSession { spark => +// import spark.implicits._ +// testData.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) +// } +// +// runQuery( +// s""" +// |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) +// |STORED BY 'io.delta.hive.DeltaStorageHandler' +// |LOCATION '${dir.getCanonicalPath}'""".stripMargin +// ) +// +// checkAnswer("SELECT * FROM deltaTbl", testData) +// +// // Change the underlying Delta table to a different schema +// val testData2 = testData.map(_.swap) +// +// withSparkSession { spark => +// import spark.implicits._ +// testData2.toDF("a", "b") +// .write +// .format("delta") +// .mode("overwrite") +// .option("overwriteSchema", "true") +// .save(dir.getCanonicalPath) +// } +// +// // Should detect the underlying schema change and fail the query +// val e = intercept[Exception] { +// runQuery("SELECT * FROM deltaTbl") +// } +// assert(e.getMessage.contains(s"schema is not the same")) +// +// // Re-create the table because Hive doesn't allow `ALTER TABLE` on a non-native table. +// // TODO Investigate whether there is a more convenient way to update the table schema. +// runQuery("DROP TABLE deltaTbl") +// runQuery( +// s""" +// |CREATE EXTERNAL TABLE deltaTbl(a STRING, b INT) +// |STORED BY 'io.delta.hive.DeltaStorageHandler' +// |LOCATION '${dir.getCanonicalPath}'""".stripMargin +// ) +// +// // After fixing the schema, the query should work again. +// checkAnswer("SELECT * FROM deltaTbl", testData2) +// } +// } +// } test("read a non-partitioned table") { // Create a Delta table withTable("deltaNonPartitionTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-non-partitioned") { tablePath => val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - withSparkSession{ spark => - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaNonPartitionTbl(c1 int, c2 string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -254,19 +248,13 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("read a partitioned table") { // Create a Delta table withTable("deltaPartitionTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-partitioned") { tablePath => val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta") - .partitionBy("c2").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaPartitionTbl(c1 int, c2 string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -284,7 +272,7 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("partition prune") { withTable("deltaPartitionTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-partition-prune") { tablePath => val testData = Seq( ("hz", "20180520", "Jim", 3), ("hz", "20180718", "Jone", 7), @@ -293,12 +281,6 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { ("sz", "20181212", "Linda", 8) ) - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("city", "date", "name", "cnt").write.format("delta") - .partitionBy("date", "city").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaPartitionTbl( @@ -306,7 +288,7 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { | `date` string, | name string, | cnt int) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -370,24 +352,18 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("should not touch files not needed when querying a partitioned table") { withTable("deltaPartitionTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-touch-files-needed-for-partitioned") { tablePath => val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta") - .partitionBy("c2").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaPartitionTbl(c1 int, c2 string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) // Delete the partition not needed in the below query to verify the partition pruning works - val foo1PartitionFile = new File(dir, "c2=foo1") + val foo1PartitionFile = new File(tablePath, "c2=foo1") assert(foo1PartitionFile.exists()) JavaUtils.deleteRecursively(foo1PartitionFile) checkFilterPushdown( @@ -398,63 +374,57 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { } } - test("auto-detected delta partition change") { - withTable("deltaPartitionTbl") { - withTempDir { dir => - val testData1 = Seq( - ("hz", "20180520", "Jim", 3), - ("hz", "20180718", "Jone", 7) - ) - - withSparkSession { spark => - import spark.implicits._ - testData1.toDS.toDF("city", "date", "name", "cnt").write.format("delta") - .partitionBy("date", "city").save(dir.getCanonicalPath) - - runQuery( - s""" - |create external table deltaPartitionTbl( - | city string, - | `date` string, - | name string, - | cnt int) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' - """.stripMargin - ) - - checkAnswer("select * from deltaPartitionTbl", testData1) - - // insert another partition data - val testData2 = Seq(("bj", "20180520", "Trump", 1)) - testData2.toDS.toDF("city", "date", "name", "cnt").write.mode("append").format("delta") - .partitionBy("date", "city").save(dir.getCanonicalPath) - val testData = testData1 ++ testData2 - checkAnswer("select * from deltaPartitionTbl", testData) - - // delete one partition - val deltaTable = DeltaTable.forPath(spark, dir.getCanonicalPath) - deltaTable.delete("city='hz'") - checkAnswer("select * from deltaPartitionTbl", testData.filterNot(_._1 == "hz")) - } - } - } - } +// test("auto-detected delta partition change") { +// withTable("deltaPartitionTbl") { +// withTempDir { dir => +// val testData1 = Seq( +// ("hz", "20180520", "Jim", 3), +// ("hz", "20180718", "Jone", 7) +// ) +// +// withSparkSession { spark => +// import spark.implicits._ +// testData1.toDS.toDF("city", "date", "name", "cnt").write.format("delta") +// .partitionBy("date", "city").save(dir.getCanonicalPath) +// +// runQuery( +// s""" +// |create external table deltaPartitionTbl( +// | city string, +// | `date` string, +// | name string, +// | cnt int) +// |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' +// """.stripMargin +// ) +// +// checkAnswer("select * from deltaPartitionTbl", testData1) +// +// // insert another partition data +// val testData2 = Seq(("bj", "20180520", "Trump", 1)) +// testData2.toDS.toDF("city", "date", "name", "cnt").write.mode("append").format("delta") +// .partitionBy("date", "city").save(dir.getCanonicalPath) +// val testData = testData1 ++ testData2 +// checkAnswer("select * from deltaPartitionTbl", testData) +// +// // delete one partition +// val deltaTable = DeltaTable.forPath(spark, dir.getCanonicalPath) +// deltaTable.delete("city='hz'") +// checkAnswer("select * from deltaPartitionTbl", testData.filterNot(_._1 == "hz")) +// } +// } +// } +// } test("read a partitioned table that contains special chars in a partition column") { withTable("deltaPartitionTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-special-chars-in-partition-column") { tablePath => val testData = (0 until 10).map(x => (x, s"+ =%${x % 2}")) - withSparkSession { spark => - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta") - .partitionBy("c2").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaPartitionTbl(c1 int, c2 string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -465,39 +435,14 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("map Spark types to Hive types correctly") { withTable("deltaTbl") { - withTempDir { dir => - val testData = Seq( - TestClass( - 97.toByte, - Array(98.toByte, 99.toByte), - true, - 4, - 5L, - "foo", - 6.0f, - 7.0, - 8.toShort, - new java.sql.Date(60000000L), - new java.sql.Timestamp(60000000L), - new java.math.BigDecimal(12345.6789), - Array("foo", "bar"), - Map("foo" -> 123L), - TestStruct("foo", 456L) - ) - ) - - withSparkSession { spark => - import spark.implicits._ - testData.toDF.write.format("delta").save(dir.getCanonicalPath) - } - + withHiveGoldenTable("deltatbl-map-types-correctly") { tablePath => runQuery( s""" |create external table deltaTbl( |c1 tinyint, c2 binary, c3 boolean, c4 int, c5 bigint, c6 string, c7 float, c8 double, |c9 smallint, c10 date, c11 timestamp, c12 decimal(38, 18), c13 array, |c14 map, c15 struct) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -526,19 +471,13 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("column names should be case insensitive") { // Create a Delta table withTable("deltaCaseInsensitiveTest") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-column-names-case-insensitive") { tablePath => val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - withSparkSession{ spark => - import spark.implicits._ - testData.toDS.toDF("FooBar", "BarFoo").write.format("delta") - .partitionBy("BarFoo").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaCaseInsensitiveTest(fooBar int, Barfoo string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) @@ -574,24 +513,19 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { test("fail the query when the path is deleted after the table is created") { withTable("deltaTbl") { - withTempDir { dir => + withHiveGoldenTable("deltatbl-deleted-path") { tablePath => val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - withSparkSession{ spark => - import spark.implicits._ - testData.toDS.toDF("c1", "c2").write.format("delta").save(dir.getCanonicalPath) - } - runQuery( s""" |create external table deltaTbl(c1 int, c2 string) - |stored by 'io.delta.hive.DeltaStorageHandler' location '${dir.getCanonicalPath}' + |stored by 'io.delta.hive.DeltaStorageHandler' location '${tablePath}' """.stripMargin ) checkAnswer("select * from deltaTbl", testData) - JavaUtils.deleteRecursively(dir) + JavaUtils.deleteRecursively(new File(tablePath)) val e = intercept[Exception] { checkAnswer("select * from deltaTbl", testData) @@ -607,19 +541,14 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { case "tez" => "hive.tez.input.format" case other => throw new UnsupportedOperationException(s"Unsupported engine: $other") } - withTempDir { dir => + withHiveGoldenTable("deltatbl-incorrect-format-config") { tablePath => withTable("deltaTbl") { - withSparkSession { spark => - import spark.implicits._ - val testData = (0 until 10).map(x => (x, s"foo${x % 2}")) - testData.toDS.toDF("a", "b").write.format("delta").save(dir.getCanonicalPath) - } runQuery( s""" |CREATE EXTERNAL TABLE deltaTbl(a INT, b STRING) |STORED BY 'io.delta.hive.DeltaStorageHandler' - |LOCATION '${dir.getCanonicalPath}'""".stripMargin) + |LOCATION '${tablePath}'""".stripMargin) withHiveConf(formatKey, "org.apache.hadoop.hive.ql.io.HiveInputFormat") { val e = intercept[Exception] { @@ -633,25 +562,4 @@ abstract class HiveConnectorTest extends HiveTest with BeforeAndAfterEach { } } -case class TestStruct(f1: String, f2: Long) - -/** A special test class that covers all Spark types we support in the Hive connector. */ -case class TestClass( - c1: Byte, - c2: Array[Byte], - c3: Boolean, - c4: Int, - c5: Long, - c6: String, - c7: Float, - c8: Double, - c9: Short, - c10: java.sql.Date, - c11: java.sql.Timestamp, - c12: BigDecimal, - c13: Array[String], - c14: Map[String, Long], - c15: TestStruct -) - case class OneItem[T](t: T) diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala index 7983aca85fc..9d1fc443910 100644 --- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala +++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala @@ -20,6 +20,8 @@ import java.io.{Closeable, File} import java.nio.file.Files import java.util.{Locale, TimeZone} +import io.delta.hive.util.JavaUtils + import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -30,10 +32,6 @@ import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.delta.{DeltaLog, SparkSessionHelper} -// scalastyle:off funsuite import org.scalatest.{BeforeAndAfterAll, FunSuite} // TODO Yarn is using log4j2. Disable its verbose logs. @@ -167,15 +165,6 @@ trait HiveTest extends FunSuite with BeforeAndAfterAll { } } - protected def withSparkSession(f: SparkSession => Unit): Unit = { - val spark = SparkSessionHelper.spark - try f(spark) finally { - // Clean up resources so that we can use new DeltaLog and SparkSession - spark.stop() - DeltaLog.clearCache() - } - } - protected def withHiveConf(key: String, value: String)(body: => Unit): Unit = { val hiveConfField = driver.getClass.getDeclaredField("conf") hiveConfField.setAccessible(true) diff --git a/hive/src/test/scala/org/apache/spark/sql/delta/SparkSessionHelper.scala b/hive/src/test/scala/org/apache/spark/sql/delta/SparkSessionHelper.scala deleted file mode 100644 index e662c19ff69..00000000000 --- a/hive/src/test/scala/org/apache/spark/sql/delta/SparkSessionHelper.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright (2020) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.delta - -import org.apache.spark.sql.SparkSession - -object SparkSessionHelper { - /** - * Start a special Spark cluster using local mode to process Delta's metadata. The Spark UI has - * been disabled and `SparkListener`s have been removed to reduce the memory usage of Spark. - * `DeltaLog` cache size is also set to "1" if the user doesn't specify it, to cache only the - * recent accessed `DeltaLog`. - */ - def spark: SparkSession = { - // TODO Configure `spark` to pick up the right Hadoop configuration. - if (System.getProperty("delta.log.cacheSize") == null) { - System.setProperty("delta.log.cacheSize", "1") - } - val sparkSession = SparkSession.builder() - .master("local[*]") - .appName("Delta Connector") - .config("spark.ui.enabled", "false") - .getOrCreate() - // Trigger codes that add `SparkListener`s before stopping the listener bus. Otherwise, they - // would fail to add `SparkListener`s. - sparkSession.sharedState - sparkSession.sessionState - sparkSession.sparkContext.listenerBus.stop() - sparkSession - } -} diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 29b569fd689..ec6744923eb 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -177,12 +177,6 @@ This file is divided into 3 sections: - - - ^FunSuite[A-Za-z]*$ - Tests must extend org.apache.spark.SparkFunSuite instead. - - ^println$ diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala index be5cd66a449..42c8f0660fe 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala @@ -30,7 +30,7 @@ import io.delta.standalone.internal.util.DataTypeParser import io.delta.standalone.internal.util.GoldenTableUtils._ import io.delta.standalone.types._ import org.apache.hadoop.conf.Configuration -// scalastyle:off funsuite + import org.scalatest.FunSuite /** diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index 06ce09cffc6..a7c1b6b47d2 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -32,7 +32,6 @@ import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -// scalastyle:off funsuite import org.scalatest.FunSuite /** diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala index 7483762bbcd..bd03aea72d3 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala @@ -33,7 +33,6 @@ import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -// scalastyle:off funsuite import org.scalatest.FunSuite /** diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala index 8bb96301d8b..92a90f5959d 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala @@ -23,7 +23,6 @@ import io.delta.standalone.internal.util.GoldenTableUtils._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -// scalastyle:off funsuite import org.scalatest.FunSuite /** From 8410a20d4699af414ea12b983b2f763f849fbd55 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Mon, 13 Sep 2021 13:09:42 -0700 Subject: [PATCH 068/291] Builder classes for action classes (#123) add builders for action classes & corresponding tests --- .../scala/io/delta/hive/DeltaHelper.scala | 8 +- .../io/delta/standalone/actions/AddFile.java | 51 +++++ .../delta/standalone/actions/CommitInfo.java | 108 ++++++++++ .../io/delta/standalone/actions/JobInfo.java | 53 +++++ .../io/delta/standalone/actions/Metadata.java | 77 ++++++++ .../delta/standalone/actions/RemoveFile.java | 92 +++++++++ .../standalone/internal/DeltaLogSuite.scala | 187 +++++++++++++++++- 7 files changed, 573 insertions(+), 3 deletions(-) diff --git a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala index 71843a80626..bda07d942d4 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala @@ -80,8 +80,12 @@ object DeltaHelper { snapshotToUse.getAllFiles.asScala ).map { addF => // Drop unused potential huge fields - val f = new AddFile(addF.getPath, addF.getPartitionValues, addF.getSize, - addF.getModificationTime, addF.isDataChange, null, null) + val f = AddFile.builder( + addF.getPath, + addF.getPartitionValues, + addF.getSize, + addF.getModificationTime, + addF.isDataChange).build() val status = toFileStatus(fs, rootPath, f, blockSize) localFileToPartition += diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java index 751f7156018..9ddde9ca6d3 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java @@ -127,4 +127,55 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(path, partitionValues, size, modificationTime, dataChange, stats, tags); } + + /** + * @return a new {@code AddFile.Builder} + */ + public static Builder builder(String path, Map partitionValues, long size, + long modificationTime, boolean dataChange) { + return new Builder(path, partitionValues, size, modificationTime, dataChange); + } + + /** + * Builder class for AddFile. Enables construction of AddFile object with default values. + */ + public static class Builder { + // required AddFile fields + private final String path; + private final Map partitionValues; + private final long size; + private final long modificationTime; + private final boolean dataChange; + // optional AddFile fields + private String stats; + private Map tags; + + public Builder(String path, Map partitionValues, long size, + long modificationTime, boolean dataChange) { + this.path = path; + this.partitionValues = partitionValues; + this.size = size; + this.modificationTime = modificationTime; + this.dataChange = dataChange; + } + + public Builder stats(String stats) { + this.stats = stats; + return this; + } + + public Builder tags(Map tags) { + this.tags = tags; + return this; + } + + /** + * @return a new {@code AddFile} with the same properties as {@code this} + */ + public AddFile build() { + AddFile addFile = new AddFile(this.path, this.partitionValues, this.size, + this.modificationTime, this.dataChange, this.stats, this.tags); + return addFile; + } + } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index d96a9678001..9a9ea2355e9 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -196,4 +196,112 @@ public int hashCode() { jobInfo, notebookInfo, clusterId, readVersion, isolationLevel, isBlindAppend, operationMetrics, userMetadata); } + + /** + * @return a new {@code CommitInfo.Builder} + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder class for CommitInfo. Enables construction of CommitInfo object with default values. + */ + public static class Builder { + private Optional version = Optional.empty(); + private Timestamp timestamp; + private Optional userId = Optional.empty(); + private Optional userName = Optional.empty(); + private String operation; + private Map operationParameters; + private Optional jobInfo = Optional.empty(); + private Optional notebookInfo = Optional.empty(); + private Optional clusterId = Optional.empty(); + private Optional readVersion = Optional.empty(); + private Optional isolationLevel = Optional.empty(); + private Optional isBlindAppend = Optional.empty(); + private Optional> operationMetrics = Optional.empty(); + private Optional userMetadata = Optional.empty(); + + public Builder version(Long version) { + this.version = Optional.of(version); + return this; + } + + public Builder timestamp(Timestamp timestamp) { + this.timestamp = timestamp; + return this; + } + + public Builder userId(String userId) { + this.userId = Optional.of(userId); + return this; + } + + public Builder userName(String userName) { + this.userName = Optional.of(userName); + return this; + } + + public Builder operation(String operation) { + this.operation = operation; + return this; + } + + public Builder operationParameters(Map operationParameters) { + this.operationParameters = operationParameters; + return this; + } + + public Builder jobInfo(JobInfo jobInfo) { + this.jobInfo = Optional.of(jobInfo); + return this; + } + + public Builder notebookInfo(NotebookInfo notebookInfo ) { + this.notebookInfo = Optional.of(notebookInfo); + return this; + } + + public Builder clusterId(String clusterId) { + this.clusterId = Optional.of(clusterId); + return this; + } + + public Builder readVersion(Long readVersion) { + this.readVersion = Optional.of(readVersion); + return this; + } + + public Builder isolationLevel(String isolationLevel) { + this.isolationLevel = Optional.of(isolationLevel); + return this; + } + + public Builder isBlindAppend(Boolean isBlindAppend) { + this.isBlindAppend = Optional.of(isBlindAppend); + return this; + } + + public Builder operationMetrics(Map operationMetrics) { + this.operationMetrics = Optional.of(operationMetrics); + return this; + } + + public Builder userMetadata(String userMetadata) { + this.userMetadata = Optional.of(userMetadata); + return this; + } + + /** + * @return a new {@code CommitInfo} with the same properties as {@code this} + */ + public CommitInfo build() { + CommitInfo commitInfo = new CommitInfo(this.version, this.timestamp, this.userId, + this.userName, this.operation, this.operationParameters, this.jobInfo, + this.notebookInfo, this.clusterId, this.readVersion, this.isolationLevel, + this.isBlindAppend, this.operationMetrics, this.userMetadata); + return commitInfo; + } + } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java index 64b02c440e7..2dbe4b8f0a4 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java @@ -69,4 +69,57 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(jobId, jobName, runId, jobOwnerId, triggerType); } + + /** + * @return a new {@code JobInfo.Builder} + */ + public static Builder builder(String jobId) { + return new Builder(jobId); + } + + /** + * Builder class for JobInfo. Enables construction of JobInfo object with default values. + */ + public static class Builder { + // required JobInfo fields + private final String jobId; + // optional JobInfo fields + private String jobName; + private String runId; + private String jobOwnerId; + private String triggerType; + + public Builder(String jobId) { + this.jobId = jobId; + } + + public Builder jobName(String jobName) { + this.jobName = jobName; + return this; + } + + public Builder runId(String runId) { + this.runId = runId; + return this; + } + + public Builder jobOwnerId(String jobOwnerId) { + this.jobOwnerId = jobOwnerId; + return this; + } + + public Builder triggerType(String triggerType) { + this.triggerType = triggerType; + return this; + } + + /** + * @return a new {@code JobInfo} with the same properties as {@code this} + */ + public JobInfo build() { + JobInfo jobInfo = new JobInfo(this.jobId, this.jobName, this.runId, this.jobOwnerId, + this.triggerType); + return jobInfo; + } + } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index 928434b7230..087db579e26 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -131,4 +131,81 @@ public int hashCode() { return Objects.hash(id, name, description, format, partitionColumns, configuration, createdTime, schema); } + + /** + * @return a new {@code Metadata.Builder} + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder class for Metadata. Enables construction of Metadata object with default values. + */ + public static class Builder { + private String id = java.util.UUID.randomUUID().toString(); + private String name; + private String description; + private Format format = new Format("parquet", Collections.emptyMap()); + private List partitionColumns = Collections.emptyList(); + private Map configuration = Collections.emptyMap(); + private Optional createdTime = Optional.of(System.currentTimeMillis()); + private StructType schema; + + public Builder id(String id) { + this.id = id; + return this; + } + + public Builder name(String name) { + this.name = name; + return this; + } + + public Builder description(String description) { + this.description = description; + return this; + } + + public Builder format(Format format) { + this.format = format; + return this; + } + + public Builder partitionColumns(List partitionColumns) { + this.partitionColumns = partitionColumns; + return this; + } + + public Builder configuration(Map configuration) { + this.configuration = configuration; + return this; + } + + public Builder createdTime(Long createdTime) { + this.createdTime = Optional.of(createdTime); + return this; + } + + public Builder schema(StructType schema) { + this.schema = schema; + return this; + } + + /** + * @return a new {@code Metadata} with the same properties as {@code this} + */ + public Metadata build() { + Metadata metadata = new Metadata( + this.id, + this.name, + this.description, + this.format, + this.partitionColumns, + this.configuration, + this.createdTime, + this.schema); + return metadata; + } + } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java index f3ad9d1fff9..63b7367a6cb 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java @@ -2,6 +2,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.Optional; public class RemoveFile implements FileAction { @@ -54,4 +55,95 @@ public long getSize() { public Map getTags() { return Collections.unmodifiableMap(tags); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RemoveFile removeFile = (RemoveFile) o; + return Objects.equals(path, removeFile.path) && + Objects.equals(deletionTimestamp, removeFile.deletionTimestamp) && + Objects.equals(dataChange, removeFile.dataChange) && + Objects.equals(extendedFileMetadata, removeFile.extendedFileMetadata) && + Objects.equals(partitionValues, removeFile.partitionValues) && + Objects.equals(size, removeFile.size) && + Objects.equals(tags, removeFile.tags); + } + + @Override + public int hashCode() { + return Objects.hash(path, deletionTimestamp, dataChange, extendedFileMetadata, + partitionValues, size, tags); + } + + /** + * @return a new {@code RemoveFile.Builder} + */ + public static Builder builder(String path) { + return new Builder(path); + } + + /** + * Builder class for RemoveFile. Enables construction of RemoveFile object with default values. + */ + public static class Builder { + // required RemoveFile fields + private final String path; + // optional RemoveFile fields + private Optional deletionTimestamp = Optional.empty(); + private boolean dataChange = true; + private boolean extendedFileMetadata = false; + private Map partitionValues; + private long size = 0; + private Map tags; + + public Builder(String path) { + this.path = path; + } + + public Builder deletionTimestamp(Long deletionTimestamp) { + this.deletionTimestamp = Optional.of(deletionTimestamp); + return this; + } + + public Builder dataChange(boolean dataChange) { + this.dataChange = dataChange; + return this; + } + + public Builder extendedFileMetadata(boolean extendedFileMetadata) { + this.extendedFileMetadata = extendedFileMetadata; + return this; + } + + public Builder partitionValues(Map partitionValues) { + this.partitionValues = partitionValues; + return this; + } + + public Builder size(long size) { + this.size = size; + return this; + } + + public Builder tags(Map tags) { + this.tags = tags; + return this; + } + + /** + * @return a new {@code RemoveFile} with the same properties as {@code this} + */ + public RemoveFile build() { + RemoveFile removeFile = new RemoveFile( + this.path, + this.deletionTimestamp, + this.dataChange, + this.extendedFileMetadata, + this.partitionValues, + this.size, + this.tags); + return removeFile; + } + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index a7c1b6b47d2..e323abd53e9 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -19,15 +19,20 @@ package io.delta.standalone.internal import java.io.File import java.nio.file.Files import java.sql.Timestamp +import java.util.Collections +import java.util.Optional import java.util.UUID import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog, Snapshot} -import io.delta.standalone.actions.{JobInfo => JobInfoJ, NotebookInfo => NotebookInfoJ} +import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, + Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, + RemoveFile => RemoveFileJ} import io.delta.standalone.internal.actions.Action import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.GoldenTableUtils._ +import io.delta.standalone.types.{IntegerType, StructField => StructFieldJ, StructType => StructTypeJ} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -346,4 +351,184 @@ class DeltaLogSuite extends FunSuite { } } + test("builder action class constructor for Metadata") { + val metadataFromBuilderDefaults = MetadataJ.builder().build() + val metadataFromConstructorDefaults = new MetadataJ( + metadataFromBuilderDefaults.getId(), + null, + null, + new FormatJ("parquet", Collections.emptyMap()), + Collections.emptyList(), + Collections.emptyMap(), + metadataFromBuilderDefaults.getCreatedTime(), + null); + assert(metadataFromBuilderDefaults == metadataFromConstructorDefaults) + + val metadataFromBuilder = MetadataJ.builder() + .id("test_id") + .name("test_name") + .description("test_description") + .format(new FormatJ("csv", Collections.emptyMap())) + .partitionColumns(List("id", "name").asJava) + .configuration(Map("test"->"foo").asJava) + .createdTime(0L) + .schema(new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) + .build() + val metadataFromConstructor = new MetadataJ( + "test_id", + "test_name", + "test_description", + new FormatJ("csv", Collections.emptyMap()), + List("id", "name").asJava, + Map("test"->"foo").asJava, + Optional.of(0L), + new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) + assert(metadataFromBuilder == metadataFromConstructor) + } + + test("builder action class constructor for AddFile") { + val addFileFromBuilderDefaults = AddFileJ.builder( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true).build() + val addFileFromConstructorDefaults = new AddFileJ( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true, + null, + null) + assert(addFileFromBuilderDefaults == addFileFromConstructorDefaults) + + val addFileFromBuilder = AddFileJ.builder( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true) + .stats("test_stats") + .tags(Map("test"->"foo").asJava) + .build() + val addFileFromConstructor = new AddFileJ( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true, + "test_stats", + Map("test"->"foo").asJava) + assert(addFileFromBuilder == addFileFromConstructor) + } + + test("builder action class constructor for JobInfo") { + val jobInfoFromBuilderDefaults = JobInfoJ.builder("test").build() + val jobInfoFromConstructorDefaults = new JobInfoJ( + "test", + null, + null, + null, + null) + assert(jobInfoFromBuilderDefaults == jobInfoFromConstructorDefaults) + + val jobInfoFromBuilder = JobInfoJ.builder("test") + .jobName("test_name") + .runId("test_id") + .jobOwnerId("test_job_id") + .triggerType("test_trigger_type") + .build() + val jobInfoFromConstructor = new JobInfoJ( + "test", + "test_name", + "test_id", + "test_job_id", + "test_trigger_type") + assert(jobInfoFromBuilder == jobInfoFromConstructor) + } + + test("builder action class constructor for RemoveFile") { + val removeFileJFromBuilderDefaults = RemoveFileJ.builder("/test").build() + val removeFileJFromConstructorDefaults = new RemoveFileJ( + "/test", + Optional.empty(), + true, + false, + null, + 0L, + null) + assert(removeFileJFromBuilderDefaults == removeFileJFromConstructorDefaults) + + val removeFileJFromBuilder = RemoveFileJ.builder("/test") + .deletionTimestamp(0L) + .dataChange(false) + .extendedFileMetadata(true) + .partitionValues(Map("test"->"foo").asJava) + .size(1L) + .tags(Map("tag"->"foo").asJava) + .build() + val removeFileJFromConstructor = new RemoveFileJ( + "/test", + Optional.of(0L), + false, + true, + Map("test"->"foo").asJava, + 1L, + Map("tag"->"foo").asJava) + assert(removeFileJFromBuilder == removeFileJFromConstructor) + } + + test("builder action class constructor for CommitInfo") { + val commitInfoFromBuilderDefaults = CommitInfoJ.builder().build() + val commitInfoFromConstructorDefaults = new CommitInfoJ( + Optional.empty(), + null, + Optional.empty(), + Optional.empty(), + null, + null, + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty()) + assert(commitInfoFromBuilderDefaults == commitInfoFromConstructorDefaults) + + val commitInfoFromBuilder = CommitInfoJ.builder() + .version(0L) + .timestamp(new Timestamp(1540415658000L)) + .userId("test_id") + .userName("test_name") + .operation("test_op") + .operationParameters(Map("test"->"op").asJava) + .jobInfo(JobInfoJ.builder("test").build()) + .notebookInfo(new NotebookInfoJ("test")) + .clusterId("test_clusterId") + .readVersion(0L) + .isolationLevel("test_level") + .isBlindAppend(true) + .operationMetrics(Map("test"->"metric").asJava) + .userMetadata("user_metadata") + .build() + val commitInfoFromConstructor = new CommitInfoJ( + Optional.of(0L), + new Timestamp(1540415658000L), + Optional.of("test_id"), + Optional.of("test_name"), + "test_op", + Map("test"->"op").asJava, + Optional.of(JobInfoJ.builder("test").build()), + Optional.of(new NotebookInfoJ("test")), + Optional.of("test_clusterId"), + Optional.of(0L), + Optional.of("test_level"), + Optional.of(true), + Optional.of(Map("test"->"metric").asJava), + Optional.of("user_metadata")) + assert(commitInfoFromBuilder == commitInfoFromConstructor) + } } From 17c00fb873dffc6cdaad7e2e84cd2b67d84ce95c Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 17 Sep 2021 09:29:43 -0700 Subject: [PATCH 069/291] Expression framework prototype (#37) (#138) * WIP; have a basic skeleton for expressions; have a column resolver; todo comparator * WIP; figuring out comparators * finished first pass at a basic skeleton, using just Ints and Booleans Literal types * add leaf expression; make expression and predicate both ABCs; add children() and bound() methods * add nullSafeBoundEval method to BinaryExpression * add verifyInputDataTypes function to Expression * big refactor; add DataType to Column constructor; no more need for 'bound' checks; use nullSafeEval; refactor Comparator usage; short circuit null checks in eval * rename createColumn to column * Update ExpressionSuite.scala * add IsNotNull predicate; test; null check to Column::eval * make Expression interface; add back Predicate interface with default dataType field; make more member fields final * add Not expression; add nullSafeEval to UnaryExpression; add test for Not expr --- .../io/delta/standalone/expressions/And.java | 20 +++ .../expressions/BinaryComparison.java | 20 +++ .../expressions/BinaryExpression.java | 38 ++++++ .../expressions/BinaryOperator.java | 26 ++++ .../expressions/CastingComparator.java | 15 +++ .../delta/standalone/expressions/Column.java | 55 ++++++++ .../delta/standalone/expressions/EqualTo.java | 16 +++ .../standalone/expressions/Expression.java | 32 +++++ .../io/delta/standalone/expressions/In.java | 69 +++++++++++ .../standalone/expressions/IsNotNull.java | 24 ++++ .../expressions/LeafExpression.java | 11 ++ .../standalone/expressions/LessThan.java | 15 +++ .../delta/standalone/expressions/Literal.java | 68 ++++++++++ .../io/delta/standalone/expressions/Not.java | 24 ++++ .../io/delta/standalone/expressions/Or.java | 20 +++ .../standalone/expressions/Predicate.java | 11 ++ .../expressions/UnaryExpression.java | 36 ++++++ .../io/delta/standalone/expressions/Util.java | 20 +++ .../io/delta/standalone/types/StructType.java | 7 ++ .../standalone/internal/DeltaLogImpl.scala | 25 +++- .../internal/data/PartitionRowRecord.scala | 77 ++++++++++++ .../standalone/internal/ExpressionSuite.scala | 117 ++++++++++++++++++ 22 files changed, 745 insertions(+), 1 deletion(-) create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/And.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/CastingComparator.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Column.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Expression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/In.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/LessThan.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Literal.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Not.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Or.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Predicate.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Util.java create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/expressions/And.java b/standalone/src/main/java/io/delta/standalone/expressions/And.java new file mode 100644 index 00000000000..bc580b585ef --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/And.java @@ -0,0 +1,20 @@ +package io.delta.standalone.expressions; + +/** + * Usage: new And(expr1, expr2) - Logical AND + */ +public final class And extends BinaryOperator implements Predicate { + + public And(Expression left, Expression right) { + super(left, right, "&&"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + if (!(leftResult instanceof Boolean) || !(rightResult instanceof Boolean)) { + throw new RuntimeException("'And' expression children.eval results must be Booleans"); + } + + return (boolean) leftResult && (boolean) rightResult; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java new file mode 100644 index 00000000000..16b86a8bbfa --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java @@ -0,0 +1,20 @@ +package io.delta.standalone.expressions; + +/** + * A [[BinaryOperator]] that compares the left and right [[Expression]]s and returns a boolean value. + */ +public abstract class BinaryComparison extends BinaryOperator implements Predicate { + private final CastingComparator comparator; + + public BinaryComparison(Expression left, Expression right, String symbol) { + super(left, right, symbol); + + // super asserted that left and right DataTypes were the same + + comparator = Util.createCastingComparator(left.dataType()); + } + + protected int compare(Object leftResult, Object rightResult) { + return comparator.compare(leftResult, rightResult); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java new file mode 100644 index 00000000000..2d02d6328f2 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java @@ -0,0 +1,38 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; + +import java.util.Arrays; +import java.util.List; + +/** + * An expression with two inputs and one output. The output is by default evaluated to null + * if any input is evaluated to null. + */ +public abstract class BinaryExpression implements Expression { + protected final Expression left; + protected final Expression right; + + public BinaryExpression(Expression left, Expression right) { + this.left = left; + this.right = right; + } + + @Override + public final Object eval(RowRecord record) { + Object leftResult = left.eval(record); + if (null == leftResult) return null; + + Object rightResult = right.eval(record); + if (null == rightResult) return null; + + return nullSafeEval(leftResult, rightResult); + } + + protected abstract Object nullSafeEval(Object leftResult, Object rightResult); + + @Override + public List children() { + return Arrays.asList(left, right); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java new file mode 100644 index 00000000000..9b2c67b7846 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java @@ -0,0 +1,26 @@ +package io.delta.standalone.expressions; + +/** + * A [[BinaryExpression]] that is an operator, with two properties: + * + * 1. The string representation is "x symbol y", rather than "funcName(x, y)". + * 2. Two inputs are expected to be of the same type. If the two inputs have different types, a + * RuntimeException will be thrown. + */ +public abstract class BinaryOperator extends BinaryExpression { + protected final String symbol; + + public BinaryOperator(Expression left, Expression right, String symbol) { + super(left, right); + this.symbol = symbol; + + if (!left.dataType().equals(right.dataType())) { + throw new RuntimeException("BinaryOperator left and right DataTypes must be the same"); + } + } + + @Override + public String toString() { + return "(" + left.toString() + " " + symbol + " " + right.toString() + ")"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/CastingComparator.java b/standalone/src/main/java/io/delta/standalone/expressions/CastingComparator.java new file mode 100644 index 00000000000..c30dda35a94 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/CastingComparator.java @@ -0,0 +1,15 @@ +package io.delta.standalone.expressions; + +import java.util.Comparator; + +public class CastingComparator> { + private final Comparator comparator; + + public CastingComparator() { + comparator = Comparator.naturalOrder(); + } + + int compare(Object a, Object b) { + return comparator.compare((T) a, (T) b); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Column.java b/standalone/src/main/java/io/delta/standalone/expressions/Column.java new file mode 100644 index 00000000000..ee7a83f415f --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Column.java @@ -0,0 +1,55 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.*; + +/** + * A column whose row-value will be computed based on the data in a [[RowRecord]]. + * + * Usage: new Column(columnName, columnDataType). + * + * It is recommended that you instantiate using a table schema (StructType). + * e.g. schema.column(columnName) + */ +public final class Column extends LeafExpression { + private final String name; + private final DataType dataType; + private final RowRecordEvaluator evaluator; + + public Column(String name, DataType dataType) { + this.name = name; + this.dataType = dataType; + + if (dataType instanceof IntegerType) { + evaluator = (record -> record.getInt(name)); + } else if (dataType instanceof BooleanType) { + evaluator = (record -> record.getBoolean(name)); + } else { + throw new RuntimeException("Couldn't find matching rowRecord DataType for column: " + name); + } + } + + public String name() { + return name; + } + + @Override + public Object eval(RowRecord record) { + return record.isNullAt(name) ? null : evaluator.nullSafeEval(record); + } + + @Override + public DataType dataType() { + return dataType; + } + + @Override + public String toString() { + return "Column(" + name + ")"; + } + + @FunctionalInterface + private interface RowRecordEvaluator { + Object nullSafeEval(RowRecord record); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java b/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java new file mode 100644 index 00000000000..2306df80100 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java @@ -0,0 +1,16 @@ +package io.delta.standalone.expressions; + +/** + * Usage: new EqualTo(expr1, expr2) - Returns true if `expr1` equals `expr2`, else false. + */ +public final class EqualTo extends BinaryComparison { + + public EqualTo(Expression left, Expression right) { + super(left, right, "="); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) == 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java new file mode 100644 index 00000000000..c0c58dcb6f6 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java @@ -0,0 +1,32 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.DataType; + +import java.util.List; + +/** + * An expression in Delta Standalone. + */ +public interface Expression { + + /** + * Returns the result of evaluating this expression on a given input RowRecord. + */ + Object eval(RowRecord record); + + /** + * Returns the [[DataType]] of the result of evaluating this expression. + */ + DataType dataType(); + + /** + * Returns the String representation of this expression. + */ + String toString(); + + /** + * Returns a List of the children of this node. Children should not change. + */ + List children(); +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/In.java b/standalone/src/main/java/io/delta/standalone/expressions/In.java new file mode 100644 index 00000000000..c7a99fa7770 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/In.java @@ -0,0 +1,69 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Usage: new In(expr, exprList) - Returns true if `expr` is equal to any in `exprList`, else false. + */ +public final class In implements Predicate { + private final Expression value; + private final List elems; + private final CastingComparator comparator; + + public In(Expression value, List elems) { + if (null == value) { + throw new IllegalArgumentException("'In' expression 'value' cannot be null"); + } + if (null == elems) { + throw new IllegalArgumentException("'In' expression 'elems' cannot be null"); + } + if (elems.isEmpty()) { + throw new IllegalArgumentException("'In' expression 'elems' cannot be empty"); + } + + boolean allSameDataType = elems.stream().allMatch(x -> x.dataType().equals(value.dataType())); + + if (!allSameDataType) { + throw new IllegalArgumentException("In expression 'elems' and 'value' must all be of the same DataType"); + } + + this.value = value; + this.elems = elems; + this.comparator = Util.createCastingComparator(value.dataType()); + } + + @Override + public Boolean eval(RowRecord record) { + Object result = value.eval(record); + if (null == result) { + throw new RuntimeException("'In' expression 'value.eval' result can't be null"); + } + + return elems.stream().anyMatch(setElem -> { + Object setElemValue = setElem.eval(record); + + if (null == setElemValue) { + throw new RuntimeException("'In' expression 'elems(i).eval' result can't be null"); + } + + return comparator.compare(result, setElemValue) == 0; + }); + } + + @Override + public String toString() { + String elemsStr = elems.stream().map(Expression::toString).collect(Collectors.joining(", ")); + return value + " IN (" + elemsStr + ")"; + } + + @Override + public List children() { + return Stream.concat(Stream.of(value), elems.stream()).collect(Collectors.toList()); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java b/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java new file mode 100644 index 00000000000..fbcfaafc9da --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java @@ -0,0 +1,24 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +/** + * Usage: new IsNotNull(expr) - Returns true if `expr` is not null, else false. + */ +public class IsNotNull extends UnaryExpression implements Predicate { + public IsNotNull(Expression child) { + super(child); + } + + @Override + public Object eval(RowRecord record) { + return child.eval(record) != null; + } + + @Override + public String toString() { + return "(" + child.toString() + ") IS NOT NULL"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java new file mode 100644 index 00000000000..0df332e0e04 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java @@ -0,0 +1,11 @@ +package io.delta.standalone.expressions; + +import java.util.Collections; +import java.util.List; + +public abstract class LeafExpression implements Expression { + @Override + public List children() { + return Collections.emptyList(); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java new file mode 100644 index 00000000000..2bca8434f31 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java @@ -0,0 +1,15 @@ +package io.delta.standalone.expressions; + +/** + * Usage: new LessThan(expr1, expr2) - Returns true if `expr1` is less than `expr2`, else false. + */ +public final class LessThan extends BinaryComparison { + public LessThan(Expression left, Expression right) { + super(left, right, "<"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) < 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java new file mode 100644 index 00000000000..eb23a6f3028 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java @@ -0,0 +1,68 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; +import io.delta.standalone.types.IntegerType; + +import java.util.Objects; + +public final class Literal extends LeafExpression { + public static final Literal True = Literal.of(Boolean.TRUE); + public static final Literal False = Literal.of(false); + + private final Object value; + private final DataType dataType; + + public Literal(Object value, DataType dataType) { + Literal.validateLiteralValue(value, dataType); + + this.value = value; + this.dataType = dataType; + } + + public Object value() { + return value; + } + + @Override + public Object eval(RowRecord record) { + return value; + } + + @Override + public DataType dataType() { + return dataType; + } + + @Override + public String toString() { + return value.toString(); + } + + private static void validateLiteralValue(Object value, DataType dataType) { + // TODO + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Literal literal = (Literal) o; + return Objects.equals(value, literal.value) && + Objects.equals(dataType, literal.dataType); + } + + @Override + public int hashCode() { + return Objects.hash(value, dataType); + } + + public static Literal of(int value) { + return new Literal(value, new IntegerType()); + } + + public static Literal of(boolean value) { + return new Literal(value, new BooleanType()); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Not.java b/standalone/src/main/java/io/delta/standalone/expressions/Not.java new file mode 100644 index 00000000000..56f843da592 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Not.java @@ -0,0 +1,24 @@ +package io.delta.standalone.expressions; + +/** + * Usage: new Not(expr) - Logical not. + */ +public class Not extends UnaryExpression implements Predicate { + public Not(Expression child) { + super(child); + } + + @Override + public Object nullSafeEval(Object childResult) { + if (!(childResult instanceof Boolean)) { + throw new RuntimeException("'Not' expression child.eval result must be a Boolean"); + } + + return !((boolean) childResult); + } + + @Override + public String toString() { + return "(NOT " + child.toString() + ")"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Or.java b/standalone/src/main/java/io/delta/standalone/expressions/Or.java new file mode 100644 index 00000000000..8beb2e33230 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Or.java @@ -0,0 +1,20 @@ +package io.delta.standalone.expressions; + +/** + * Usage: new Or(expr1, expr2) - Logical OR + */ +public final class Or extends BinaryOperator implements Predicate { + + public Or(Expression left, Expression right) { + super(left, right, "||"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + if (!(leftResult instanceof Boolean) || !(rightResult instanceof Boolean)) { + throw new RuntimeException("'Or' expression left.eval and right.eval results must be Booleans"); + } + + return (boolean) leftResult || (boolean) rightResult; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java b/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java new file mode 100644 index 00000000000..a839d7f7631 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java @@ -0,0 +1,11 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +public interface Predicate extends Expression { + @Override + default DataType dataType() { + return new BooleanType(); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java new file mode 100644 index 00000000000..febf9223ce8 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java @@ -0,0 +1,36 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; + +import java.util.Collections; +import java.util.List; + +/** + * An expression with one input and one output. The output is by default evaluated to null + * if the input is evaluated to null. + */ +public abstract class UnaryExpression implements Expression { + protected final Expression child; + + public UnaryExpression(Expression child) { + this.child = child; + } + + @Override + public Object eval(RowRecord record) { + Object childResult = child.eval(record); + + if (null == childResult) return null; + + return nullSafeEval(childResult); + } + + protected Object nullSafeEval(Object childResult) { + throw new RuntimeException("UnaryExpressions must override either eval or nullSafeEval"); + } + + @Override + public List children() { + return Collections.singletonList(child); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Util.java b/standalone/src/main/java/io/delta/standalone/expressions/Util.java new file mode 100644 index 00000000000..ff173d774fc --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Util.java @@ -0,0 +1,20 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; +import io.delta.standalone.types.IntegerType; + +public final class Util { + + public static CastingComparator createCastingComparator(DataType dataType) { + if (dataType instanceof IntegerType) { + return new CastingComparator(); + } + + if (dataType instanceof BooleanType) { + return new CastingComparator(); + } + + throw new RuntimeException("Couldn't find matching comparator for DataType: " + dataType.toString()); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index 544c2b77740..ed7ee17b294 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -38,6 +38,8 @@ package io.delta.standalone.types; +import io.delta.standalone.expressions.Column; + import java.util.Arrays; import java.util.HashMap; @@ -89,6 +91,11 @@ public StructField get(String fieldName) { return nameToField.get(fieldName); } + public Column column(String fieldName) { + StructField field = nameToField.get(fieldName); + return new Column(fieldName, field.getDataType()); + } + /** * @return a readable indented tree representation of this {@code StructType} * and all of its nested elements diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 1a29aef6e33..38be82a8a7b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -24,10 +24,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import io.delta.standalone.{DeltaLog, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} -import io.delta.standalone.internal.actions.Action +import io.delta.standalone.expressions.{And, Expression, Literal} +import io.delta.standalone.internal.actions.{Action, AddFile} +import io.delta.standalone.internal.data.PartitionRowRecord import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore import io.delta.standalone.internal.util.{ConversionUtils, FileNames} +import io.delta.standalone.types.StructType /** * Scala implementation of Java interface [[DeltaLog]]. @@ -108,4 +111,24 @@ private[standalone] object DeltaLogImpl { new DeltaLogImpl(hadoopConf, path, path.getParent) } + + /** + * Filters the given [[AddFile]]s by the given `partitionFilters`, returning those that match. + * @param files The active files in the DeltaLog state, which contains the partition value + * information + * @param partitionFilters Filters on the partition columns + */ + def filterFileList( + partitionSchema: StructType, + files: Seq[AddFile], + partitionFilters: Seq[Expression]): Seq[AddFile] = { + val expr = partitionFilters.reduceLeftOption(new And(_, _)).getOrElse(Literal.True) + // TODO: compressedExpr = ... + + files.filter { addFile => + val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) + val result = expr.eval(partitionRowRecord) + result == true + } + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala new file mode 100644 index 00000000000..67cdd46da5b --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala @@ -0,0 +1,77 @@ +package io.delta.standalone.internal.data + +import java.sql.{Date, Timestamp} + +import io.delta.standalone.data.{RowRecord => RowRecordJ} +import io.delta.standalone.types.{BooleanType, IntegerType, StructType} + +/** + * A RowRecord representing a Delta Lake partition of Map(partitionKey -> partitionValue) + */ +private[internal] class PartitionRowRecord( + partitionSchema: StructType, + partitionValues: Map[String, String]) extends RowRecordJ { + + private val partitionFieldToType = + partitionSchema.getFields.map { f => f.getName -> f.getDataType }.toMap + + require(partitionFieldToType.keySet == partitionValues.keySet, + s""" + |Column mismatch between partitionSchema and partitionValues. + |partitionSchema: ${partitionFieldToType.keySet.mkString(", ")} + |partitionValues: ${partitionValues.keySet.mkString(", ")} + |""".stripMargin) + + private def requireFieldExists(fieldName: String): Unit = { + // this is equivalent to checking both partitionValues and partitionFieldToType maps + // due to `require` statement above + require(partitionValues.contains(fieldName)) + } + + override def getSchema: StructType = partitionSchema + + override def getLength: Int = partitionSchema.getFieldNames.length + + override def isNullAt(fieldName: String): Boolean = { + requireFieldExists(fieldName) + null == partitionValues(fieldName) + } + + override def getInt(fieldName: String): Int = { + requireFieldExists(fieldName) + require(partitionFieldToType(fieldName).isInstanceOf[IntegerType]) + partitionValues(fieldName).toInt + } + + override def getLong(fieldName: String): Long = ??? + + override def getByte(fieldName: String): Byte = ??? + + override def getShort(fieldName: String): Short = ??? + + override def getBoolean(fieldName: String): Boolean = { + requireFieldExists(fieldName) + require(partitionFieldToType(fieldName).isInstanceOf[BooleanType]) + partitionValues(fieldName).toBoolean + } + + override def getFloat(fieldName: String): Float = ??? + + override def getDouble(fieldName: String): Double = ??? + + override def getString(fieldName: String): String = ??? + + override def getBinary(fieldName: String): Array[Byte] = ??? + + override def getBigDecimal(fieldName: String): java.math.BigDecimal = ??? + + override def getTimestamp(fieldName: String): Timestamp = ??? + + override def getDate(fieldName: String): Date = ??? + + override def getRecord(fieldName: String): RowRecordJ = ??? + + override def getList[T](fieldName: String): java.util.List[T] = ??? + + override def getMap[K, V](fieldName: String): java.util.Map[K, V] = ??? +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala new file mode 100644 index 00000000000..17ad2f73127 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala @@ -0,0 +1,117 @@ +package io.delta.standalone.internal + +import scala.collection.JavaConverters._ + +import io.delta.standalone.data.RowRecord +import io.delta.standalone.expressions._ +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.types.{IntegerType, StructField, StructType} + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +// scalastyle:off println +class ExpressionSuite extends FunSuite { + // scalastyle:on funsuite + + private def testPredicate( + predicate: Expression, + expectedResult: Boolean, + record: RowRecord = null) = { + println(predicate.toString()) + println(predicate.eval(record)) + assert(predicate.eval(record) == expectedResult) + } + + private def testPartitionFilter( + inputSchema: StructType, + inputFiles: Seq[AddFile], + filters: Seq[Expression], + expectedMatchedFiles: Seq[AddFile]) = { + println("filters:\n\t" + filters.map(_.toString()).mkString("\n\t")) + val matchedFiles = DeltaLogImpl.filterFileList(inputSchema, inputFiles, filters) + assert(matchedFiles.length == expectedMatchedFiles.length) + assert(matchedFiles.forall(expectedMatchedFiles.contains(_))) + } + + test("basic predicate") { + testPredicate(new And(Literal.False, Literal.False), expectedResult = false) + testPredicate(new And(Literal.True, Literal.False), expectedResult = false) + testPredicate(new And(Literal.False, Literal.True), expectedResult = false) + testPredicate(new And(Literal.True, Literal.True), expectedResult = true) + + testPredicate(new Or(Literal.False, Literal.False), expectedResult = false) + testPredicate(new Or(Literal.True, Literal.False), expectedResult = true) + testPredicate(new Or(Literal.False, Literal.True), expectedResult = true) + testPredicate(new Or(Literal.True, Literal.True), expectedResult = true) + + testPredicate(new Not(Literal.False), expectedResult = true) + testPredicate(new Not(Literal.True), expectedResult = false) + + testPredicate(new EqualTo(Literal.of(1), Literal.of(1)), expectedResult = true) + testPredicate(new EqualTo(Literal.of(1), Literal.of(2)), expectedResult = false) + + testPredicate(new LessThan(Literal.of(1), Literal.of(1)), expectedResult = false) + testPredicate(new LessThan(Literal.of(1), Literal.of(2)), expectedResult = true) + + val inSet = (0 to 10).map(Literal.of).asJava + testPredicate(new In(Literal.of(1), inSet), expectedResult = true) + testPredicate(new In(Literal.of(100), inSet), expectedResult = false) + } + + test("basic partition filter") { + val schema = new StructType(Array( + new StructField("col1", new IntegerType()), + new StructField("col2", new IntegerType()))) + + val add00 = AddFile("1", Map("col1" -> "0", "col2" -> "0"), 0, 0, dataChange = true) + val add01 = AddFile("2", Map("col1" -> "0", "col2" -> "1"), 0, 0, dataChange = true) + val add02 = AddFile("2", Map("col1" -> "0", "col2" -> "2"), 0, 0, dataChange = true) + val add10 = AddFile("3", Map("col1" -> "1", "col2" -> "0"), 0, 0, dataChange = true) + val add11 = AddFile("4", Map("col1" -> "1", "col2" -> "1"), 0, 0, dataChange = true) + val add12 = AddFile("4", Map("col1" -> "1", "col2" -> "2"), 0, 0, dataChange = true) + val add20 = AddFile("4", Map("col1" -> "2", "col2" -> "0"), 0, 0, dataChange = true) + val add21 = AddFile("4", Map("col1" -> "2", "col2" -> "1"), 0, 0, dataChange = true) + val add22 = AddFile("4", Map("col1" -> "2", "col2" -> "2"), 0, 0, dataChange = true) + val inputFiles = Seq(add00, add01, add02, add10, add11, add12, add20, add21, add22) + + val f1Expr1 = new EqualTo(schema.column("col1"), Literal.of(0)) + val f1Expr2 = new EqualTo(schema.column("col2"), Literal.of(1)) + val f1 = new And(f1Expr1, f1Expr2) + + testPartitionFilter(schema, inputFiles, f1 :: Nil, add01 :: Nil) + testPartitionFilter(schema, inputFiles, f1Expr1 :: f1Expr2 :: Nil, add01 :: Nil) + + val f2Expr1 = new LessThan(schema.column("col1"), Literal.of(1)) + val f2Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f2 = new And(f2Expr1, f2Expr2) + testPartitionFilter(schema, inputFiles, f2 :: Nil, add00 :: Nil) + testPartitionFilter(schema, inputFiles, f2Expr1 :: f2Expr2 :: Nil, add00 :: Nil) + + val f3Expr1 = new EqualTo(schema.column("col1"), Literal.of(2)) + val f3Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f3 = new Or(f3Expr1, f3Expr2) + testPartitionFilter(schema, inputFiles, f3 :: Nil, Seq(add20, add21, add22, add00, add10)) + + val inSet4 = (2 to 10).map(Literal.of).asJava + val f4 = new In(schema.column("col1"), inSet4) + testPartitionFilter(schema, inputFiles, f4 :: Nil, add20 :: add21 :: add22 :: Nil) + + val inSet5 = (100 to 110).map(Literal.of).asJava + val f5 = new In(schema.column("col1"), inSet5) + testPartitionFilter(schema, inputFiles, f5 :: Nil, Nil) + } + + test("not null partition filter") { + val schema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true))) + + val add0Null = AddFile("1", Map("col1" -> "0", "col2" -> null), 0, 0, dataChange = true) + val addNull1 = AddFile("1", Map("col1" -> null, "col2" -> "1"), 0, 0, dataChange = true) + val inputFiles = Seq(add0Null, addNull1) + + val f1 = new IsNotNull(schema.column("col1")) + testPartitionFilter(schema, inputFiles, f1 :: Nil, add0Null :: Nil) + } +} From 479a22bfcc4136a9c93a4f6ddd4192e7078605bd Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 17 Sep 2021 09:32:49 -0700 Subject: [PATCH 070/291] [NEW] [0] API Prototype (#140) * Expression framework prototype (#37) * WIP; have a basic skeleton for expressions; have a column resolver; todo comparator * WIP; figuring out comparators * finished first pass at a basic skeleton, using just Ints and Booleans Literal types * add leaf expression; make expression and predicate both ABCs; add children() and bound() methods * add nullSafeBoundEval method to BinaryExpression * add verifyInputDataTypes function to Expression * big refactor; add DataType to Column constructor; no more need for 'bound' checks; use nullSafeEval; refactor Comparator usage; short circuit null checks in eval * rename createColumn to column * Update ExpressionSuite.scala * add IsNotNull predicate; test; null check to Column::eval * make Expression interface; add back Predicate interface with default dataType field; make more member fields final * add Not expression; add nullSafeEval to UnaryExpression; add test for Not expr * added interfaces * add RowRecordBuilder; remove duplicate ClosableIterator * add newline to LogStore.java * update interface for OptimisticTransaction with javadoc * update DeltaLog; remove RowRecordBuilder; remove RowRecord build interface * update Operation; add writerId to CommitInfo.java * minor comment change * update javadoc for CommitResult and OptTxn * fix typo * add 2 new public methods to OptTxn interface * add asParquet method to StructType * Update Operation.java * rename writerId to engineInfo * respond to PR comments; fix Operation enum str; remove StructType asParquet; fix LogStore version --- .../io/delta/standalone/CommitResult.java | 19 +++ .../java/io/delta/standalone/DeltaLog.java | 10 ++ .../java/io/delta/standalone/Operation.java | 108 +++++++++++++++ .../standalone/OptimisticTransaction.java | 68 ++++++++++ .../delta/standalone/actions/CommitInfo.java | 17 ++- .../io/delta/standalone/storage/LogStore.java | 126 ++++++++++++++++++ .../util/ParquetToSparkSchemaConverter.java | 28 ++++ 7 files changed, 373 insertions(+), 3 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/CommitResult.java create mode 100644 standalone/src/main/java/io/delta/standalone/Operation.java create mode 100644 standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java create mode 100644 standalone/src/main/java/io/delta/standalone/storage/LogStore.java create mode 100644 standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java diff --git a/standalone/src/main/java/io/delta/standalone/CommitResult.java b/standalone/src/main/java/io/delta/standalone/CommitResult.java new file mode 100644 index 00000000000..89026996273 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/CommitResult.java @@ -0,0 +1,19 @@ +package io.delta.standalone; + +/** + * Wrapper around the result of {@link OptimisticTransaction.commit}. + */ +public class CommitResult { + private final long version; + + public CommitResult(long version) { + this.version = version; + } + + /** + * @return the table version that was committed. + */ + public long getVersion() { + return version; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index 6c15846dadb..427bd343118 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -68,6 +68,16 @@ public interface DeltaLog { */ Snapshot getSnapshotForTimestampAsOf(long timestamp); + /** + * Returns a new {@link OptimisticTransaction} that can be used to read the current state of the + * log and then commit updates. The reads and updates will be checked for logical conflicts + * with any concurrent writes to the log. + * + * Note that all reads in a transaction must go through the returned transaction object, and not + * directly to the {@link DeltaLog} otherwise they will not be checked for conflicts. + */ + OptimisticTransaction startTransaction(); + /** * @param version the commit version to retrieve {@link CommitInfo} * @return the {@link CommitInfo} of the commit at the provided version. diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java new file mode 100644 index 00000000000..f3aae66806f --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -0,0 +1,108 @@ +package io.delta.standalone; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * An operation that can be performed on a Delta table. + * + * An operation is tracked as the first line in delta logs, and powers `DESCRIBE HISTORY` for Delta + * tables. + */ +public final class Operation { + + /** + * Supported operation types. + */ + public enum Name { + /** Recorded during batch inserts. */ + WRITE("WRITE"), + + /** Recorded during streaming inserts. */ + STREAMING_UPDATE("STREAMING UPDATE"), + + /** Recorded while deleting certain partitions. */ + DELETE("DELETE"), + + /** Recorded when truncating the table. */ + TRUNCATE("TRUNCATE"), + + /** Recorded when converting a table into a Delta table. */ + CONVERT("CONVERT"), + + // TODO: the rest + + MANUAL_UPDATE("Manual Update"); + + /** Actual value that will be recorded in the transaction log */ + private final String logStr; + + Name(String logStr) { + this.logStr = logStr; + } + + @Override + public String toString() { + return logStr; + } + } + + private final Name name; + private final Map parameters; + private final Map operationMetrics; + private final Optional userMetadata; + + public Operation(Name name) { + this(name, Collections.emptyMap(), Collections.emptyMap(), Optional.empty()); + } + + public Operation(Name name, Map parameters) { + this(name, parameters, Collections.emptyMap(), Optional.empty()); + } + + public Operation(Name name, Map parameters, Map operationMetrics) { + this(name, parameters, operationMetrics, Optional.empty()); + } + + public Operation(Name name, Map parameters, Map operationMetrics, + Optional userMetadata) { + this.name = name; + this.parameters = parameters; + this.operationMetrics = operationMetrics; + this.userMetadata = userMetadata; + } + + /** + * TODO + * @return + */ + public Name getName() { + return name; + } + + /** + * TODO + * @return + */ + public Map getParameters() { + return null == parameters ? null : Collections.unmodifiableMap(parameters); + } + + /** + * TODO + * @return + */ + public Map getOperationMetrics() { + return null == operationMetrics ? null : Collections.unmodifiableMap(operationMetrics); + } + + /** + * TODO + * @return + */ + public Optional getUserMetadata() { + return null == userMetadata ? Optional.empty() : userMetadata; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java new file mode 100644 index 00000000000..f247ccb5114 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -0,0 +1,68 @@ +package io.delta.standalone; + +import io.delta.standalone.actions.Action; +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.actions.Metadata; +import io.delta.standalone.expressions.Expression; + +import java.util.List; + +public interface OptimisticTransaction { + + /** + * Modifies the state of the log by adding a new commit that is based on a read at + * the given `lastVersion`. In the case of a conflict with a concurrent writer this + * method will throw an exception. + * + * @param actions Set of actions to commit. + * @param op Details of operation that is performing this transactional commit. + * @param engineInfo String used to identify the writer engine. It should resemble + * "{engineName}-{engineVersion}". + * @return a {@link CommitResult}, wrapping the table version that was committed. + */ + CommitResult commit(Iterable actions, Operation op, String engineInfo); + + /** + * Mark files matched by the `readPredicates` as read by this transaction. + * + * Internally, the `readPredicates` parameter and the resultant `readFiles` will be used to + * determine if logical conflicts between this transaction and previously-committed transactions + * can be resolved (i.e. no error thrown). + * + * For example: + * - This transaction TXN1 reads partition 'date=2021-09-08' to perform an UPDATE and tries to + * commit at the next table version N. + * - After TXN1 starts, another transaction TXN2 reads partition 'date=2021-09-07' and commits + * first at table version N (with no other metadata changes). + * - TXN1 sees that another commit won, and needs to know whether to commit at version N+1 or + * fail. Using the `readPredicates` and resultant `readFiles`, TXN1 can see that none of its + * readFiles were changed by TXN2. Thus there are no logical conflicts and TXN1 can commit at + * table version N+1. + * + * @param readPredicates Predicates used to determine which files were read. + * @return Files matching the given predicates. + */ + List markFilesAsRead(Iterable readPredicates); + + /** + * Records an update to the metadata that should be committed with this transaction. + * Note that this must be done before writing out any files so that file writing + * and checks happen with the final metadata for the table. + * + * IMPORTANT: It is the responsibility of the caller to ensure that files currently + * present in the table are still valid under the new metadata. + * + * @param metadata The new metadata for the delta table. + */ + void updateMetadata(Metadata metadata); + + /** + * Mark the entire table as tainted (i.e. read) by this transaction. + */ + void readWholeTable(); + + /** + * @return the latest version that has committed for the idempotent transaction with given `id`. + */ + long txnVersion(String id); +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index 9a9ea2355e9..8ac56440638 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -43,6 +43,7 @@ public class CommitInfo implements Action { private final Optional isBlindAppend; private final Optional> operationMetrics; private final Optional userMetadata; + private final Optional writerId; public CommitInfo(Optional version, Timestamp timestamp, Optional userId, Optional userName, String operation, @@ -51,7 +52,7 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional Optional readVersion, Optional isolationLevel, Optional isBlindAppend, Optional> operationMetrics, - Optional userMetadata) { + Optional userMetadata, Optional writerId) { this.version = version; this.timestamp = timestamp; this.userId = userId; @@ -66,6 +67,7 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional this.isBlindAppend = isBlindAppend; this.operationMetrics = operationMetrics; this.userMetadata = userMetadata; + this.writerId = writerId; } /** @@ -169,6 +171,14 @@ public Optional getUserMetadata() { return userMetadata; } + /** + * @return the writerId of the operation that performed this commit. It should be of the form + * "{engineName}-{engineVersion}-deltaStandalone-{deltaStandaloneVersion}" + */ + public Optional getWriterId() { + return writerId; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -187,14 +197,15 @@ public boolean equals(Object o) { Objects.equals(isolationLevel, that.isolationLevel) && Objects.equals(isBlindAppend, that.isBlindAppend) && Objects.equals(operationMetrics, that.operationMetrics) && - Objects.equals(userMetadata, that.userMetadata); + Objects.equals(userMetadata, that.userMetadata) && + Objects.equals(writerId, that.writerId); } @Override public int hashCode() { return Objects.hash(version, timestamp, userId, userName, operation, operationParameters, jobInfo, notebookInfo, clusterId, readVersion, isolationLevel, isBlindAppend, - operationMetrics, userMetadata); + operationMetrics, userMetadata, writerId); } /** diff --git a/standalone/src/main/java/io/delta/standalone/storage/LogStore.java b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java new file mode 100644 index 00000000000..5c4922a3b71 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java @@ -0,0 +1,126 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.storage; + +import io.delta.standalone.data.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.FileNotFoundException; +import java.nio.file.FileAlreadyExistsException; +import java.util.Iterator; + +/** + * :: DeveloperApi :: + * + * General interface for all critical file system operations required to read and write the + * Delta logs. The correctness is predicated on the atomicity and durability guarantees of + * the implementation of this interface. Specifically, + * + * 1. Atomic visibility of files: If isPartialWriteVisible is false, any file written through + * this store must be made visible atomically. In other words, this should not generate partial + * files. + * + * 2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final + * destination. + * + * 3. Consistent listing: Once a file has been written in a directory, all future listings for + * that directory must return that file. + * + * All subclasses of this interface is required to have a constructor that takes Configuration + * as a single parameter. This constructor is used to dynamically create the LogStore. + * + * LogStore and its implementations are not meant for direct access but for configuration based + * on storage system. See [[https://docs.delta.io/latest/delta-storage.html]] for details. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ +public abstract class LogStore { + + private Configuration initHadoopConf; + + public LogStore(Configuration initHadoopConf) { + this.initHadoopConf = initHadoopConf; + } + + /** + * :: DeveloperApi :: + * + * Hadoop configuration that should only be used during initialization of LogStore. Each method + * should use their `hadoopConf` parameter rather than this (potentially outdated) hadoop + * configuration. + */ + public Configuration initHadoopConf() { return initHadoopConf; } + + /** + * :: DeveloperApi :: + * + * Load the given file and return an `Iterator` of lines, with line breaks removed from each line. + * Callers of this function are responsible to close the iterator if they are done with it. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ + public abstract CloseableIterator read(Path path, Configuration hadoopConf); + + /** + * :: DeveloperApi :: + * + * Write the given `actions` to the given `path` with or without overwrite as indicated. + * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file + * already exists and overwrite = false. Furthermore, if isPartialWriteVisible returns false, + * implementation must ensure that the entire file is made visible atomically, that is, + * it should not generate partial files. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ + public abstract void write( + Path path, + Iterator actions, + Boolean overwrite, + Configuration hadoopConf) throws FileAlreadyExistsException; + + /** + * :: DeveloperApi :: + * + * List the paths in the same directory that are lexicographically greater or equal to + * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ + public abstract Iterator listFrom( + Path path, + Configuration hadoopConf) throws FileNotFoundException; + + /** + * :: DeveloperApi :: + * + * Resolve the fully qualified path for the given `path`. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ + public abstract Path resolvePathOnPhysicalStorage(Path path, Configuration hadoopConf); + + /** + * :: DeveloperApi :: + * + * Whether a partial write is visible for the underlying file system of `path`. + * + * @since 0.3.0 // TODO: double check this will be the new DSW version + */ + public abstract Boolean isPartialWriteVisible(Path path, Configuration hadoopConf); +} diff --git a/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java b/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java new file mode 100644 index 00000000000..2dbfe86ee01 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java @@ -0,0 +1,28 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.util; + +import io.delta.standalone.types.StructType; +import org.apache.parquet.schema.MessageType; + +public final class ParquetToSparkSchemaConverter { + + public static MessageType convert(StructType catalystSchema) { + // TODO + return null; + } +} From 1de2365581966f9132aef44687f29ad7dcbd7b05 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 17 Sep 2021 10:10:11 -0700 Subject: [PATCH 071/291] [NEW] [1] Commit Prototype & Skeleton (#141) * Rebased off of 0_API branch; started skeleton; WIP on scala impl of OptimisticTransaction * finish most of txn.prepareCommit * implement verifyNewMetadata; add comments for AnalysisException throughout code; add some Schema utils * added some comments for future tests * finished (mostly) commit and doCommit functions; added test comments as future work; TODO log store refactor * added HDFS Log Store write impl * add conversion utils from java actions to scala actions * starting to write a simple test; fixed java api Collection.unmodifiable.. null error (but not all); WIP fixing CommitInfo operationParams JSON escaped value bug * got a very basic test working * minor comment changes * clean up code after rebase with dsw_prototype_0_API * remove DeltaOperations scala class * removed unnecessary StandaloneHadoopConf vals * rename writerId to engineInfo; fix failing scalastyle * remove schemaStr from Metadata constructor and use schema.toJson instead * empty commit; PR isn't updating with previous commit --- .../java/io/delta/standalone/Operation.java | 1 + .../io/delta/standalone/actions/AddFile.java | 4 +- .../delta/standalone/actions/CommitInfo.java | 25 +- .../io/delta/standalone/actions/Format.java | 5 + .../io/delta/standalone/actions/Metadata.java | 3 +- .../standalone/internal/DeltaLogImpl.scala | 48 +++- .../internal/OptimisticTransactionImpl.scala | 239 ++++++++++++++++++ .../standalone/internal/SnapshotImpl.scala | 8 +- .../standalone/internal/actions/actions.scala | 41 ++- .../internal/data/PartitionRowRecord.scala | 42 ++- .../internal/exception/DeltaErrors.scala | 94 ++++++- .../storage/HDFSReadOnlyLogStore.scala | 99 +++++++- .../internal/storage/ReadOnlyLogStore.scala | 45 +++- .../internal/util/ConversionUtils.scala | 113 ++++++++- .../internal/util/SchemaMergingUtils.scala | 87 +++++++ .../internal/util/SchemaUtils.scala | 57 +++++ .../internal/ConversionUtilsSuite.scala | 27 ++ .../standalone/internal/DeltaLogSuite.scala | 11 +- .../internal/OptimisticTransactionSuite.scala | 106 ++++++++ .../standalone/internal/util/TestUtils.scala | 38 +++ 20 files changed, 1033 insertions(+), 60 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index f3aae66806f..80947b4a1ba 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -87,6 +87,7 @@ public Name getName() { * @return */ public Map getParameters() { + // TODO: be consistent with AddFile getter ternary return null == parameters ? null : Collections.unmodifiableMap(parameters); } diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java index 9ddde9ca6d3..03322135f6c 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java @@ -66,7 +66,7 @@ public String getPath() { * @see Delta Protocol Partition Value Serialization */ public Map getPartitionValues() { - return Collections.unmodifiableMap(partitionValues); + return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; } /** @@ -106,7 +106,7 @@ public String getStats() { * @return an unmodifiable {@code Map} containing metadata about this file */ public Map getTags() { - return Collections.unmodifiableMap(tags); + return tags != null ? Collections.unmodifiableMap(tags) : null; } @Override diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index 8ac56440638..4bc122b72c5 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.sql.Timestamp; @@ -43,7 +44,7 @@ public class CommitInfo implements Action { private final Optional isBlindAppend; private final Optional> operationMetrics; private final Optional userMetadata; - private final Optional writerId; + private final Optional engineInfo; public CommitInfo(Optional version, Timestamp timestamp, Optional userId, Optional userName, String operation, @@ -52,7 +53,7 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional Optional readVersion, Optional isolationLevel, Optional isBlindAppend, Optional> operationMetrics, - Optional userMetadata, Optional writerId) { + Optional userMetadata, Optional engineInfo) { this.version = version; this.timestamp = timestamp; this.userId = userId; @@ -67,7 +68,7 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional this.isBlindAppend = isBlindAppend; this.operationMetrics = operationMetrics; this.userMetadata = userMetadata; - this.writerId = writerId; + this.engineInfo = engineInfo; } /** @@ -172,11 +173,11 @@ public Optional getUserMetadata() { } /** - * @return the writerId of the operation that performed this commit. It should be of the form + * @return the engineInfo of the operation that performed this commit. It should be of the form * "{engineName}-{engineVersion}-deltaStandalone-{deltaStandaloneVersion}" */ - public Optional getWriterId() { - return writerId; + public Optional getEngineInfo() { + return engineInfo; } @Override @@ -198,14 +199,14 @@ public boolean equals(Object o) { Objects.equals(isBlindAppend, that.isBlindAppend) && Objects.equals(operationMetrics, that.operationMetrics) && Objects.equals(userMetadata, that.userMetadata) && - Objects.equals(writerId, that.writerId); + Objects.equals(engineInfo, that.engineInfo); } @Override public int hashCode() { return Objects.hash(version, timestamp, userId, userName, operation, operationParameters, jobInfo, notebookInfo, clusterId, readVersion, isolationLevel, isBlindAppend, - operationMetrics, userMetadata, writerId); + operationMetrics, userMetadata, engineInfo); } /** @@ -233,6 +234,7 @@ public static class Builder { private Optional isBlindAppend = Optional.empty(); private Optional> operationMetrics = Optional.empty(); private Optional userMetadata = Optional.empty(); + private Optional engineInfo = Optional.empty(); public Builder version(Long version) { this.version = Optional.of(version); @@ -304,6 +306,11 @@ public Builder userMetadata(String userMetadata) { return this; } + public Builder engineInfo(String engineInfo) { + this.engineInfo = Optional.of(engineInfo); + return this; + } + /** * @return a new {@code CommitInfo} with the same properties as {@code this} */ @@ -311,7 +318,7 @@ public CommitInfo build() { CommitInfo commitInfo = new CommitInfo(this.version, this.timestamp, this.userId, this.userName, this.operation, this.operationParameters, this.jobInfo, this.notebookInfo, this.clusterId, this.readVersion, this.isolationLevel, - this.isBlindAppend, this.operationMetrics, this.userMetadata); + this.isBlindAppend, this.operationMetrics, this.userMetadata, this.engineInfo); return commitInfo; } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Format.java b/standalone/src/main/java/io/delta/standalone/actions/Format.java index 98244ab59b7..6cb56478930 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Format.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Format.java @@ -33,6 +33,11 @@ public Format(String provider, Map options) { this.options = options; } + public Format() { + this.provider = "parquet"; + this.options = Collections.emptyMap(); + } + /** * @return the name of the encoding for files in this table */ diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index 087db579e26..1bd31ff84e2 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -128,8 +128,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(id, name, description, format, partitionColumns, - configuration, createdTime, schema); + return Objects.hash(id, name, description, format, partitionColumns, configuration, createdTime, schema); } /** diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 38be82a8a7b..e60992b78b7 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -16,16 +16,17 @@ package io.delta.standalone.internal +import java.io.IOException import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import io.delta.standalone.{DeltaLog, VersionLog} +import io.delta.standalone.{DeltaLog, OptimisticTransaction, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} import io.delta.standalone.expressions.{And, Expression, Literal} -import io.delta.standalone.internal.actions.{Action, AddFile} +import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol} import io.delta.standalone.internal.data.PartitionRowRecord import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore @@ -46,6 +47,13 @@ private[internal] class DeltaLogImpl private( /** Used to read (not write) physical log files and checkpoints. */ lazy val store = new HDFSReadOnlyLogStore(hadoopConf) + /** Direct access to the underlying storage system. */ + private lazy val fs = logPath.getFileSystem(hadoopConf) + + // TODO: There is a race here where files could get dropped when increasing the + // retention interval... + private def metadata = if (snapshot == null) Metadata() else snapshot.metadataScala + /** Use ReentrantLock to allow us to call `lockInterruptibly`. */ private val deltaLogLock = new ReentrantLock() @@ -82,11 +90,16 @@ private[internal] class DeltaLogImpl private( }.asJava } + override def startTransaction(): OptimisticTransaction = { + update() + new OptimisticTransactionImpl(this, snapshot) + } + /** * Run `body` inside `deltaLogLock` lock using `lockInterruptibly` so that the thread can be * interrupted when waiting for the lock. */ - protected def lockInterruptibly[T](body: => T): T = { + def lockInterruptibly[T](body: => T): T = { deltaLogLock.lockInterruptibly() try { body @@ -94,6 +107,35 @@ private[internal] class DeltaLogImpl private( deltaLogLock.unlock() } } + + /** Creates the log directory if it does not exist. */ + def ensureLogDirectoryExist(): Unit = { + if (!fs.exists(logPath)) { + if (!fs.mkdirs(logPath)) { + throw new IOException(s"Cannot create $logPath") + } + } + } + + /** + * Asserts that the client is up to date with the protocol and + * allowed to write to the table that is using the given `protocol`. + */ + def assertProtocolWrite(protocol: Protocol): Unit = { + if (protocol != null && Action.writerVersion < protocol.minWriterVersion) { + throw new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, protocol) + } + } + + /** + * Checks whether this table only accepts appends. If so it will throw an error in operations that + * can remove data such as DELETE/UPDATE/MERGE. + */ + def assertRemovable(): Unit = { + if (metadata.configuration.getOrElse("appendOnly", "false").toBoolean) { + throw DeltaErrors.modifyAppendOnlyTableException + } + } } private[standalone] object DeltaLogImpl { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala new file mode 100644 index 00000000000..d620a86f179 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -0,0 +1,239 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.ConcurrentModificationException + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{CommitResult, Operation, OptimisticTransaction} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} +import io.delta.standalone.expressions.Expression +import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, Metadata, Protocol, RemoveFile} +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.sources.StandaloneHadoopConf +import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMergingUtils, SchemaUtils} + +private[internal] class OptimisticTransactionImpl( + deltaLog: DeltaLogImpl, + snapshot: SnapshotImpl) extends OptimisticTransaction { + + /** Tracks if this transaction has already committed. */ + private var committed = false + + /** Stores the updated metadata (if any) that will result from this txn. */ + private var newMetadata: Option[Metadata] = None + + /** Stores the updated protocol (if any) that will result from this txn. */ + private var newProtocol: Option[Protocol] = None + + // Whether this transaction is creating a new table. + private var isCreatingNewTable: Boolean = false + + /** The protocol of the snapshot that this transaction is reading at. */ + def protocol: Protocol = newProtocol.getOrElse(snapshot.protocolScala) + + /** + * Returns the metadata for this transaction. The metadata refers to the metadata of the snapshot + * at the transaction's read version unless updated during the transaction. + */ + def metadata: Metadata = newMetadata.getOrElse(snapshot.metadataScala) + + /////////////////////////////////////////////////////////////////////////// + // Public Java API Methods + /////////////////////////////////////////////////////////////////////////// + + override def commit( + actionsJ: java.lang.Iterable[ActionJ], + op: Operation, + engineInfo: String): CommitResult = { + val actions = actionsJ.asScala.map(ConversionUtils.convertActionJ).toSeq + + // Try to commit at the next version. + var finalActions = prepareCommit(actions) + + // TODO blind append check & create commitInfo using engineInfo + + val commitVersion = doCommit(snapshot.version + 1, finalActions) + + postCommit(commitVersion) + + new CommitResult(commitVersion) + } + + override def markFilesAsRead( + readPredicates: java.lang.Iterable[Expression]): java.util.List[AddFileJ] = { + // TODO + null + } + + override def updateMetadata(metadata: MetadataJ): Unit = { + + } + + override def readWholeTable(): Unit = { + + } + + override def txnVersion(id: String): Long = { + // TODO + 0L + } + + /////////////////////////////////////////////////////////////////////////// + // Critical Internal-Only Methods + /////////////////////////////////////////////////////////////////////////// + + /** + * Prepare for a commit by doing all necessary pre-commit checks and modifications to the actions. + * @return The finalized set of actions. + */ + private def prepareCommit(actions: Seq[Action]): Seq[Action] = { + assert(!committed, "Transaction already committed.") + + val userCommitInfo = actions.exists(_.isInstanceOf[CommitInfo]) + assert(!userCommitInfo, + "CommitInfo actions are created automatically. Users shouldn't try to commit them directly") + + // If the metadata has changed, add that to the set of actions + var finalActions = newMetadata.toSeq ++ actions + val metadataChanges = finalActions.collect { case m: Metadata => m } + assert(metadataChanges.length <= 1, + "Cannot change the metadata more than once in a transaction.") + + metadataChanges.foreach(m => verifyNewMetadata(m)) + finalActions = newProtocol.toSeq ++ finalActions + + if (snapshot.version == -1) { + deltaLog.ensureLogDirectoryExist() + + // If this is the first commit and no protocol is specified, initialize the protocol version. + if (!finalActions.exists(_.isInstanceOf[Protocol])) { + finalActions = protocol +: finalActions + } + + // If this is the first commit and no metadata is specified, throw an exception + if (!finalActions.exists(_.isInstanceOf[Metadata])) { + throw DeltaErrors.metadataAbsentException() + } + } + + val partitionColumns = metadata.partitionColumns.toSet + finalActions.foreach { + case newVersion: Protocol => + require(newVersion.minReaderVersion > 0, "The reader version needs to be greater than 0") + require(newVersion.minWriterVersion > 0, "The writer version needs to be greater than 0") + if (!isCreatingNewTable) { + val currentVersion = snapshot.protocolScala + if (newVersion.minReaderVersion < currentVersion.minReaderVersion || + newVersion.minWriterVersion < currentVersion.minWriterVersion) { + throw DeltaErrors.protocolDowngradeException(currentVersion, newVersion) + } + } + case a: AddFile if partitionColumns != a.partitionValues.keySet => + throw DeltaErrors.addFilePartitioningMismatchException( + a.partitionValues.keySet.toSeq, partitionColumns.toSeq) + case _ => // nothing + } + + deltaLog.assertProtocolWrite(snapshot.protocolScala) + + // We make sure that this isn't an appendOnly table as we check if we need to delete files. + val removes = actions.collect { case r: RemoveFile => r } + if (removes.exists(_.dataChange)) deltaLog.assertRemovable() + + finalActions + } + + /** + * Commit `actions` using `attemptVersion` version number. + * + * If you detect any conflicts, try to resolve logical conflicts and commit using a new version. + * + * @return the real version that was committed. + * @throws IllegalStateException if the attempted commit version is ahead of the current delta log + * version + * @throws ConcurrentModificationException if any conflicts are detected + */ + private def doCommit(attemptVersion: Long, actions: Seq[Action]): Long = lockCommitIfEnabled { + try { + deltaLog.store.write( + FileNames.deltaFile(deltaLog.logPath, attemptVersion), + actions.map(_.json).toIterator + ) + + val postCommitSnapshot = deltaLog.update() + if (postCommitSnapshot.version < attemptVersion) { + throw new IllegalStateException( + s"The committed version is $attemptVersion " + + s"but the current version is ${postCommitSnapshot.version}.") + } + + attemptVersion + } catch { + case _: java.nio.file.FileAlreadyExistsException => + throw new DeltaErrors.DeltaConcurrentModificationException("TODO msg") + } + } + + /** + * Perform post-commit operations + */ + private def postCommit(commitVersion: Long): Unit = { + committed = true + + // TODO: checkpoint + } + + /////////////////////////////////////////////////////////////////////////// + // Helper Methods + /////////////////////////////////////////////////////////////////////////// + + private def verifyNewMetadata(metadata: Metadata): Unit = { + SchemaMergingUtils.checkColumnNameDuplication(metadata.schema, "in the metadata update") + SchemaUtils.checkFieldNames(SchemaMergingUtils.explodeNestedFieldNames(metadata.dataSchema)) + + try { + SchemaUtils.checkFieldNames(metadata.partitionColumns) + } catch { + // TODO: case e: AnalysisException ? + case e: RuntimeException => throw DeltaErrors.invalidPartitionColumn(e) + } + + // TODO: this function is still incomplete + val needsProtocolUpdate = Protocol.checkProtocolRequirements(metadata, protocol) + + if (needsProtocolUpdate.isDefined) { + newProtocol = needsProtocolUpdate + } + } + + private def isCommitLockEnabled: Boolean = { +// TODO: +// spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED).getOrElse( +// deltaLog.store.isPartialWriteVisible(deltaLog.logPath)) + true + } + + private def lockCommitIfEnabled[T](body: => T): T = { + if (isCommitLockEnabled) { + deltaLog.lockInterruptibly(body) + } else { + body + } + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 14afd9552c1..df6e4ff3e9e 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -151,11 +151,11 @@ private[internal] class SnapshotImpl( */ private def assertProtocolRead(): Unit = { if (null != protocolScala) { - val clientVersion = Action.readerVersion - val tblVersion = protocolScala.minReaderVersion + val clientReadVersion = Action.readerVersion + val tblReadVersion = protocolScala.minReaderVersion - if (clientVersion < tblVersion) { - throw DeltaErrors.InvalidProtocolVersionException(clientVersion, tblVersion) + if (clientReadVersion < tblReadVersion) { + throw new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, protocolScala) } } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala index b765906f1ed..4dd43cc2f7e 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala @@ -67,11 +67,31 @@ private[internal] case class Protocol( def simpleString: String = s"($minReaderVersion,$minWriterVersion)" } +private[internal] object Protocol { + val MIN_READER_VERSION_PROP = "delta.minReaderVersion" + val MIN_WRITER_VERSION_PROP = "delta.minWriterVersion" + + /** + * Verify that the protocol version of the table satisfies the version requirements of all the + * configurations to be set for the table. Returns the minimum required protocol if not. + */ + def checkProtocolRequirements(metadata: Metadata, protocol: Protocol): Option[Protocol] = { + assert(!metadata.configuration.contains(MIN_READER_VERSION_PROP), s"Should not have the " + + s"protocol version ($MIN_READER_VERSION_PROP) as part of table properties") + assert(!metadata.configuration.contains(MIN_WRITER_VERSION_PROP), s"Should not have the " + + s"protocol version ($MIN_WRITER_VERSION_PROP) as part of table properties") + + // TODO: requiredMinimumProtocol(...) + + Some(protocol) + } +} + /** * Sets the committed version for a given application. Used to make operations * like streaming append idempotent. */ -case class SetTransaction( +private[internal] case class SetTransaction( appId: String, version: Long, @JsonDeserialize(contentAs = classOf[java.lang.Long]) @@ -120,6 +140,11 @@ private[internal] case class AddFile( /** * Logical removal of a given file from the reservoir. Acts as a tombstone before a file is * deleted permanently. + * + * Note that for protocol compatibility reasons, the fields `partitionValues`, `size`, and `tags` + * are only present when the extendedFileMetadata flag is true. New writers should generally be + * setting this flag, but old writers (and FSCK) won't, so readers must check this flag before + * attempting to consume those values. */ private[internal] case class RemoveFile( path: String, @@ -141,7 +166,7 @@ private[internal] case class RemoveFile( * ignore this, CDC readers should scan all ChangeFiles in a version rather than computing * changes from AddFile and RemoveFile actions. */ -case class AddCDCFile( +private[internal] case class AddCDCFile( path: String, partitionValues: Map[String, String], size: Long, @@ -178,6 +203,13 @@ private[internal] case class Metadata( DataTypeParser.fromJson(s).asInstanceOf[StructType] }.getOrElse(new StructType(Array.empty)) + /** Columns written out to files. */ + @JsonIgnore + lazy val dataSchema: StructType = { + val partitions = partitionColumns.toSet + new StructType(schema.getFields.filterNot(f => partitions.contains(f.getName))) + } + override def wrap: SingleAction = SingleAction(metaData = this) } @@ -220,7 +252,8 @@ private[internal] case class CommitInfo( /** Whether this commit has blindly appended without caring about existing files */ isBlindAppend: Option[Boolean], operationMetrics: Option[Map[String, String]], - userMetadata: Option[String]) extends Action with CommitMarker { + userMetadata: Option[String], + engineInfo: Option[String]) extends Action with CommitMarker { override def wrap: SingleAction = SingleAction(commitInfo = this) override def withTimestamp(timestamp: Long): CommitInfo = { @@ -235,7 +268,7 @@ private[internal] case class CommitInfo( private[internal] object CommitInfo { def empty(version: Option[Long] = None): CommitInfo = { CommitInfo(version, null, None, None, null, null, None, None, - None, None, None, None, None, None) + None, None, None, None, None, None, None) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala index 67cdd46da5b..e24fe44ef0f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.standalone.internal.data import java.sql.{Date, Timestamp} @@ -43,11 +59,11 @@ private[internal] class PartitionRowRecord( partitionValues(fieldName).toInt } - override def getLong(fieldName: String): Long = ??? + override def getLong(fieldName: String): Long = 0 // TODO - override def getByte(fieldName: String): Byte = ??? + override def getByte(fieldName: String): Byte = 0 // TODO - override def getShort(fieldName: String): Short = ??? + override def getShort(fieldName: String): Short = 0 // TODO override def getBoolean(fieldName: String): Boolean = { requireFieldExists(fieldName) @@ -55,23 +71,23 @@ private[internal] class PartitionRowRecord( partitionValues(fieldName).toBoolean } - override def getFloat(fieldName: String): Float = ??? + override def getFloat(fieldName: String): Float = 0 // TODO - override def getDouble(fieldName: String): Double = ??? + override def getDouble(fieldName: String): Double = 0 // TODO - override def getString(fieldName: String): String = ??? + override def getString(fieldName: String): String = null // TODO - override def getBinary(fieldName: String): Array[Byte] = ??? + override def getBinary(fieldName: String): Array[Byte] = null // TODO - override def getBigDecimal(fieldName: String): java.math.BigDecimal = ??? + override def getBigDecimal(fieldName: String): java.math.BigDecimal = null // TODO - override def getTimestamp(fieldName: String): Timestamp = ??? + override def getTimestamp(fieldName: String): Timestamp = null // TODO - override def getDate(fieldName: String): Date = ??? + override def getDate(fieldName: String): Date = null // TODO - override def getRecord(fieldName: String): RowRecordJ = ??? + override def getRecord(fieldName: String): RowRecordJ = null // TODO - override def getList[T](fieldName: String): java.util.List[T] = ??? + override def getList[T](fieldName: String): java.util.List[T] = null // TODO - override def getMap[K, V](fieldName: String): java.util.Map[K, V] = ??? + override def getMap[K, V](fieldName: String): java.util.Map[K, V] = null // TODO } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index cdc5090c408..517aa33ba82 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -16,21 +16,30 @@ package io.delta.standalone.internal.exception -import java.io.FileNotFoundException +import java.io.{FileNotFoundException, IOException} +import java.util.ConcurrentModificationException +import io.delta.standalone.internal.actions.Protocol +import io.delta.standalone.internal.sources.StandaloneHadoopConf import org.apache.hadoop.fs.Path - import io.delta.standalone.types.StructType /** A holder object for Delta errors. */ private[internal] object DeltaErrors { - /** Thrown when the protocol version of a table is greater than supported by this client. */ - case class InvalidProtocolVersionException( - clientProtocolVersion: Int, - tableProtocolVersion: Int) extends RuntimeException( - s"Delta protocol version $tableProtocolVersion is too new for this version of Delta " + - s"Standalone Reader $clientProtocolVersion. Please upgrade to a newer release.") + /** + * Thrown when the protocol version of a table is greater than the one supported by this client + */ + class InvalidProtocolVersionException( + clientProtocol: Protocol, + tableProtocol: Protocol) extends RuntimeException( + s""" + |Delta protocol version ${tableProtocol.simpleString} is too new for this version of Delta + |Standalone Reader/Writer ${clientProtocol.simpleString}. Please upgrade to a newer release. + |""".stripMargin) + + class DeltaConcurrentModificationException(message: String) + extends ConcurrentModificationException(message) def deltaVersionsNotContiguousException(deltaVersions: Seq[Long]): Throwable = { new IllegalStateException(s"Versions ($deltaVersions) are not contiguous.") @@ -61,6 +70,7 @@ private[internal] object DeltaErrors { } def noReproducibleHistoryFound(logPath: Path): Throwable = { + // TODO: AnalysisException ? new RuntimeException(s"No reproducible commits found at $logPath") } @@ -83,6 +93,7 @@ private[internal] object DeltaErrors { } def noHistoryFound(logPath: Path): Throwable = { + // TODO: AnalysisException ? new RuntimeException(s"No commits found at $logPath") } @@ -112,4 +123,71 @@ private[internal] object DeltaErrors { """.stripMargin ) } + + def metadataAbsentException(): Throwable = { + new IllegalStateException( + "Couldn't find Metadata while committing the first version of the Delta table.") + } + + def protocolDowngradeException(oldProtocol: Protocol, newProtocol: Protocol): Throwable = { + // TODO: class ProtocolDowngradeException ? + new RuntimeException("Protocol version cannot be downgraded from " + + s"${oldProtocol.simpleString} to ${newProtocol.simpleString}") + } + + def addFilePartitioningMismatchException( + addFilePartitions: Seq[String], + metadataPartitions: Seq[String]): Throwable = { + new IllegalStateException( + s""" + |The AddFile contains partitioning schema different from the table's partitioning schema + |expected: ${DeltaErrors.formatColumnList(metadataPartitions)} + |actual: ${DeltaErrors.formatColumnList(addFilePartitions)} + """.stripMargin) + } + + def modifyAppendOnlyTableException: Throwable = { + new UnsupportedOperationException( + "This table is configured to only allow appends. If you would like to permit " + + s"updates or deletes, use 'ALTER TABLE SET TBLPROPERTIES " + + s"(appendOnly=false)'.") + } + + def invalidColumnName(name: String): Throwable = { + // TODO: AnalysisException ?? + new RuntimeException( + s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\\n\\t=". + |Please use alias to rename it. + """.stripMargin.split("\n").mkString(" ").trim) + } + + // TODO: AnalysisException ?? + def invalidPartitionColumn(e: RuntimeException): Throwable = { + // TODO: AnalysisException ?? + new RuntimeException( + """Found partition columns having invalid character(s) among " ,;{}()\n\t=". Please """ + + "change the name to your partition columns. This check can be turned off by setting " + + """spark.conf.set("spark.databricks.delta.partitionColumnValidity.enabled", false) """ + + "however this is not recommended as other features of Delta may not work properly.", + e) + } + + def incorrectLogStoreImplementationException(cause: Throwable): Throwable = { + new IOException(s""" + |The error typically occurs when the default LogStore implementation, that + |is, HDFSLogStore, is used to write into a Delta table on a non-HDFS storage system. + |In order to get the transactional ACID guarantees on table updates, you have to use the + |correct implementation of LogStore that is appropriate for your storage system. + |See https://docs.delta.io/latest/delta-storage.html for details. + """.stripMargin, cause) + } + + /////////////////////////////////////////////////////////////////////////// + // Helper Methods + /////////////////////////////////////////////////////////////////////////// + + private def formatColumn(colName: String): String = s"`$colName`" + + private def formatColumnList(colNames: Seq[String]): String = + colNames.map(formatColumn).mkString("[", ", ", "]") } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala index 724824e48a7..46835f76ca5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala @@ -16,20 +16,33 @@ package io.delta.standalone.internal.storage -import java.io.{BufferedReader, FileNotFoundException, InputStreamReader} +import java.io.{BufferedReader, FileNotFoundException, InputStreamReader, IOException} import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.FileAlreadyExistsException +import java.util.{EnumSet, UUID} import scala.collection.JavaConverters._ +import scala.util.control.NonFatal +import io.delta.standalone.internal.exception.DeltaErrors import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileContext, FileStatus, Options, Path, RawLocalFileSystem} +import org.apache.hadoop.fs.CreateFlag.CREATE +import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} /** - * Default implementation of [[ReadOnlyLogStore]] for Hadoop [[FileSystem]] implementations. + * The [[LogStore]] implementation for HDFS, which uses Hadoop [[FileContext]] API's to + * provide the necessary atomic and durability guarantees: + * + * 1. Atomic visibility of files: `FileContext.rename` is used write files which is atomic for HDFS. + * + * 2. Consistent file listing: HDFS file listing is consistent. */ private[internal] class HDFSReadOnlyLogStore(hadoopConf: Configuration) extends ReadOnlyLogStore { + val noAbstractFileSystemExceptionMessage = "No AbstractFileSystem" + override def read(path: Path): Seq[String] = { val fs = path.getFileSystem(hadoopConf) val stream = fs.open(path) @@ -49,4 +62,84 @@ private[internal] class HDFSReadOnlyLogStore(hadoopConf: Configuration) extends val files = fs.listStatus(path.getParent) files.filter(_.getPath.getName >= path.getName).sortBy(_.getPath.getName).iterator } + + override def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit = { + val isLocalFs = path.getFileSystem(hadoopConf).isInstanceOf[RawLocalFileSystem] + if (isLocalFs) { + // We need to add `synchronized` for RawLocalFileSystem as its rename will not throw an + // exception when the target file exists. Hence we must make sure `exists + rename` in + // `writeInternal` for RawLocalFileSystem is atomic in our tests. + synchronized { + writeInternal(path, actions, overwrite) + } + } else { + // rename is atomic and also will fail when the target file exists. Not need to add the extra + // `synchronized`. + writeInternal(path, actions, overwrite) + } + } + + private def writeInternal(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { + val fc: FileContext = try { + getFileContext(path) + } catch { + case e: IOException if e.getMessage.contains(noAbstractFileSystemExceptionMessage) => + val newException = DeltaErrors.incorrectLogStoreImplementationException(e) + throw newException + } + if (!overwrite && fc.util.exists(path)) { + // This is needed for the tests to throw error with local file system + throw new FileAlreadyExistsException(path.toString) + } + + val tempPath = createTempPath(path) + var streamClosed = false // This flag is to avoid double close + var renameDone = false // This flag is to save the delete operation in most of cases. + val stream = fc.create( + tempPath, EnumSet.of(CREATE), CreateOpts.checksumParam(ChecksumOpt.createDisabled())) + + try { + actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) + stream.close() + streamClosed = true + try { + val renameOpt = if (overwrite) Options.Rename.OVERWRITE else Options.Rename.NONE + fc.rename(tempPath, path, renameOpt) + renameDone = true + // TODO: this is a workaround of HADOOP-16255 - remove this when HADOOP-16255 is resolved + tryRemoveCrcFile(fc, tempPath) + } catch { + case e: org.apache.hadoop.fs.FileAlreadyExistsException => + throw new FileAlreadyExistsException(path.toString) + } + } finally { + if (!streamClosed) { + stream.close() + } + if (!renameDone) { + fc.delete(tempPath, false) + } + } + } + + private def createTempPath(path: Path): Path = { + new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}.tmp") + } + + private def getFileContext(path: Path): FileContext = { + FileContext.getFileContext(path.toUri, hadoopConf) + } + + private def tryRemoveCrcFile(fc: FileContext, path: Path): Unit = { + try { + val checksumFile = new Path(path.getParent, s".${path.getName}.crc") + if (fc.util.exists(checksumFile)) { + // checksum file exists, deleting it + fc.delete(checksumFile, true) + } + } catch { + case NonFatal(_) => // ignore, we are removing crc file as "best-effort" + } + } + } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala index 8082b98f0c7..55da34fa669 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala @@ -19,12 +19,21 @@ package io.delta.standalone.internal.storage import org.apache.hadoop.fs.{FileStatus, Path} /** - * General interface for all critical file system operations required to read the - * [[io.delta.standalone.DeltaLog]]. + * General interface for all critical file system operations required to read and write the + * [[io.delta.standalone.DeltaLog]]. The correctness of the [[io.delta.standalone.DeltaLog]] is + * predicated on the atomicity and durability guarantees of the implementation of this interface. + * Specifically, * - * Provides read functionality only. No writing. + * 1. Atomic visibility of files: Any file written through this store must + * be made visible atomically. In other words, this should not generate partial files. + * + * 2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final + * destination. + * + * 3. Consistent listing: Once a file has been written in a directory, all future listings for + * that directory must return that file. */ -private[internal] trait ReadOnlyLogStore { +private[internal] trait ReadOnlyLogStore { // TODO: rename and refactor /** Read the given `path` */ def read(path: String): Seq[String] = read(new Path(path)) @@ -43,4 +52,32 @@ private[internal] trait ReadOnlyLogStore { * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. */ def listFrom(path: Path): Iterator[FileStatus] + + /** + * Write the given `actions` to the given `path` without overwriting any existing file. + * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file + * already exists. Furthermore, implementation must ensure that the entire file is made + * visible atomically, that is, it should not generate partial files. + */ + final def write(path: String, actions: Iterator[String]): Unit = write(new Path(path), actions) + + /** + * Write the given `actions` to the given `path` with or without overwrite as indicated. + * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file + * already exists and overwrite = false. Furthermore, implementation must ensure that the + * entire file is made visible atomically, that is, it should not generate partial files. + */ + def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit + + /** + * Whether a partial write is visible when writing to `path`. + * + * As this depends on the underlying file system implementations, we require the input of `path` + * here in order to identify the underlying file system, even though in most cases a log store + * only deals with one file system. + * + * The default value is only provided here for legacy reasons, which will be removed. + * Any LogStore implementation should override this instead of relying on the default. + */ + def isPartialWriteVisible(path: Path): Boolean = true } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index ceefe3ebb67..bb37a35cd31 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -19,16 +19,20 @@ package io.delta.standalone.internal.util import java.lang.{String => StringJ} import java.util.{Optional => OptionalJ} -import collection.JavaConverters._ +import scala.collection.JavaConverters._ import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, AddCDCFile => AddCDCFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} import io.delta.standalone.internal.actions.{Action, AddCDCFile, AddFile, CommitInfo, Format, JobInfo, Metadata, NotebookInfo, Protocol, RemoveFile, SetTransaction} /** - * Provide helper methods to convert from Scala to Java types. + * Provide helper methods to convert from Scala to Java types and vice versa. */ private[internal] object ConversionUtils { + /////////////////////////////////////////////////////////////////////////// + // Scala to Java conversions + /////////////////////////////////////////////////////////////////////////// + /** * This is a workaround for a known issue in Scala 2.11: `asJava` doesn't handle `null`. * See https://github.com/scala/scala/pull/4343 @@ -148,7 +152,8 @@ private[internal] object ConversionUtils { toJavaStringOptional(internal.isolationLevel), toJavaBooleanOptional(internal.isBlindAppend), toJavaMapOptional(internal.operationMetrics), - toJavaStringOptional(internal.userMetadata) + toJavaStringOptional(internal.userMetadata), + toJavaStringOptional(internal.engineInfo) ) } @@ -181,7 +186,7 @@ private[internal] object ConversionUtils { def convertAction(internal: Action): ActionJ = internal match { case x: AddFile => convertAddFile(x) - case a: AddCDCFile => convertAddCDCFile(a) + case x: AddCDCFile => convertAddCDCFile(x) case x: RemoveFile => convertRemoveFile(x) case x: CommitInfo => convertCommitInfo(x) case x: Format => convertFormat(x) @@ -191,4 +196,104 @@ private[internal] object ConversionUtils { case x: SetTransaction => convertSetTransaction(x) case x: Protocol => convertProtocol(x) } + + /////////////////////////////////////////////////////////////////////////// + // Java to Scala conversions + /////////////////////////////////////////////////////////////////////////// + +// private implicit def toScalaLongOption(opt: OptionalJ[java.lang.Long]): Option[Long] = +// if (opt.isPresent) Some(opt.get()) else None +// +// private implicit def toScalaStringOption(opt: OptionalJ[StringJ]): Option[String] = +// if (opt.isPresent) Some(opt.get()) else None + + // TODO verify this actually works + private implicit def toScalaOption[J, S](opt: OptionalJ[J]): Option[S] = + if (opt.isPresent) Some(opt.get().asInstanceOf[S]) else None + + def convertActionJ(external: ActionJ): Action = external match { + case x: AddFileJ => convertAddFileJ(x) + case x: RemoveFileJ => convertRemoveFileJ(x) + case x: CommitInfoJ => convertCommitInfoJ(x) + case x: MetadataJ => convertMetadataJ(x) + case x: ProtocolJ => convertProtocolJ(x) + // TODO others + case _ => throw new UnsupportedOperationException("cannot convert this Java Action") + } + + def convertAddFileJ(external: AddFileJ): AddFile = { + AddFile( + external.getPath, + external.getPartitionValues.asScala.toMap, + external.getSize, + external.getModificationTime, + external.isDataChange, + external.getStats, + if (external.getTags != null) external.getTags.asScala.toMap else null + ) + } + + def convertRemoveFileJ(external: RemoveFileJ): RemoveFile = { + RemoveFile( + external.getPath, + external.getDeletionTimestamp, // implicit check this! + external.isDataChange, + external.isExtendedFileMetadata, + if (external.isExtendedFileMetadata && external.getPartitionValues != null) { + external.getPartitionValues.asScala.toMap + } else null, + if (external.isExtendedFileMetadata) external.getSize else 0, + if (external.isExtendedFileMetadata && external.getTags != null) { + external.getTags.asScala.toMap + } else null + ) + } + + def convertCommitInfoJ(external: CommitInfoJ): CommitInfo = { + CommitInfo( + external.getVersion, // implicit check this! + external.getTimestamp, + external.getUserId, // implicit check this! + external.getUserName, // implicit check this! + external.getOperation, + external.getOperationParameters.asScala.toMap, + None, // TODO: Option[JobInfo] + None, // TODO: Option[NotebookInfo] + external.getClusterId, // implicit check this! + external.getReadVersion, // implicit check this! + external.getIsolationLevel, // implicit check this! + external.getIsBlindAppend, // implicit check this! + external.getOperationMetrics, // implicit check this! + external.getUserMetadata, // implicit check this! + external.getEngineInfo // implicit check this! + ) + } + + def convertMetadataJ(external: MetadataJ): Metadata = { + Metadata( + external.getId, + external.getName, + external.getDescription, + convertFormatJ(external.getFormat), + external.getSchema.toJson, + external.getPartitionColumns.asScala, + external.getConfiguration.asScala.toMap, + external.getCreatedTime // implicit check this! + ) + } + + def convertProtocolJ(external: ProtocolJ): Protocol = { + Protocol( + external.getMinReaderVersion, + external.getMinWriterVersion + ) + } + + def convertFormatJ(external: FormatJ): Format = { + Format( + external.getProvider, + external.getOptions.asScala.toMap + ) + } + } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala new file mode 100644 index 00000000000..d6cf81427e1 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala @@ -0,0 +1,87 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType} + +private[internal] object SchemaMergingUtils { + + /** + * Returns all column names in this schema as a flat list. For example, a schema like: + * | - a + * | | - 1 + * | | - 2 + * | - b + * | - c + * | | - nest + * | | - 3 + * will get flattened to: "a", "a.1", "a.2", "b", "c", "c.nest", "c.nest.3" + */ + def explodeNestedFieldNames(schema: StructType): Seq[String] = { + def explode(schema: StructType): Seq[Seq[String]] = { + def recurseIntoComplexTypes(complexType: DataType): Seq[Seq[String]] = { + complexType match { + case s: StructType => explode(s) + case a: ArrayType => recurseIntoComplexTypes(a.getElementType) + case m: MapType => + recurseIntoComplexTypes(m.getKeyType).map(Seq("key") ++ _) ++ + recurseIntoComplexTypes(m.getValueType).map(Seq("value") ++ _) + case _ => Nil + } + } + + schema.getFields.flatMap { f => + val name = f.getName + f.getDataType match { + case s: StructType => + Seq(Seq(name)) ++ explode(s).map(nested => Seq(name) ++ nested) + case a: ArrayType => + Seq(Seq(name)) ++ recurseIntoComplexTypes(a).map(nested => Seq(name) ++ nested) + case m: MapType => + Seq(Seq(name)) ++ recurseIntoComplexTypes(m).map(nested => Seq(name) ++ nested) + case _ => Seq(name) :: Nil + } + } + } + + explode(schema).map { nameParts => + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + } + } + + /** + * Checks if input column names have duplicate identifiers. This throws an exception if + * the duplication exists. + * + * @param schema the schema to check for duplicates + * @param colType column type name, used in an exception message + */ + def checkColumnNameDuplication(schema: StructType, colType: String): Unit = { + val columnNames = explodeNestedFieldNames(schema) + // scalastyle:off caselocale + val names = columnNames.map(_.toLowerCase) + // scalastyle:on caselocale + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"$x" + } + + throw new RuntimeException( + s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") + } + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala new file mode 100644 index 00000000000..14260df9edd --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala @@ -0,0 +1,57 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import io.delta.standalone.internal.exception.DeltaErrors + +private[internal] object SchemaUtils { + + /** + * Verifies that the column names are acceptable by Parquet and henceforth Delta. Parquet doesn't + * accept the characters ' ,;{}()\n\t'. We ensure that neither the data columns nor the partition + * columns have these characters. + */ + def checkFieldNames(names: Seq[String]): Unit = { + ParquetSchemaConverter.checkFieldNames(names) + + // The method checkFieldNames doesn't have a valid regex to search for '\n'. That should be + // fixed in Apache Spark, and we can remove this additional check here. + names.find(_.contains("\n")).foreach(col => throw DeltaErrors.invalidColumnName(col)) + } + + private object ParquetSchemaConverter { + def checkFieldName(name: String): Unit = { + // ,;{}()\n\t= and space are special characters in Parquet schema + checkConversionRequirement( + !name.matches(".*[ ,;{}()\n\t=].*"), + s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\\n\\t=". + |Please use alias to rename it. + """.stripMargin.split("\n").mkString(" ").trim) + } + + def checkFieldNames(names: Seq[String]): Unit = { + names.foreach(checkFieldName) + } + + def checkConversionRequirement(f: => Boolean, message: String): Unit = { + if (!f) { + // TODO: AnalysisException ? + throw new RuntimeException(message) + } + } + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala new file mode 100644 index 00000000000..b7f7d17c634 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala @@ -0,0 +1,27 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class ConversionUtilsSuite extends FunSuite { + // scalastyle:on funsuite + + // TODO: test convertAction and convertActionJ + +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index e323abd53e9..f6e74ac67ee 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -29,7 +29,7 @@ import io.delta.standalone.{DeltaLog, Snapshot} import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, RemoveFile => RemoveFileJ} -import io.delta.standalone.internal.actions.Action +import io.delta.standalone.internal.actions.{Action, Protocol} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.GoldenTableUtils._ import io.delta.standalone.types.{IntegerType, StructField => StructFieldJ, StructType => StructTypeJ} @@ -239,8 +239,8 @@ class DeltaLogSuite extends FunSuite { withLogForGoldenTable("deltalog-invalid-protocol-version") { _ => } } - assert(e.getMessage === - DeltaErrors.InvalidProtocolVersionException(Action.readerVersion, 99).getMessage) + assert(e.getMessage ===new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, + Protocol(99)).getMessage) } test("get commit info") { @@ -495,6 +495,7 @@ class DeltaLogSuite extends FunSuite { Optional.empty(), Optional.empty(), Optional.empty(), + Optional.empty(), Optional.empty()) assert(commitInfoFromBuilderDefaults == commitInfoFromConstructorDefaults) @@ -513,6 +514,7 @@ class DeltaLogSuite extends FunSuite { .isBlindAppend(true) .operationMetrics(Map("test"->"metric").asJava) .userMetadata("user_metadata") + .engineInfo("engine_info") .build() val commitInfoFromConstructor = new CommitInfoJ( Optional.of(0L), @@ -528,7 +530,8 @@ class DeltaLogSuite extends FunSuite { Optional.of("test_level"), Optional.of(true), Optional.of(Map("test"->"metric").asJava), - Optional.of("user_metadata")) + Optional.of("user_metadata"), + Optional.of("engine_info")) assert(commitInfoFromBuilder == commitInfoFromConstructor) } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala new file mode 100644 index 00000000000..e1d017f8936 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -0,0 +1,106 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.{Collections, Optional, UUID} + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{DeltaLog, Operation} +import io.delta.standalone.actions.{AddFile => AddFileJ, Format => FormatJ, Metadata => MetadataJ} +import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} +import io.delta.standalone.internal.util.TestUtils._ +import org.apache.hadoop.conf.Configuration + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class OptimisticTransactionSuite extends FunSuite { + // scalastyle:on funsuite + +// val schema = new StructType(Array( +// new StructField("col1", new IntegerType(), true), +// new StructField("col2", new StringType(), true))) + + val metadata = new MetadataJ(UUID.randomUUID().toString, null, null, new FormatJ(), + Collections.emptyList(), Collections.emptyMap(), Optional.of(100L), new StructType(Array.empty)) + + val add1 = new AddFileJ("fake/path/1", Collections.emptyMap(), 100, 100, true, null, null) + val add2 = new AddFileJ("fake/path/2", Collections.emptyMap(), 100, 100, true, null, null) + + val ManualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + + test("basic") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val actions = Seq(metadata, add1, add2) + txn.commit(actions.asJava, ManualUpdate, "test-writer-id") + + val versionLogs = log.getChanges(0, true).asScala.toList + val readActions = versionLogs(0).getActions.asScala + + assert(actions.toSet.subsetOf(readActions.toSet)) + } + } + + // TODO: test prepareCommit > assert not already committed + + // TODO: test prepareCommit > assert user didn't commit a CommitInfo + + // TODO: test prepareCommit > have more than 1 Metadata in transaction + + // TODO: test prepareCommit > 1st commit & ensureLogDirectoryExist throws + + // TODO: test prepareCommit > 1st commit & commitValidationEnabled & metadataAbsentException + + // TODO: test prepareCommit > 1st commit & !commitValidationEnabled & no metadataAbsentException + + // TODO: test prepareCommit > protocolDowngradeException (reader) + + // TODO: test prepareCommit > protocolDowngradeException (writer) + + // TODO: test prepareCommit > commitValidationEnabled & addFilePartitioningMismatchException + + // TODO: test prepareCommit > !commitValidationEnabled & no addFilePartitioningMismatchException + + // TODO: test prepareCommit > assertProtocolWrite + + // TODO: test prepareCommit > assertRemovable + + // TODO: test verifyNewMetadata > SchemaMergingUtils.checkColumnNameDuplication + + // TODO: test verifyNewMetadata > SchemaUtils.checkFieldNames(dataSchema) > ParquetSchemaConverter + + // TODO: test verifyNewMetadata > SchemaUtils.checkFieldNames(dataSchema) > invalidColumnName + + // TODO: test verifyNewMetadata > ...checkFieldNames(partitionColumns) > ParquetSchemaConverter + + // TODO: test verifyNewMetadata > ...checkFieldNames(partitionColumns) > invalidColumnName + + // TODO: test verifyNewMetadata > Protocol.checkProtocolRequirements + + // TODO: test commit + // - commitInfo is actually added to final actions + // - isBlindAppend == true + // - isBlindAppend == false + // - different operation names + + // TODO: test doCommit > IllegalStateException + + // TODO: test doCommit > DeltaConcurrentModificationException +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala b/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala new file mode 100644 index 00000000000..b26c3c2c151 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala @@ -0,0 +1,38 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import java.io.File +import java.nio.file.Files +import java.util.UUID + +import org.apache.commons.io.FileUtils + +object TestUtils { + + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + def withTempDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory(UUID.randomUUID().toString).toFile + try f(dir) finally { + FileUtils.deleteDirectory(dir) + } + } + +} From db66ca4eb1da1f49b65376aad8614e5b0423d33b Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 17 Sep 2021 11:23:52 -0700 Subject: [PATCH 072/291] fix circle CI errors for delta standalone writer feature branch (#143) --- project/StandaloneMimaExcludes.scala | 7 ++++- .../io/delta/standalone/CommitResult.java | 2 +- .../java/io/delta/standalone/DeltaLog.java | 2 ++ .../java/io/delta/standalone/Operation.java | 12 +++------ .../delta/standalone/actions/CommitInfo.java | 26 +++++++++++++++++++ .../io/delta/standalone/storage/LogStore.java | 24 +++++++++++++++++ .../standalone/internal/ExpressionSuite.scala | 16 ++++++++++++ 7 files changed, 79 insertions(+), 10 deletions(-) diff --git a/project/StandaloneMimaExcludes.scala b/project/StandaloneMimaExcludes.scala index 9100372250b..a14024837cb 100644 --- a/project/StandaloneMimaExcludes.scala +++ b/project/StandaloneMimaExcludes.scala @@ -22,10 +22,15 @@ import com.typesafe.tools.mima.core.ProblemFilters._ */ object StandaloneMimaExcludes { val ignoredABIProblems = Seq( + // scalastyle:off line.size.limit + // Ignore changes to internal Scala codes ProblemFilters.exclude[Problem]("io.delta.standalone.internal.*"), // Public API changes in 0.2.0 -> 0.3.0 - ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.getChanges") + ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.getChanges"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.startTransaction") + + // scalastyle:on line.size.limit ) } diff --git a/standalone/src/main/java/io/delta/standalone/CommitResult.java b/standalone/src/main/java/io/delta/standalone/CommitResult.java index 89026996273..49f7543f7b2 100644 --- a/standalone/src/main/java/io/delta/standalone/CommitResult.java +++ b/standalone/src/main/java/io/delta/standalone/CommitResult.java @@ -1,7 +1,7 @@ package io.delta.standalone; /** - * Wrapper around the result of {@link OptimisticTransaction.commit}. + * Wrapper around the result of {@link OptimisticTransaction#commit(Iterable, Operation, String)}.. */ public class CommitResult { private final long version; diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index 427bd343118..d03c5791fbe 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -75,6 +75,8 @@ public interface DeltaLog { * * Note that all reads in a transaction must go through the returned transaction object, and not * directly to the {@link DeltaLog} otherwise they will not be checked for conflicts. + * + * @return a new OptimisticTransaction */ OptimisticTransaction startTransaction(); diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index 80947b4a1ba..c55a1392edb 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -75,16 +75,14 @@ public Operation(Name name, Map parameters, Map } /** - * TODO - * @return + * @return operation name */ public Name getName() { return name; } /** - * TODO - * @return + * @return operation parameters */ public Map getParameters() { // TODO: be consistent with AddFile getter ternary @@ -92,16 +90,14 @@ public Map getParameters() { } /** - * TODO - * @return + * @return operation metrics */ public Map getOperationMetrics() { return null == operationMetrics ? null : Collections.unmodifiableMap(operationMetrics); } /** - * TODO - * @return + * @return user metadata for this operation */ public Optional getUserMetadata() { return null == userMetadata ? Optional.empty() : userMetadata; diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index 4bc122b72c5..ef57ef6bb37 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -46,6 +46,32 @@ public class CommitInfo implements Action { private final Optional userMetadata; private final Optional engineInfo; + // For binary compatibility with version 0.2.0 + public CommitInfo(Optional version, Timestamp timestamp, Optional userId, + Optional userName, String operation, + Map operationParameters, Optional jobInfo, + Optional notebookInfo, Optional clusterId, + Optional readVersion, Optional isolationLevel, + Optional isBlindAppend, + Optional> operationMetrics, + Optional userMetadata) { + this.version = version; + this.timestamp = timestamp; + this.userId = userId; + this.userName = userName; + this.operation = operation; + this.operationParameters = operationParameters; + this.jobInfo = jobInfo; + this.notebookInfo = notebookInfo; + this.clusterId = clusterId; + this.readVersion = readVersion; + this.isolationLevel = isolationLevel; + this.isBlindAppend = isBlindAppend; + this.operationMetrics = operationMetrics; + this.userMetadata = userMetadata; + this.engineInfo = Optional.empty(); + } + public CommitInfo(Optional version, Timestamp timestamp, Optional userId, Optional userName, String operation, Map operationParameters, Optional jobInfo, diff --git a/standalone/src/main/java/io/delta/standalone/storage/LogStore.java b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java index 5c4922a3b71..851968b69d7 100644 --- a/standalone/src/main/java/io/delta/standalone/storage/LogStore.java +++ b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java @@ -64,6 +64,8 @@ public LogStore(Configuration initHadoopConf) { * Hadoop configuration that should only be used during initialization of LogStore. Each method * should use their `hadoopConf` parameter rather than this (potentially outdated) hadoop * configuration. + * + * @return the initial hadoop configuration. */ public Configuration initHadoopConf() { return initHadoopConf; } @@ -74,6 +76,10 @@ public LogStore(Configuration initHadoopConf) { * Callers of this function are responsible to close the iterator if they are done with it. * * @since 0.3.0 // TODO: double check this will be the new DSW version + * + * @param path the path to load + * @param hadoopConf the latest hadoopConf + * @return the CloseableIterator of lines in the given file. */ public abstract CloseableIterator read(Path path, Configuration hadoopConf); @@ -87,6 +93,11 @@ public LogStore(Configuration initHadoopConf) { * it should not generate partial files. * * @since 0.3.0 // TODO: double check this will be the new DSW version + * + * @param path the path to write to + * @param actions actions to be written + * @param overwrite if true, overwrites the file if it already exists + * @param hadoopConf the latest hadoopConf */ public abstract void write( Path path, @@ -101,6 +112,11 @@ public abstract void write( * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. * * @since 0.3.0 // TODO: double check this will be the new DSW version + * + * @param path the path to load + * @param hadoopConf the latest hadoopConf + * @return an Iterator of the paths lexicographically greater or equal to (UTF-8 sorting) the + * given `path` */ public abstract Iterator listFrom( Path path, @@ -112,6 +128,10 @@ public abstract Iterator listFrom( * Resolve the fully qualified path for the given `path`. * * @since 0.3.0 // TODO: double check this will be the new DSW version + * + * @param path the path to resolve + * @param hadoopConf the latest hadoopConf + * @return the resolved path */ public abstract Path resolvePathOnPhysicalStorage(Path path, Configuration hadoopConf); @@ -121,6 +141,10 @@ public abstract Iterator listFrom( * Whether a partial write is visible for the underlying file system of `path`. * * @since 0.3.0 // TODO: double check this will be the new DSW version + * + * @param path the path in question + * @param hadoopConf the latest hadoopConf + * @return true if partial writes are visible for the given `path`, else false */ public abstract Boolean isPartialWriteVisible(Path path, Configuration hadoopConf); } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala index 17ad2f73127..98bff3a6ac4 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala @@ -1,3 +1,19 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.standalone.internal import scala.collection.JavaConverters._ From 8279d6a61bf45296efb92e9dd7244f48d89bc242 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Mon, 20 Sep 2021 09:31:22 +0800 Subject: [PATCH 073/291] Support for Hive 3 (#36) (#151) --- build.sbt | 23 ++++++++++++++----- .../io/delta/hive/DeltaInputFormat.scala | 7 +----- .../io/delta/hive/DeltaStorageHandler.scala | 2 +- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/build.sbt b/build.sbt index ddfd1945ff5..d24b6d1c638 100644 --- a/build.sbt +++ b/build.sbt @@ -24,8 +24,9 @@ lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") lazy val testScalastyle = taskKey[Unit]("testScalastyle") val sparkVersion = "2.4.3" -val hadoopVersion = "2.7.2" -val hiveVersion = "2.3.7" +val hadoopVersion = "3.1.0" +val hiveVersion = "3.1.2" +val tezVersion = "0.9.2" val hiveDeltaVersion = "0.5.0" lazy val commonSettings = Seq( @@ -129,11 +130,13 @@ lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule(organization = "com.google.protobuf") ), "org.apache.hive" % "hive-metastore" % hiveVersion % "provided" excludeAll( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule("org.apache.hive", "hive-exec") ), "org.apache.hive" % "hive-cli" % hiveVersion % "test" excludeAll( @@ -142,6 +145,8 @@ lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( ExclusionRule("ch.qos.logback", "logback-classic"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), ExclusionRule("org.apache.hive", "hive-exec"), + ExclusionRule("com.google.guava", "guava"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule(organization = "com.google.protobuf") ), "org.scalatest" %% "scalatest" % "3.0.5" % "test" @@ -172,6 +177,7 @@ lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") se "org.apache.hive" % "hive-exec" % hiveVersion % "provided" excludeAll( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") ), "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "test" classifier "tests", @@ -182,6 +188,8 @@ lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") se ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), ExclusionRule("ch.qos.logback", "logback-classic"), + ExclusionRule("com.google.guava", "guava"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm") ), "org.scalatest" %% "scalatest" % "3.0.5" % "test" @@ -205,12 +213,14 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule(organization = "com.google.protobuf") ), "org.jodd" % "jodd-core" % "3.5.2", "org.apache.hive" % "hive-metastore" % hiveVersion % "provided" excludeAll( ExclusionRule(organization = "org.apache.spark"), ExclusionRule(organization = "org.apache.parquet"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule("org.apache.hive", "hive-exec") ), "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "test" classifier "tests", @@ -223,13 +233,14 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") ExclusionRule("ch.qos.logback", "logback-classic"), ExclusionRule("org.pentaho", "pentaho-aggdesigner-algorithm"), ExclusionRule("org.apache.hive", "hive-exec"), + ExclusionRule(organization = "org.eclipse.jetty"), ExclusionRule(organization = "com.google.protobuf") ), "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion % "test", "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion % "test", - "org.apache.tez" % "tez-mapreduce" % "0.8.4" % "test", - "org.apache.tez" % "tez-dag" % "0.8.4" % "test", - "org.apache.tez" % "tez-tests" % "0.8.4" % "test" classifier "tests", + "org.apache.tez" % "tez-mapreduce" % tezVersion % "test", + "org.apache.tez" % "tez-dag" % tezVersion % "test", + "org.apache.tez" % "tez-tests" % tezVersion % "test" classifier "tests", "com.esotericsoftware" % "kryo-shaded" % "4.0.2" % "test", "org.scalatest" %% "scalatest" % "3.0.5" % "test" ) @@ -249,7 +260,7 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("org.slf4j", "slf4j-api"), ExclusionRule("org.apache.parquet", "parquet-hadoop") ), - "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.6.7.1", + "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.9.8", "org.json4s" %% "json4s-jackson" % "3.5.3" excludeAll ( ExclusionRule("com.fasterxml.jackson.core"), ExclusionRule("com.fasterxml.jackson.module") diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala index 72a9c435d4c..72469ba559e 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala @@ -74,12 +74,7 @@ class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable]) reporter: Reporter): RecordReader[NullWritable, ArrayWritable] = { split match { case deltaSplit: DeltaInputSplit => - if (Utilities.getUseVectorizedInputFileFormat(job)) { - throw new UnsupportedOperationException( - "Reading Delta tables using Parquet's VectorizedReader is not supported") - } else { - new DeltaRecordReaderWrapper(this.realInput, deltaSplit, job, reporter) - } + new DeltaRecordReaderWrapper(this.realInput, deltaSplit, job, reporter) case _ => throw new IllegalArgumentException("Expected DeltaInputSplit but it was: " + split) } diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala index 6cbf1fb101e..330fefa09ae 100644 --- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala +++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala @@ -23,10 +23,10 @@ import scala.collection.mutable import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.HiveMetaHook -import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.metastore.api.MetaException import org.apache.hadoop.hive.metastore.api.Table import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.index.IndexSearchCondition import org.apache.hadoop.hive.ql.io.IOConstants From fb0a0ec6b36db1074bf66ec775922ab70a85a928 Mon Sep 17 00:00:00 2001 From: Gerhard Brueckl Date: Mon, 20 Sep 2021 15:34:34 +0200 Subject: [PATCH 074/291] Support ADLS Gen1 and all simple data types (#107) * add PowerBI connector * add parameter to buffer the binary file before reading it to mitigate streaming errors * update global readme to include reference to the Power BI connector * fix issue with online refresh * add inline-documentation update readme.md add *.pbix to .gitignore * - added supprt for ADLS Gen1 and recursive load of folder content in general (option IterateFolderContent) - better support for data types - now all simple data types are supported and casted correctly * fix docs/sample for ADLS Gen1 * - add support from TimeZoneOffset - fix issue with special characters in column names * update README.md Co-authored-by: gbrueckl --- powerbi/PowerBI_Delta.pbit | Bin 46386 -> 62082 bytes powerbi/README.md | 58 ++++++++++++++++++++------ powerbi/fn_ReadDeltaTable.pq | 77 +++++++++++++++++++++-------------- 3 files changed, 92 insertions(+), 43 deletions(-) diff --git a/powerbi/PowerBI_Delta.pbit b/powerbi/PowerBI_Delta.pbit index c9714467ecfc6302b03485189ed10fb6c4e47a32..c7453ef7de8c04fb241745a460449a05fe35830c 100644 GIT binary patch delta 58061 zcmZU)Wk6Ix8~1I1fFLO)CEXw(AhAm+UD6E#g0#}L9HqOvkzTr`MQKUtR#|H4SZay) zaNke9&zH0C<;=_#GuQ9>T|@Lzj~udehEpawq!+(Qu_gkYYd7t3! zkNEwWRGH;__x8F;5{mBg19nbSa`Tl7Il|Sq0kIeHWn_aQcDy7}? zA*L9%5+IU|gT!Y=&iuICFUxKXx1~)VMp}^031j?vI4%T%47e3O2mEnS z#|j_mUU%(FeWitkb%u0tq+%wRJ&O!z_MX(Ns+z2Wp*~RKO&;fY15e?M<|tpac7eB} zPXgY&(1<~#=3N3`M@}w!wg3I8Xc-i9t32zf@ZNX$SbWn?0DZ(e)q1ObyQhr%2ss

`%_E0na4hoLs|Gb(K&5ov9n!gbF;F(ncs+xE11zNDM)48!ml2OI>a& zZ=miKIDRox4VUm;XHr;CT+6g1r1PTmsOnPD=2FL4-_B<460q?7c(g^`mrJI zLd3UyWlB4l>KfgdV`6x=i-nr^L80HR3b5K5o%%ZkR5gtNMKNQSue)5_c3Q3_*p@6U z+qL^3Rwg#rTM2)3-R#R{m+gF!hDO3Va#b-tRoNH0*=X@qPp>M+ZP-E7CHFHZ#;v+_ z+fTk2xOPm}*}1X@*@j`-vgeom5MSQC94$S%Ca=lc0d>YM{rd2#Nv|8NQE|4l4qTon zjTxlAOQ_uX?p1R@??0HCVgv%63_>}PPVy(Tb=N&CuQs3f28+N+V?cuyUBMYC3V+Y-LCuu&!y6&plRa5A-D3H*??@jV^sEJL+oP2lH`cyOO-cd*MJKv~?CkQ*iXjBs#lnqhJP+ zTI#*-X*XSWHax;W$?S0P0hwBq7{8g-V zhep@^B}pQ-1vi~vxR2I)_g8h}<1T3HwxnSiHC;EX(7yrRYxi~LTV{?sn@SxQk~164 zQ(h_g8HS&oQRMhtSq4QP_(Y;?F?^uFIv3n)*`Rd8ajDM*1&B@`RkqnJSfWPTflDtt z*QLf&{Lz~2oK;v+Rc+@Zr%}DpDbuQ^lbUg@9CQq;(kIyIe)f8Mr}*{esAt}?Pfg8( zS(E&8E+Qyuk#J=kT3)7(G8VOsHb&nfI#yExs@74LHB+COicjVjra9_uo^;OYp(2{B zFvHJj99o0~#;${-tOPvP<~)^vsg!x0h$&VjvsD~1?&_8hq$wViQ=rG`3q!^Ii1*(D z9oV`pC&J}E9Eh#|TKEH$+QF`0T34Dv{E`_MJrmUBo=>t{by9Bn{GJ!}D4~(5w=l`2 zG1?#8h6EQvXTJ$_p?Ges0kZbx3;g1lj^3r1DsHr94h-n=iLN=F9`XRZ5cA#G3Y~Fp zrW;pxIm=F`dKXRKOm#YDZJBRZ-9Gly*`H+W5pdK5CF$*O}a<4-T(Ecs=}{gfeOL$h=lN_LkfECsr{AV(IC;Xn^IZHqjJyB!J*hwtO&mi+)1gpV`?y;niBj(P_+!(l{hda? z8n^XE%7v<$X)cR>WR6B->;A=9H+}0&2Td)H;(cTFOyw*%uuGs>#W+(YfLbMMo$i_F zSxBMdWgi_4z@TmK2s1N%uIL=8i#|B(`7#om`X!!a4C@gb(makBL`|HgUk_*m)8%F8l=0d{<@keyBk8^cd|+ECPz`>L5qdx4H;E>qC= zeS)*Y&}u>&aEW1IiQxTXH<5N}adJoj;e_#s$nj-oe9>ADHBP+b`u%SfPKqGOxCrz8 zZPbbGVs*D8y6C3KIk(_*jZnK9Z*TRjHO3mivTN3dvRk}$lXBx%gf~h=)|yt0ZS;3@ zR6xk`ztQ~s_-@w*Dl3D7B*uTZ+pyK8TPOV{v&Q=+pk*7@BrBWaX_s7LMVhXdg!2{o zGb=nPaLl<3N z{_12&(1jN0<8wKcxl8%{HUsDIl5P48WX=XP!@)JtQe;OS{oRB9>*31@1=8raupSas zz3(%EfQ(8sE!XOOTW94LvScMAZy{I}5e>L!8-lgQF4ubx7v_4O#qBQEWJx4HvLnA- zkuxqgekDsgGf5)WwuPkb;c9fzN$*a8?-CsL(+crH-L#Y*jT3X#ElcI|9I-D9UR_dh z&bIK`n`~Je3No{(I@!;@K|Vqn`15^W*6Qpz1x)?lNpsqz?l^l|G1R@BS+Pt$2^soV z7d1v(W{(Dp-iFe_g?D2VxbLr=TYM8_y)>f{HvW|xc|D1_q`+J{4}Gg3>T&*y>HU1F z9Fs+L(2gkw5uU|Z-*>$R3oS0X^ub3Ylcx#Ze#BD3Kkevu%Hx7Cr5I|`vgBDn!wxWA z1N3Q`{_SILqz~n;_u>ad-ACK2;M^MQj@qCno$oxE>0Id7*|Uh?EKIa{N1NY||5xUu zyfS|}AV^HGb!sqrI^D3h|9ISoFX}DHyU$tnjO83t$6hUiYzqP;l!eWHtoa2JmWi6x zq%HkyVCoyorg_(qe-w+9?Pe7O;67zyL+&>;95{=wer1qN}&$!*Il(e+ogsxH^6(+WVpXt9~wi`Jf7 zDALyb7qy21>>#7zr_9<;6VIk9i{jL2NZ7Qt$yGt#zY4qb0AfN_{~^CHeqcK-Gi~gn zI$qA`1KMni7|W_TqBaP>td1?la8fWLS9D#_S^X{$us-F`02h69xB*wI0G(SLz>+Fq zoM0`ss5z-2W?iTtS~`CdJ0iHg9b)irBHb*Df7dSEy?6Faeay}crTMIEb;^uM4!iyO z*y~!Gy{$4vL}f;``id6tjb&FWZ9-mT7$wXelDOKWE!A?^ygeQ91eY9<3roK4`7Iv}~9@_H|z z=j{cXiob@fT`7s$Mnp%$7+a+j7Tb%nmVA6^9gOGOS!Qe^m`F684(TIb28mX(*1S%7*=y zy2@O&*Pc+t-<$&0k{YHi*E=P#SdDkZ z<+A(|W5;eIPm7LKk z_S2_7sHT6a)(|h0lx6cu$+`&Soc+Zl;WyCN`920|O><*iQJ@KN=wp7ri{#I@U8$VP zhi7LQJM03Rgozf0tRE8=cf;1MM`!|8)4VRfF`dVVsse*@0_igw-vTPG>b$M4Z?>Zv zjoEP5rP*DpdROfxBJ}f5Cx)!6UUc&E1)KhX@ADO{l7!hg zz)tp<=_DoGs;jpIN%HR&!vnvdaqGcKFCRPu>GdhF{~pX#%Kc|{=|%%-YwZrm4VoLD-3zigcYwErm?X>M4E|Fl3;4i7^LJzgctn4mX*8#4h(>|sfRW)Z_((LU;^)eiY1#(V3T$#vR zLe#@TMKklA2pp~m+GPPL*S{E+q-8UVZPR_8Vr`Pqc490nshe>4Czuy6HMMD$a=-wgP8vyOtmGcwQp{FS@BY zVg3eotCIWl@UqG`=alKM@-d_JzvCPIJJzYHCL{m0p*v&nYTk^o$?osFGNE?p%y-dg zN%Ayi?VM&xSQ zXKL541?hbSiEC;eUxxNk09Y`RGDf#-H=*+fT={egfd_cY?acpt%**UOG@g_e{n8R( zoU+f%(n_e{FYR6Fpr87t(V-7#w@AyX8)Qn?ayWa<=p(4m8yTal+329688!0mX_5YC zD{r%;urO&VO0?$}u6rtioRjT})yKn6-`a4ZL`#FUT=axBp5%`;jsmLi0i86hRi~9$ zTO;-ruBpVj;SqgnXBBv(Po>wF`6u2EiF5O;mFx?RMa{Cv^btZ$7Owo&VcFbN+M8*V zFL{5l$@5?C{){XXZ%EGVl24{%?VR0?gY2SppVdvx2U|y1XfRO_VwSb>ZVdynwU$iVK(N7m4NS=o%zQwD8fzB@BlCk_uc!{i{J=LH#<5GwoemREr!F~dkQ-=@ZJo?2#0Jq~1kHuZjk8m-XSQmSr zlJG$UHGmGzTvfa%I<|w)gDVk9W7S?@U02{dCtbMMB|H>&eESAAq}iBt@yQy8%V^-s zcs+wjI=g#bf`PTnjt@1s02#-qq9pmK`^JKje$j>$eIM3nPe%{qlyvCJ62Yz&@GHp4 zs)&ze)Pmquri7{CJFu`DaU+8MY(DL1czR3edSt-2uV5 z8P=IGESc<5AIo#|Ph(hMsDTf~i_?MP$RKxMWH8~gM#B6e~CXcf90&5BrcnZwV!QJgy{%k82Z0p0%cSP~jz0dgY~~-cp*Si{AVAs7{zn zEf{HamYT~fV%8M{f~E6&&FgZg$jAX*J=iWo0si3-ab)dxU(>7)gY{fF($ce5^l&b@ zKX^4`jOzW92mMCZouAz}Thk(d8DYh zU&#BHLNDpjouQ@{giN{k4%{g9cU=56CK^&+I8=_GZ*mwKBPOMg{lN4v9rEMfWnWSRKJ&_cHt(V1q#7C|Z{QWjG#JUBJv1M5{ z#KsGti+;T|@dNvh4i{#F`u4_~_v7D!%i*7=DMopIKo;cLa@(hWwnEAw*$ECVyFJ}m zf$S^lv~hixi=T@BgJO5SQIKTHwO_iT&pjQmRy>+}B{sln_&z=c^wH5_ z&z@tjFI>;sSUsyAWTb5Oz|pljJEfCn0uvB6+1iFcC_oDdS1v1S@Va8Sc~%-L-q(?g zYuStUBL6g&PT)s=k7iE|W$jtW!EX6?KFvu}md;qyS}g%xRRb%y#CUKh4aXr-ph>_Vwwi$49g5+mVlXxJcEZeX|B^jDCxBzJo#yfS zdBP{)x&F20>^sfwq5+dsCX#rCMn@(Rt8x3BtibpRew>~lB-2xX3t@#_2*;elTpM5k zLv4;}DWSj5=%%*f*DB#*c%+sZbN?D0rng#h&|&PRK@VgiMhW%d@k*#(yOP$u)KU-Z zm{^&9Ft#p!(({}%zNb6Fx?nVXe-wf{H2~1v=yj>=zZm$&t&p=R`j?%g#prY*C~Qd56t~6D0`t*eM}$XJ;#!@Q7t+umiNi zghy3>^y{Pr{*`tk4wfKu(xk=r;Vi}FNZ+@*)~Wv6!Ixf5;Nn{l1BnxJmgC_veA zl53NyOl!aURd9&(ztXRgeaZ?!|0p+QI9!8RG$9OzvD|k7WcVUHJo0Z{gHrFuJWJp+zeP0PF<(`fH6d3A*b(yuixpwP~r#9_?}J#m<`pjSByj(hXEnusY1D zWNYEi{j#;zq5irhm-cRK^c1y^o0*DH4*ObdCwyhD;@W(hi|HE+!(yWwYYz=F#EQN0 z+teIp#el=z?zl@lf6;IM#&A7&hQTw5>4!r}uaqUn{YXHW`SX2_=Sq#LADq1yJd0#! zBbdANGQN=X+aVfcz=*ZuIdZ%E`zJ@jOuyZ54F6HODc<dxlE(#@s^Vtu z!V0KrKCFTS9F-GiWoMlvSMYB|olIY`F}7d4Ayv`t<+1wWE@g9M2zS@_`{lis@-XL= zMDV8DJeG86S%-b_iz(;u3WptPcw)*s^#0TAxN8BowV<#|Lpbk=@}o)WasPyGQ9wr$Cv# zC7cr$E{1uw>2Pt)VuA3BcIuTdBdRdW8XD89X@9JHQWesUDc zaB;>`X!X1Md@e-qCsruSmc^4904*y|dn`#)y6^d9YLr6Y2eF+pUD^5HvL-LynmM1) zzvFs=YLjjg=*g)KC^(+qlstui{=g42JgF9_6~xeA$w=t$W6MMAGbahKr*}Y|Bxv5 ze9GB8sOD+4n z5A5vaO7k}l;T{i5f7*<%RYeq>+iue%F<*T~+he>J@C`;s9Lit!iBGN>xDP2g2!yQ7 z`I1cak-WzHh2LHC3>V;sql>cFvCr&WA;Z|mn|Kn$He6^bHTAbHCIJ$n_&aANMRMtO z%j^|iKgvC;# z#?RvKi~KPCmuK=vWbTa@vGPYOv6|XgN<*c1`T^<|4uxQ*i5_^S-{uH1a}S-X0%f;T za+3mxmEes|-3(7i^ZRHsYBteTTWJE{qVPCFX^v0>k|Bqri$qLZlPc4`rHWW2Nbly9@@!$ zeRi;(=e&MefKJ%PUOO99RT-0;w}l-hVr_(k6pxT?#Esn3fBb{nTg`fy--V=x#fOwj zQj~~c?!7WoNe=&OEg%2O6agh@K}tyGgy6`#cBP}?Pe8-g(x-f}b}?`@`nIK`(MV~! zSOf5|()`s>lb{x$hqT?Txt?S(ONPR;HZQLfa<`AGzOzF-9oBp81f=m#AM1-~qEu@x z_4niIo%Nb{CLXu*XIQf~Aj(v1!55eI!028YUsvNB!oe1j-d&~pOE^`q+n;=?2@-tj z2Egb4CV9_$mp}VFX*-!j?S{#=_3M?MZn3HE!_-o1pOVMbfe`T^(>Gl8srAwZ1LMPV z1~t1*hQkr+xMdT?|G$BhxZU)$jMa07?kW)d%D3&8H%zI8*^TRb5eKq(LPQb+56G$} zi>K0f-{o#658T$S0-p%%8M9L<&3jqQQeqFyER(61`nfEVck2@LkEU4azKv%M{p%n| zKdsklDrw1urZIy~s56o}_R%x-BT=H^9v+>H(N?p?bq?yFr5Ca%-OQ>@K4^R$YCn$0 zh&B%jr6*Y#BJAd;uZq>hN6MPeSAnHXTYQy}S5MR)g7gf4jd4|JH5IqyLTv6{U14OQ z+Jf~-ykzAEP7E36&v&x={-XA0fcTy^`$C5}KP4kza>eA0^=d#V?NN(Pt$IIU3|W6p ziV1m6M0bcv7-p5)nT!q*askaDWjJC**}N)T6pIgvoK>q!-8k zdSdul3vez07xy)>E*F$1^g+;pL>P6Mcfx8VJG2!;$zqU6Viuh#h4G?g%roO7MP*D? z5887ssNY&By%^HEmJpab=MQJg1b!c2;0Q zl@KQrdPS~0cBcg^{d8EXVGdlb7aKSj0bp|)E3%vtTEoCSn4dscV;cWOB1mN_pzl2m zX7Xnv@6bm|gxq95$P>PkWt8S%a=q9yRe*|=XGr+|`msa?Od4`P9klC zh#)+bKDk@p>THt@$rD+L3T<0i8OE=<&6(Z{0fn@7@rMC{cde8M@k{Ki2Xn8b{4;MT zv;^+of8(jbFx1OZUu!fK1J}s2u8YGB= zHcRO97S=Z9H`-Z9pd%skC;eLd*c|ryGfIk*WVRAg7l3%se8G9LG-UagS4EA#Gbr;-pX3zjB*p~m;V^@CA=`Bz3#vpn0nbL_W zHm@x9evoC3@sMA|R)1C#>3GN~IiP%$q?Q-@ylEMB)4 z!GlF$Btwe>^{H)>q)ZZbzOgO>M zGil9hx{qf35-CQ9-&WY&p*O+T-J!VSrA{X=k=c906gghvKIW@&IineDB2y(q(k6{I zv8QD>&vX^iH-T@N*NgpMVa=RU%v&=A`4R5Lo1CmG(7pLxpd&uK03E@VlvtP7k}Irm zm)c=QKNMSVT`^y+!0nP8jA_G$%z)zIhXc~%=vRt5Xr@gnVw|`%emTD@>_c;f7t*AE zNcCRpib5QP$6}=M=|k=)3QV`e*^A%U`Q7%Hxi)kPlwxpP?*!SqeCx+8Ogibaz29862Ix*F?PfUnVaI;TK$VVfYjxT%W#xPmpTA@>P=mru|;oAe2; z=VGDIM+5F0ITLxKv2fjx&ioJC(NiR*Tg6aN~%U8x?d0M z1s;uWivEl+^T&%znq&WDtAGV|_@2xK+sibH6uJA@yENCGI#lsj4cR~cf0!lee=rMR z#fCIQ5CH1iA9a=o_nJ-23$AUW+eNE2Dn}EY=kGM~X+NOGjf~hh;ev1BG69K?k~!`u zFc-2RuV#f;mhzWx zrWeY2&GvLQv2r$tBT*tNj?)BeqzG z@}Dhnb^ZDa)~wcRWy7J8!Yu*}V=%h{iws zzD%{U)rsp0%qa;@uaw-rY2AIc0rAu=I_IWM1}M3%O1&qAYn%mU`YQu7)bvKy5Rj`7RJ4wEXw_PSzKsv1t-DI8#JkIKP&q|A|r4>qL`q zAAoop`*n)&r%Ut+Z_qtYbc?vbHTiDpKRT9fxX3u-IK^kzR_O#2DQiD>)b`fte0|$p zE%YB&8CpIyGTI+8Hi(!xFQfjIR-VK~lV_~1QeGwTV6MtH;RGReQ7OrT>FG2|G@ZsmZgyvB46DvY#&J)wePvK-J&ovR9yn{`7!be@GegGx zvQ=Y#)QtXrLkr{T*DIO>hSu*ZdoRNN7qIMsz_Ry$@``{56=%>LuShynQoZ4lPgyCc zr)ldmR^6wBo_*Mdx*VTh*#yP1+Z*#b!LN0vCmqi8?4+?t9-RN&f92xNlAtEZoOIxnA36=Nan zYxWVSnEOh1mkpN0vgwA|8vm;-?wEjl<~8iM##m5Uq00HIT8s$nzcIzJJxuVGhlaQ@ zUS!@ZNyMv-XT!e=4S*=YaPe*Li`77J`P4LKkAei)lGxe+TN3gBbj@J;B}zH2hSe@( z*6!b0+~X7(O$amp>9O>6U`cqh)@bF&#Dkb_RUV>!B_urxlpP3=H~}4Qz=Ewwpm*v_ zN_N&HSP_-p4T8EGq_jA)XBx9XK^z<;Y{Q_GyIBm%uQt~xxW7$_rSvg6I$-zjvFd^R zjp-wAHDFD9jKFocpZV(_mRGW`*t>7;7tm^qzFWG=rM>j#DZf-U4Vshz$3>usN z_QD=>LB&MvBSI1a&kts74p{{2IVW!)qcH#JjEuC`L zo1u03EyUJ3?iOBVZ^QV{gzT&x&^~i__CbU85kgWtZJGzG?JZE%fB@HR->I)bfyQf| z_4GdijU2q`Nn8d=7ynL|-*ikb%Sv&G8{f-j$Po_7ivU04-rwN}nIn*c$dBI15)5p- zfrgU>q|k7e5@!gooi3bk<}cKtm-f z$SJvTmo*8!k2U_k@#QR9aUl!B(FYgewT2guBVYPg6?+?H_$9R^z+gwJG|ytFYRy%h>%AEv}AA%#2R;ef`o zQYZ;;n8uv?Z~CzoqB=QA0mdQPrc~xDvmB0HJHM6e*Qjf)YeP(pES0az=JoDbzXTp! z^xtX>7@Y2d1zGTHXFWl12`x(Yi2h=7Wo2(5q+8~nNRd$3U34AZWnWlvThuh>@PHUN zf3J-@O=ehEFUx|n^C$g?zP~Y<9u_=77378ugz{JfE(+MefU%2(^E_xT`(SzGT5vOe zubcnJvM4S}K(ne}HDCJ06z7vzRe4s_S(A_GyFdePcMZjk`Q5jJP1P?>nl7WSh+dA; zrBLbI!$GLiy;OSM<50}=LFMSKUZY)kuBqYuE==u)+F*A2k8Ow)yn!!9)_kl_67|=MgE|Qk?pRE=t$&Zn9>FzlBtDvC%&dl6qJV92T(Kw$-HmuN^H41DeJfn}w z^dEMIztrdf0|AMM2g`q>+R>4eP!nZNj81eLG>S!8`S*2IIc%=+aFnweTOZKnLIhhh6OV~YTBNZK>Ql>)1%F|-3G=L z!>g$xTxN4CvR`j3x+{kX$BT}yunzVDtA8`Jarn3^7DY|^sJ?)+>X7k^1XIgI&)v}X z#ad9*xsL??NEOV0en@_ctjTZ5mH2NvVKLS1*3lXR-OE0;kVa=G^n)gw@^m)y=R_@b zZ;2?2Adp(zJHa1d%zo+=xU)u>CS0r3I?251_>w6R>^{q|HC{WEi6{gWXcW&gk;V_Q z&pH+>a%gjx^02ANg0kW$+t#u?bp0v?q=^`&{*P$^iqd-80+V92R54-og&|q*q<6CB zbYr@IF)R}g1gCv3Gq=!8kn3kWNwvSd!wCRE=!`8=kVV5wIr`@ERY9ZACE>3t35xui z7jiX1YjLs?W34K(o6G}zaU8uk_S1aj#rMVO%=;8O2YrhAc_}HEDN3w2FP_^_|Bfn< z`u*GnQK|wyGyS>U;RT>Uk^X~-2+-iZpD>4=l2mRo7?X2=XYft>2TE|#KF<}S>+ENS;gM3k{l6;~BD=D*e%B=kf(;?NSV7e;9x?4{ zHY;IJd-;8*z08B!%W2oFp>WaHJME>}w(gOF7xA6;@=lvM4%A-c*m1Qce>c=4e9hMs zg+eQZ#e>_KQNhJs*a(23A;>McrQ3*ET?S69TbSrG3l+kXQ>8D?#4rBnM0zyvsbOI? zfePg~XYFtKTRhEPr;jO(8W~_UPyhwd3npo+Fm?LH0Q%j}JHnbyi}(5udzqGlHEmwt zCSTZhWCl_nqzLPJy7zcIM5BX?mf>7xrY!wWDD!5pkV~MpMm$Ilc52ubF`bWhn38-O z6gWW4E(I~WNTD#D^iFbP1$UAQM981}?FVL7tW*okB<9KlB(a{-X`8G*(V~;N;ZPWyVcWVAT%iF}%1{-A^ zT-x!3wmP`uSETU*WJW2n-gLM|7fWXlm8M~fF1yTHn&)jCk?yeLmvuA*_?PqiJ|rK8 zFBBM$rE8^tMvOEF5ksd1`udkR9sFB!A)p*txN?Zn+D-|c^AJEnF%phvKqP8Tmc0haF-Y`=4^19 zqW;79i_|HDGd-j1CgqUR?XOPBM773rN%YH12I99IJz*ZI?!PGEGgz~7*7qtSB)ZN|;8rcc4?Pdsq}tkI#!J`SAe)}zg< zGHh=oe6CikBw8oaY=11Nw#@-KyG0AAE^uC>@yp^=xH~wKU2Dc0>;{m0LZ6vA#Ym>4 zI8T%7j2EpHS5NJa{5E*LA9DC#z8YBYw1zI_Qx0#tEOv3+=tJuKZc(xeJc$*~d5O)s zj0X){?MkTIEd3Bh&5bS1Ma?k_T@Z0Z8&|BO_eKw|y&KM!S~YH0((PRvnt48y_lG4hHQfgz;ti0*%QD)chyz$oJZBo;Kc@sfon@`j9KSAg=R%9QQ zlHGwi#B4#d)S)aHLgtY(3}^@&PL*GC5tkD>WkVyxiQ$(ISEgjr5i2yAUuokl$$3Af zs3RGws#WR~;B~)gY6zdK8g)mR=E_p* z*O@5cJv`kU755oV`Yj1utFQaf88bc`n`y6{!}jBzb(*e5e--~t32f;i4=E_Obi4X@ zxQ~q5SM5+@fdJSMLxl-oOsbR|-xp+XzhFY&5LoN1<2jY*X`{)F8rzb3X4^Aj2L! zd+{oV#%Z}5MZjiV>0#F9xcPrL$S0`jtkdMbl8x_=JIRLEULa-7=QC?6U*Y8t#qm0I zl2v^a>zRnG-fqutnjaF2Au)QoMR9P!BN_?&RR?6k#{KArXyfFC>Sw|W!-D%KGi^e! z&W(oCtQ>;#RMN#KbO$V`>jK#e!;hh-U!iiqFoNJqCF}V(^|(`wLDVSBb!7UF&52|k zbb>$GfP4~4*J?S+qzjgd37-pBDrD*V_W#QHcqqhE41tt}-oPA+psA7IRBrG3UW5=aX3aky(qyo~=0oTwT>5XVfOefKz` zVz+;m2py>LO#o{}(XIAA&rQM(m{91MRjk5jbW_QoLbbVmrN{XGl&b3Z*z)J~gyo%VDb zyl56o|MRIu>f!XzB+pO0aj`{VYnXejGFUj*M9$jW;w&eq`LADC9`6B+IBV*3^>5hL z%1gkS3JMO$(^oD8|&_g?8n{w2ZJ>py^-7lL0bXsChp5K@8tdS9^zyRMJh3P&QfSP`AhlKtfw zt~4K!{rj3UHjUa)bBq~;Y{=Xx+l=ZddN$&g}gKj2*ORO2zg&b%FxD6MFd zpwST&l-IfBfr!cVy7{(P{XqVC$R)Tic1JNcDZ6+%aA8O1k0|5+*JZP59WgwOil_@{ z0%4q4t>!Rc8(%l+%c!9#PK&Yr-{)@wp)6e#+*8_l);e*o3W$EopM9`27a@-au?GAP z+onL_S*bbj=I%QW1@PrObJiP5ulb6SKAU#&pcr||JEnzbm|2!n)NQ_c{CkR4>1`p5 zjwa4Xn|(1iY86_ZM(WGUIEhJ^GyHtBj{jbmwVE(7L6|jqhbh~)fs%P0pyN{<0_Ee~ ze0J;qO|&*1nbWJyr3vBS9m5OXv^+rL-J|q0^FXFh?hOAxOY%{ue93M7v9(0#cPU)A z0?&wG)~s(@0wf+RpL^195SeBN|=GSHeH|IV~@6JC)y2Dd8Ccbd2T&v?`KZHHT};k^-QJ z@vG7C<{&~$4IjiC7bAz>@+{uG7gocdB=bJed#8kFQ_bLRZ{ZZSTIyV&?CSWWh;m;i z8GJl~+cXB)%8C6!#=P7%HX!YEE-wRkq-b{nq z+J6-`%P7t2U#dxh;d3g+V`=qqV96|G0)7ba!hI&HlDC0k@@cn#! zLN*F5F}dv01b45ID~bwv{l=gsk%&&qe$xG{EiD z9Np%8k_X%jy_jt3dI)}snpR`ey5SubcUmTLiu^wK4XUYV188^F8KQfp_4{2^dcQkt zs}+e&5)AVjf1;`f6{df8Ey_ja`s27i;f+J2?LDC}RQalDlQ??xcGmdb33xfsO^3i8 z8i-$TXf0L#^pOUHcCwn6d5ar;34T9a)P!>lXi{0T_JCj$&e(GYHlO|jHcn(UG9cLC z+<{Hk{{)+#AlOiaXOu3JYfxcm=o`$WXaAgFdCJ;AA&r1=KUod>QrSNyr<&MJiDZ)j zX~t1kSc4k&@p7l23B*dLf~Hehs;%J|zYi@x-1%_paTOC*G#Y?kM~2g!oz&;`IVFdz zWP#-er;SR7&z5j@hK1GU+xgO>fs#ItFFP9|Pl#VX8NpjCS8tY@Y0hTfWvCt=+-VY! zt7ySImlHSoSMj)F$89OYW}yJGUwt$?IbLQiNKT~?Qa=6VnL?g6O@wF#;-(k8dfC5_ zEw{W+czdY$`SKMo&;!!UT0Ujh9nBE`A2b7V&Rb8pegEz7+38+D-hs8PY>OtqJE<=b zgn3NGRpo))=S^xj3i6(L0O37Wpu4VpAt$c60Ytb!ac=^-nWpG&wf21;Wgy!{FSAN9 znUfnbuLV#x!iJaR;vhA23Y=!2{0T}2a8YUXD$F^Od*qvNjZri7uY8KSBEOAd!<PO=beu8G zFEqPj&7KPPQQ=zqLP7p$9$KtM^ir-|?RGi!T{!i%Fo5caVU9-A9{*YDRc5i)gTy6e zctwFjCyflKJ#RZ^-ds(a5}wf67C)l=m(CB*{(H%Ok@+y=7EY-`+O-S5T3Vl9Wy*1q1{Hltvm!DPa==0+O5BERpUmDe10Fw{)k}CZ)S$ z(>!Z??&qBIzW4j#{j!F`;aJnP*IaYX>vzRZ9lk9v=eF0lI&%dpFrT;zKJ2O1vxqsNd_;~WvIrg2gk*6aJP|-%V+igDx zxJdAfxvY6cu5BT)M0@BE;Vc89CdhxIrtdiaMoojG{}(hRJfxRcJ4vN;I%&e3F5yAn z^e-{{2Xm{%C@FnZftubje=v5eH1+goD;h1xmqxZ9IzJ!h{a)moNS0Gp9$@)+vmhZZvpflv%+q9}^Zub8zV!(`u5_3ZHq-XC-x�u1XoxuCn6RC6ytt5lDJs#W(^%prr!H)qUEa%S* zPg2|kAW=m;-TT`rgPh4^m8^i}`*C{?ECZXM z`^SlP9-iA|yncG#M`CX#rO!f(>sX0g7?sS7RVE|AD)uJJ=F32KVW9vJ+Yq2wp{2j9 z5HxVc0%V06P^=JiRM_P&D^v)5dlUo%wmzF3#~?a?CNMsYpp-PFN%C3z$w=ihWk!eT zFvl0*h*jpN=8X*AIHY}KvL*E-y-OODn1GbY1&VRF13_wbxpV%sB}fUep(r7&EXM~y zyGl@hZ>s9g86w0WC3O7*9r886*&((C<0+3A+ecghAweEyZy3|WM*5UIg{v-r^QW%Oz z;OTmjA%04a8ElygNc}WmT>oN_nqU5;oG<|{q@#YBve7duq+~Cy-A1WGJ zl)Q^!77ALBGPoA#?$ZI=a)RmMaoB z#6llB`)#p&86*hoB*k_KhrekHfgA2UaKpWS)C^)L;Qt4BLf!=9N0h(PrzlHcT(h$9 zH^UHe4XvrJB@==Z<`tXb<#iSO484(*eWQ&kcrYEl6!pslPtK7jN^=x70U{M$W(u)> zMA?Jq1^dCUEi$sil2XAg;(W?AAdxZN#MAMo;9|77HdT!LFM9DY^!OjID;RS#{I<;N zzW<^du3tL|Pvv>5w|S4P>0bK8jtE(vx7ui>|Etl*n=A3tyCXf-rEDlJXcpvxDC3?L zi1sX_xFD(yD^RChw|HB21S@;QqTu`c5Hb8@O=-mGYpX}TF&Q5-AE~0-J zj#wYN9t6lA4w7EYou~+X@`!wDWSPvTK!3JN6O$wu^HPaPSZEVGWuu0F`laSWRuqed zItW5Q3qt*RG%U7B;>76Vy>p_S528ygTb5SGZqQIeA}N|50`464r}Co1ZqPyHIe~T| zf;KUL`-o|4PVTgDFhf8=6#MbUL5$4Xvmd`X#gFnnSV@IkEJz2F+s~m{U0L^Pe?{4J zR6(q!`q!r8u$7sCvg!DncE^&n4)KUT{bAh6Chzl2q(P)k=H@czdLs2zY9(Aeb6$1~tnwQx zBN;N)bsrO(q3bduz34IhVkZ~>+)#eEanKJv7{C-2=AHEYliDO0by9Eqr%v9~yF6EnErid)6E0h@52z7b{`a`sl`%Y4&HrJ|+ z$rmYPh!#^!G3_6e0_$4{fo{nwJ=Yp$=J9^lKQSVmV*anI1ai$}>}YE#M#5HFe++^; zoLX=t1OEbKT%h?Ys^f4Jgo*_rRDa}N*$PC#6Xnv0ib1IJh*<}pB^Uk?*lh9Q{*{5} z!pC|LFb3calf4o@b@op-{lcby^!Ork+g)s^cMZ|Q3~H*Bx|wr6RVTBgMwf#MS^hkf zh%LsyIsrUdehSO=NfiMb+nmnO@Vbm!IqryNKh4koG z8wywuca^?=R{oZI*^%Ak5a@qP!8%?Z;mVM}WF)jHfBE8?fzRzCLb#jC78F+GLrxMP0`vaSezc_wv&hqF_!U7Pj5#AX$Usbh72u67r_}@5+{(o@PZGY)F3Pz1W6S%DZ3q}Rjf-uVFKQPKg zp>q_3Q4=T_1=qKo0Z=$<5B&fCo4_2`D4|PJZ^USQWUb>X8T6w6-VdlZYSl{pum>utym8QT#? zSzr{o6A2?Y#L>`K4mD{5J&<$PYnlc}(F3U=wZGKRMSZ*piW+hRsiE5cO$|Bk9-up| zKPxnt(R3qpN8`Apw3M>w*tllhg!YVo{1qDdRst4PQYqmBG}O8}qF=ds(i^GuBrY#V z2=kOPB~&zy{(x$Qe6hGvFUt)IDb@>e`ZmJIcgnL;HvW!XGXFJ6T2=Jut~QvMFz2~E z{`k~3a^Y~$9SlrJnE(S59RDjYA#qh~K;W*gbd~qZpGV?5Lt|`?Ik)<`$LcRtNvQN$fb#4sB`w<9r3z5@pV8M2hoeaI7(>yFOHICoT&PL;V8bJ zC>%u%!l+~vjAGyD)foDPcJ#bcK7Zk2b0X(fmy(uVL+fbih3HxWVqK@{L70VU?Ea1o zX#Nx_0{oi)bffa8Y|T=b4F+14JBSWUH8jS(j$R%f#s6C4F2~MV3S_ zx_(JcMpVXpecWjD`E+m7wS(QKYVu5QzqYv2Ib`Vao-hD0FQ?AGZ>2+7Z8Z+>TG}hu z-ZCwyD_OJW&DPJ~PY-6lV4HFCX;S$JyzTFIZF-%law|NKmq2PqJVMv1H;WE+cZ3z* z15$#%-a+v;8gVtJ?v(8&R*Ms~g#C`W0#Ey&LYbJg=5n+q*E_SFVCjc7V&`G=j2 zH(NY?#U_W}5Y1Xeo^xC$uKN_i_Hin3bJMMb$@3i^t?bcy&Fo(LT&fV|RhN9JrMHdy z0%y?)e@N@62mRi#RBJizZV}J!mBxwq0eicDIV}#5(=tVITKj)Ft>*d=GI@~GGHMiD zFTXyV;WQ?!z7u|5qIf6i663c;*65lg;=Oi5VmbYLG9|U_jy-EduH)^L z#6ND_VnZRwHUAF>dp0v-+X8hVryKD$&ANS|7f4Xs_lDKxJ|~RY1GpyT$=RXi_qF1e zqS~9%azUHASLVuwR9*4#4Bea{7FSpDpqSbpoNJh?L0A;K=^}fJ+`I_&b_Y9jfSs91 z9O@gUWsuK0(NXqmy1LXgr-tZ9)u&Jo;NNr>;9Sh|tXpqiUIBjqCM^nmUkXfL{SnlbbB)GXChBP$E2Nzx&ln&s=+ehvd}#-Q)NO`&|tiWs;R7t zm+r!dM+BG)Tt6CtsundWxT#HOhqAgm9j0_&t(mU-l2HNM$Ant<+O|A~ zGj6KTc6}+33MKmqRTPN)Mgf@=RezOj(=nNnW&@g5tO%5dws+iPm``hRQc}M=Ae|_$ zuJjQpk6op7(m7M?wyzKBz|HL1l>=X+0ORx{K^q&-vP%MOs~lak_)h!J1QUO~4Y78w zp877&k}Cibr6t1#ZZJX38ek#CDqmqJ2(+%RhF=zD9jm8METxDShpf7hW2dHQ~}B7*+FV#x_8cB$rCyWjzG_E8Ia1?Nag(-|T&NMlAZ9G^6j zJ-vdh*~{NY<~Sv}ULf$IP1r;?gYW1@R#{5eY9IqUFPJfc8KyQ!L0aM)`ZH2Ktm$&n z3&4eV_B(AJilivW^A^*xe2w6iAxq46Oyu4eml;x=rm?EgNUT@{!w?$D$9FT1$CD}+ zQ}~L1o(O+=-mIMw8oRbX{t|})a8Gwt-fv}1wKTdW<2jwY57uSIhX|+O;uY#F7aWUo zr1aFupgv|wt*9?^#B9E^6Cq2^fHP?UM7=`AUYM&@hYo7m+WEQ-4hzZ6w`Ir9;ditG z74e6-zdHyTKGl28*(ymP&Q>*-cbBa#zfa|oxt52?<23SgQw$ZA^Eo*A?vRquOGAhY zk!?l2PcPL8BwZZ|Bq=2f>yZgXYyJH++G9DkZpMe~g_t+VP4ot@I!WT5S&9G`>3T|1 z0#_UEj!fz`If1NG^VLdc?bTbyLHGZfj{@RWg9I)2CqVO&<({IUI*#GE!s00y9md6f zLE$PYj$k)7J;THINTBJpc;K5r^FF9hOhZ=v*iU`hEP5bN+xRJWppKMb@|=-yn_#`Q zgUz&(>NtUrL5J+Cy#T4-fLH%qQ@b9KxymiarWoE04R<@ud??1sFq-T}ZayB7eU_?v z@Hxw&Uw@hXB8EP}T2R!>D)nt{!>>?>$PU^NGR^Vq>EHr8Fb$#Mlg&+134G-~Quiy4 zjzA0Yn%q){;+oYtFXck~roP}UmgA*Bel^^j({HDnHaOy^m^#K1;2(aGhLlc?&sV3{ z(o?NuY2{j?e9X1P-2Eyi&I>y-e+}2GCa9Y(C|vT1xjd&|iLf}Q-?O>I(20CgzvPm6 zV=^Qgy+yO+)3pB4k}=2tjmJy*iwWKV4m9^x0cR6a_y~RZp$gY?^}>oX8FYasYEeilNK2xF7YJ<--{R9fwGW@uBtr@j&6=^D^GC}K`Yiv zW0IL4!Tqos!*K8tECb_ISF|LZI=%ibuN!X-WhvdxdK#2I^??`pQD?$r8Q@5O7GO9h%HUfk z?N816=J?IDTA~XT_=TI@9qw(t7d!C7(}y(Pj*DwNABDfM@K4-d&Qp(>h4Jmkh{EC9pUr+>Ane^IxVY&Ec6RtB`BLY1Ia|W!`RSz?)Tm=JAC~&B zMr|NpiYB_l^Tt4iz3`si7)b$K2>Wb3E&Kw z+mm~v4ia7arL>i43|Nq~ON)T`eA0>$9^J@z`C!ts*p)BHvVEywGluNBBX#)3li>>hOa`oN0k3y)~)m9W)q7-&3GEC3ARVRSC3JIP$G zdNHLz{F_%TZYxMqTeXvS;S_x8$37IZJ4Xse;Hql$bHAl>`a~Fv;{i73nyiX`9;?T? z|4G3uKE$a$eo~=+*V82F;*LkR_L*2>%1cO0M1neDPEhK_6NfqG#%Av~TgxR}ANe}x zew*Xe%Z1~fl_-k5Ru_?JBwIl{x_9xba(V>~+zIsddnPai4NQ6-?OnXp@qtHOp&Ujn zk3omg5EjpPf$9UNhro>LzEn;@Tc{z9&rk1!LuQ#uEwVV+%G>?}Br5pv!O?ve!(+dO zxB4`k6={4fEip#}M`E^?8}39*{59*omi^qgTgcU@2WldI?^0>U3!hMnk4#V&Y*tV% zxT^f=cTwYh7{L~?{<5um_)2SIj@9TCxKB)$-$l5hTk_J00bsERWqAyrdC*9j8EG{x z!i%Z2M8=sOhTNPRcf8=VZy+ zQ~{dh;-&lx!yEORLF-)-9W&Y>UKL*R4~|D#Gq;IG#A_EU@>KRikcC_`AXXo)H4u)n= zzlgSOLyk^Tf&NGDKl`=?YR9b?aFF#2d+9y9TJe%boOet|H!}z9#J`q$I6zQ!hM|_Z z?X2(#a`yJU{n+Ca8mIwawj|g1N8%=SSw@B-Q(^p?HuDvKla;CwW63~yRP<~l!SV*F z^!j|d)u;``3^%!?D$@sE5q{ds`AB4$dFxpr%)9!*#8Ud$q3(b>=Xi}r#w|%lRe^r9 ziZDn0X5pudfa1DP4QeWE7rs|qW~>d_Fe>|~#~DR#hrw)kW>w0-{z!dlSP{$7t#!J_ zq*csWW1B0a+{=w!SEUt=Tdm^y=~K5b{pPOo0qTX+<@yuc;62A6 z;8yMhqKj43=t-UG(1Iu3Z<|(tOkZxS8p%^}2|78;b%1`B63eFNW{o5ZLoy}z<2JfT zW4r&V>mE-h)Njv?$g&=gryih5Erw)u#8l2_El-jZ!Mo-_K=Q-XG9hbmfy=jQcco23 z{dO)3V_V6eGLvB0GVa_I29>oC0Jg|EJ`}16_eU;4scE=fy>Id!3aHlvZHv4*K7NXn zl7J?}r5fssXdKsvUcDPHrnZ^IDirbBDJJ1YyQ{5qTRpUG7Vsvz8 zSPpBu=ox=7k<_bH%S5iaXPC~#^M{p}l&$MbT`~-?i~15tryosoWof5?z{Gaoz=){P z+y$~zS9N8hIS&bAvzwxjJlr1if&)y}Lkg}iEJy;* zr=kIs2seLT5Ut7V{n!DPd0NbpAByZh_&9X=%Q0~K>15vV(StRe;**C?()-8AN0$Y# za;lcKZS(Vz9f5PSg8ZEn>f5rkL~3-EOi35)vbVg}wGg3}T@uK9cAyq9B*caiA#1^xWboDnz`675pQhhEbq zw?99oto_M`Y)4?7!byxO^Jr2ZwkT~-T>rA?k5axZtD$Fd-YmMTm z5UYVbs#%yL7!>!()+>A=}cHE?%*dA!klCOWitynWoP&X=wH zA$lZVykraFnDnj&I(B=UxF1)KgOL8x40q|588Ak4{r%HZ9=!>!nUW1iaY!XBE=}*u za(lnJUo@zH8E&9Mozv{rY_{dGF@9PTC$M}^Lp$k^Qt0~9)mo-M*d=Uye=@jgy;(YS zT?cSXo(Xm_zq@dl@4xie7j3?|J!OGz)Z9##huz)b_8)gGRNq>gR4)&o{+3y~p+7T- zp!YC5aqSz~UL{QnICo4#h^kovb9Lg`?Aliwsb5m3_FRUvyXubq;VS~u&3bQ%_eFFV zNj~wLty^aG>G0~vo(;}@h{@Q}zE3C-$qu|rKvPQ~{RB)V1V{!Qe)cr$@|S!@YXIh+ znL9v;`dIhxb5 zhqUP%BtFs-nwr+j|Kjda+*RK4EZ%gxh|GGnwCiNJP@ttUXM*ZC-%jLlFatH6ffB$y zNH`X;^&oPy`t5a1duZRW;J&+mg03{%Nm-N9%b?yZ>W{FLvy8W)MZH#%o)qx+T5|b= z3H=^?hfpTSj&xKRbyt&L@byOE^g1OnqU3tE>@0^u$b-B$#dSJTn3#ga47#~m-3euxygzZMUM;g(Hejm?g@HT7*1j=ce%mISa9 zhJOwnf-)pBT_qcr-_jzIzCUQac{~IGT7Q#Ttv7!#)ED)haK!qt=VwY*(hxEY+Iw6@ z%I5c&kx{3C)uTAuk6Un5>OHXC)Xt0|WNaJz+`m(q zZggD}g6l`!6ub02SdQ4eHW2~gw*JM_DtAdTU$THQ-VO%R)J+7b%04p$RYxct&(Q}Ic z-aoB{Z}=*QrhD$Qf``VM9+55KP2r3z+|_4guM$3~n!Og|4T#zLkS9ctVI6)gbEAuB z`AU3J6d&1d!b2EQ`(p>nt$C|78Q;gR^OH5rM{p@!EF$knN+4*{U7ZOMZ4z9l zIlITE+VJrUZtM-_F|bD<5|x55&^43oX)HshBF?*4_ewxbe3naRiR|##l;j|9^?J9d z!k+a$Po$Y`F`_fpTG2c;V!ijtIJOhc%pt9oW{&~zh{{XJ5OMv`H{#}VlN#S3$(A10 zNkhL1?4+-i;;>&WDYpDNpwg90W_MQl6e&D?;v*O~|KV++8X$0G*0Qd{euI_Ls9Lg+ za5X;aMUsuVpHWunR4^Pw$sT6Z-+q~#Y7N^O8pr20@~P!RcGYxt$J&f#{6N^0Rl)L2 z<4-?cVueN*czEv5r+)q|)_9E5ln8x&i^XH0o2b2Qpv2R4m6NcHn7BGMyJIUiY6^K+ zm}#((WZ!UR13W_xC^knnfv0kHS8(9OEFvJ8RESk&9j;1CHeXP(J&#UaBP=3qa(Y0DH0-S#H{d8rrCnX@;i+@2h&(DSmHe%I0k*XPvcf zO`ZHZh*m$Lvu^gmrs7xk&zqB3%|2@4`2r5=V4$k62MP5Rc+u>w?h#ersQ~OJQm|Lh z&~7rR$hF75WcoQ%#BDRF zk~!r3E&XKYtuPKjUP0xB8}QkDREzdf?n<6+XPf0I_jeywvJe8CSQ@@BzRsm&g@T(v z9`OcH2=QqKI4_AG`7#tq&%2G7MZMySG3r9zs;&bn40tdIqqL1$|X?+~R z#Kcs)!MYeNm|-h6J)Zb_r!H@q%uMt>$Lp7 z9EMGB_=^%&*R30~tj@T3YI7nP;UNqcM#8CR`i3BEv4@~A6uZ@#4U~UNSM?f)Ih``O z)l`2^Jo=_q9X+TGIKusKcWfUN?HejwC{Q_8ylS>Pz>ZmDd?>e(?k7`Fxep1P<=DLn z|EMRd?03+OClYqAaF&R^w40KwE<$MIY6?`cqrwcO>s_NnXLVW?b}K)Q>kFC<$Wn_8 z`^|G5QlTII^az@}`^ywxeJa(O-!zS~ESgKz^FT14paSw5YX`XLDVX){N< zGB6rzi&J>NIX+vyQH7*G867T(2?a=^GFpp_AqKKOw4!&}H?LGDx)6cE*9D-(6 zvN$5jZM;VQ1nyuDiQbNu=I?P^38G`o^_XlbIX0QDtchpA8_$bh(A9Pxe3w^_;>ULA zD8z8G@xMOBbNcEnWERQE-HFemHkEGX?F|PVp+34|90G;hH?-hW+|Yt`D#7Uj;D@l& zU|^J``ycX=D|R}3lPV7|Ita`VHv^-CE?~2SZW`RSmOye`}5OQzN~>WGV6t00j-bHwf5UL<(&LsZbuu24=gZL0L{rh zOtgT{leU^4%K5t~5;pN)8$Dc#Z&eH#E8xZ~Fw5lON2OEw0l(KCsiNgAhWGYkfT@qB z-Zj-GiF1sNeZ7$c?&;o>TD8DwI7uLQYNIeZC!=%5Gi@0?Sws)(EEUW7^MDo$aRu>z zog;9)cGCnyl;3vmK442a`%0s zl7$-1pMp;-1Q*~KM`#!CKq*u>ICWxma|ityk;K+KPgUAk1kD_Ka6#(#;@w(e*!!Yq zS-${?@ZC2(2GzvTF!w#dX@rAm^;jn*3$N+cdw{EfLQX=w=!Wvco51iOTA`#NJW)n^ zvtaT%o;HC@qS4+e>rA3f*$DL~1^xZjU@~&T95{1g&GL1O!DQqYs0@Ksnb>eJH3-)9 zim!pwRufDOY7YBzf7#mbOoW6pz}R#9dXH;MPL@eJ*q`m@^F7gCOtaumI5RF@q;8S` z`6hYD%Wb#got4NhF=Srj#q+@}b~YD$;?GvUWZF+8c%ww%FF6b!%sr^aF4k zdl~Dj59iI^U@ucPJAiHoqs4V^hOIz83G$iLC8@E+=Q7q1X7^GZqP3$T3Pl1`$C$a*I5UFAcje;dnh zp^@VG6UCZp%fxA$*$Hy+OYXr2Uq`|s`2tkv?w!tQ){ahe7j~pv8nT=ecmOxh$90*1 z?2ZRH@G2G;5=;7k3*-Og*6BR?D%@2~oEt5{P>CPBdiH)~&FWDq4t`r)65mtakDbz;6S(%$C6n5CDYxEV`DxF@bRZD86fqsWEVUFukwrTp9J|LOM+Kt`Cr0ME^xo$E4x^87F zVt6HaMyk6DZhmUa6uO7~SNf^2Zt9Dy;(w=~u9Pz!y$91zGb;DjUznPwly@3epu7}k zA+DyIjHyIc3D5YVB50au5c|6OAx+Ni1Dhc$b!>kWek?tsu{C#c zjTq=QH7*~ZhLa(Jzew@XCnlHN4#2g!SdWuV8RI}P-a6b8Qs#c}e7PCa0YUanbCc`7 zIg)w)5E~nu9D7P`K~B?^DdtUoSHHhwlBYCbzleh0oOpV)? zdRgD;pbu=y`}mXhwQJ+HA$p8S*cp*NWeX>rhIX+057H3}drQ_gR%sP$lp2(YU>3BN zpGoKMCPzNyKduMuYtFXUZ;vglBC0b%X-OO1zccRPw-!5y)IN)Z|T3vdOm;4 zdKrJqdW&FLZ@PPIi8UK3LR^l7m`+jExm9ujqMJpy$nU- z$%#rYgpF@Rr^E{XF8;JBvv**B8d@^Gk4N=Jz|_LAq$sqp+y+``0}DOHG_gMk(`vbG zC4etTGF&UHu{hoYA;Bga}ue8FBk#Jv__-T~-lr4K<<;SgKVwJxb~vA@_NmIVjpv zhsv+Jn4eXvnDv~yLK9*Rw{AZ!D(I5+`Tp4bkGQjt22F%H&DG|VWXVM*EiH^HK~vD+ z_UrO74FIN|11kk{?6(-n3&1`Hv}UqNd;U4(g}Mp?2_*OwmVr*(i5e;6BOxX8j4#_q z&h%mxYQ|10x*9XA zc2`&=fA_PwB^+V2D1O}oU$@=F!AnjEtNI86S?kk|8e+-Wv+k+ugQw5>IG3K!)V*24 zz~|=$#MboCZKSBnK49T>VUke^I}5JSghrHo$s+gcH|04O`oe;LEJW=uJ{v#=8Ul4#%$&S z7ypLi1Si3Zg{C*e+Ufdb&ODi2bP_rNevroiFS5e6@-&YT z8IaMtlGO0e4t#m*&1WQGvME!D4;OV<>%xGsCs5guFt)fexv2r22N6kBLD-wPJq@E5 z-2Os%3oS(+~gdF zs1x*gnn&XC=2@nzkmkFy+qP}XDXlyg`HV&u_;T>r(l}9t5s!6eH-7?6jzqro<2m_= zI}VLw)tl5mgSLRGxRWe4`#$@`ffMVUO;xIcy@-Q2uG}qdUN);*c`c+1Wp?QJkwvyH zu*Kr>i(j8j3?|Xsk^CICmtiWqQO_*bzUvV!f_(d%j75F7n&Yz)AJ}UuAY}B?w&Tpv zg7rg_h0w93L&zssKvoX4FIp)k{$mUM$(G;;4|FAkV1P15Y-Lz!v8ZeY+MAr?D=(gn z2n9QL%xE0b@p^5R$qp%2^=Mw{3iCB>&p#Ln*@q1imqnN7*Zq9s0_L?U>4R+{u0AZr z+U*W3$3^ty#8EyaoT>*atDA)+J&p!XKX<2` zdNxv6F9MgVZ_78(STalU4L4!nBa~16c!vhTK7qd`KYK8&+}S8f<(bKJntdKMV!_3f;^r zgq>cL7dtD2N3u6{GyUBwge2}uO4JpVe(ox6`GIRqL;RUh;5ff~{v|?xX$iDzXxIs< zsb-tFyrNU1Mf`${@e&`&1V763>+OKO^fxXER-i8V#ri`rGiB-@4Ni0KS9a%JoaE-s z*|s&3Z!_^!Wtyhv(!=xkiN8W$w1dA|EI3t_)GN+lDYP)uR26Az31(|7ZD$6auzvw# z_}gg5r{kOZ-upLUrqWdaOdIOE)Y|!n!_u*`)V$$tf5Pzhj(lHpbF108N^mnqgv;2K zXmLiAKGbq`B9H~gqKiz>_L|+E*P}zNmW+`sv1&x)uvFV_CUCVhi-LJHVKc0~EK0b- zB+lbZQZ;ZphSW*r<_mi7iu_Z)k-Y7rz4YmuYlr=5E-tm4KR}&lmS=7j@H*xIJ%O>} z3$OO7u2$BKYsYL%inpEat~LI9=M$ZgiupN4LgaMyF8vEL9(9#@B*NDL4o(R{1Z9qh zt0JhS2kxRtU3R!d2hbXsSN=(fpjxWecz6GdbTQdA$2D((_kO^Mp%uGgq}bX$WX?WI zNKG#rZA|!k09?S>Sr93HvdkPCe}6vJ{3#W6H6*N)HUahX$$MvkEh7q!$?bm&`-{WV zy$Pv=jiNz4x-={6ur&HYH+BWLihW%1AR|#NvHx-8m-81?rxAJFUe(6m0^89_pP2hp z*q!@<+0Edm^sz^?UZ{dwy{^O_KD++LAxZ+U*I6n0Nz}g4EB}&EUOz zG2JX@Kgvx>^0FRR)=RRt{o=DPFrTZJW4}FlS3agXo}ZuIuwVYf#l*l0*$@frO)o=s zG{MDkJw;}%8@)dbDiDCzK}5!-`wth_akGlR04)`s3g^Tj*c*PBW6n6(GU1y zRN{iqM;jF`8=5!A&J?l#P`e!`2;AC=V5DS=l1Hd0qa(b6zfnd9Qec#kitlVh2-jqx zvX7jr^*`2lz471Zyt#psy&!_V%65ImBLI89>%KNqjx?F82csTp<=T+CRyBjZu3Wyyp8_wS~v|0DTphk9vKcFXC$AN#WS@ zfQ)XR{+?bxzIYall(4i5o!8SOH4=SShc;tr%!w9}3K+_g)oaq3d@XT0>RhBglG5Wj zM7J-)t%Jt#UxQ`=myGybHA%CDby9*ZFx?_x`GLmNBZ&FjujxQ9AVGWCt*{lVVGSz; zC0|$FENB*+s=_+$0qq2g+u~R69(oqQB=DZYiOMYAGRV956S} zz+s7LwE3vcP6IcZAh8gytYJkUuZmYAtb;S{hHEK6I}(%?3W$Owq`ozTEE}j=*8)}R zMkv)fp&YLzO11t0RDIh6|B(B)o0X;4(^x6R>-8P%3-AY5GF?^?%+|kK9 zdA4kZ&C1`>#1t{f7P*T^iIf!VqpJcteGL{8M9RP{QdtA0`r8=rNmTSvvyDM@)-^x9 za{m|`0Rd;*j_u=JcQEz^G-!u1gbOwp`LM?TRss_88efd69in4THtvgrVo1Qb1-9+a zMNK_dvXhK<^)E&1dMT{8AtShvV26f?5~u2oHfnG_Q-dS?zM>~COUvKbxuOt1ngWj- zF1r6pA$sQa)JBsE)}C7`Bz7`RApBe;#HWX3)9WR}rNy_#X}2kU39zny9aRr>iLpl! zX5#il(mL)K1dJJ$)Hdd1P`@k}78c5*H`ujS@keSp%Yc*LXDs z8E`5p<-^*Db&25-0N~Xw39lG*UytyRksdwQXwIM!%a-zn%24?!j6a`4ZZb!t0DdvO z+18pwSmoDmetdLB3lVDjE==H})}*R8cVe~{$AH0blFrhS#?|7qG+ zu!3sZC$$DP?Tg#G6Dk+%8|1R{ReIOy>HsW2D5#&^>|=%A4IR$8S0Wo5QPD)l`e_qx zsAQs>oUPx$-(;er7e>LWSzDE?y1V~QCSvVg_|t$}lR|OKb)M+DMnBKEevPRa|EGIj zp7ms^xi$Iz?!dx?RQ3B0U((ldhsH({FYg|h#cen6@GXHY&FQI{0S&Xu55f}@X-Ukx zUS`ksuk~Rm17P3}e&lxQ&{yFypjQ`3$~n#}9k-mHUVh?=({%n^3F14*Y$oMW*;Qc+rL{k z=LnAxz&HWs_2sg3a3|gdPx&zJbubLS!uFlowu~pNAGR*e+OCJCe#+8X*e?=6?i4TJ z#;XUJ?CpiZQ9(v61koRJ?42i@J1M)#=oU?$>gIJpX2qLSQ(?Z?^Xprb1_cYWMw+O8 z+1b~WXF9slpYeA0LR<2wt_0DGl~jMG5<>zaKo@p-F$P=Y#6Xjv(+ zTt>IE$eO%-bBq`B?O)CG#D!eL2(KusXUcT>pow}w#?1k4tfkKFydb^%43k!E&be>< z0Z#9A)Au6I<0}HoS`WwX^u+OV^!}_YzfR(VLREd&$4DKx(LBUr1J9xT*=|47mG?GwF#Yr_%&VVca)%dtIT$1g)| znYbh;i{QaaJHZoO>^=&Jc0$DpFDLOrxh_4fWDtLv+0$vgTJ*xF?40Mn<4>X%_J3clx zRhT=+E4cVsrEp<6_a21yHfNVkM_t`4AYZUz-do4mS}^cQ!mkuD5509iscgpbUtHw1 zte-RM&mBUa$W{&JwkzyMHI@NS`@ zMes|Qww3rhfKgp~y_I1lNAh~}9|rCgi$Odh?qAwR1gs6AXEh@KyidzLV^5c1^0!_{ zG0D6vd`kRD-Y)K?ER>kw3=vp87>^pyFjD)a!#*byINWq~bo6A_*07L+ew#mo-0mK} ztnEO^)<&j0-_g?R$jjw7?U|#;=ys#FjHji81XV!H1MZu>Z>HTtmib3$C3r75S<&Rn z;=1oFnVd#N(%oC0Y1ugGXMBYuvwyyr7SB$tDt_yJ9qhD+Ilalm?ZeKQ?IIuXUj>^l#b?(@g>e?$a1F$lVM7wcEEd#*6gC|8y*ijqcwZBVrtOB?bnuHU9VLo>+{R+ zr`c0?a%o#Y!yx{9%|VEHmfB}@PI>%NP=%Ps!`&3N7nXlwh}f9Xj9I8&G>CCssi}Ja zk1YGK-D2E(Cj1W!QfT$Hon0StD;g$^O67 z3t!1;1&v{=^A-?Y-phW6KF zV(AhLk|*U)xNF1mpA3kv^9L)!$M&d_ub&qW&%0w5FKWu(yvoh&?biLgLgSFdMW@)= zZS>}?!wto@01=dz( z>3{s-{4u1xIWsMaSdLUeDkgk_FWA_+gzQZ<(;w3#cNa{zz>2UWYAV^IPo4CeDSO;h zo1=oQWKHth!Sb_Xd$`Ibn3 z@Sw3x<`nl*CkY`$zz=(M*Usk(s`Un>j6DHl1rDC-t=e86bd>VV)g3ix0EfM{b4Ktn z59M4_eLsId&6f7Z+RB}!$%XiOaG!;^ioBhc+K0K>VzK`BlzZf(sDb5v`cGz1~T7=xUERUX*Ztvrx@h_ zI7RzxAk;oSgzu*0W=a?`gjdOM!IKf|padnUbTEe)juF@-s{|!Z--xl_3e?PGy8$Y4 zuPui;Eb*pL`~7 zg(}RF!qUMCKnPh1r}zX2NC&--#Ny$TMYd11r>^bXb@*orfL~rjf5V~c47j+`q1JNs z)Cl2bQoj|IdN&F&!OP}S>9%*&tDP%@%aWDzFT6-x>_Oj`;_XLnzXOceuzME2!qU1% zJ)+Axp0(%i{2^M-sr2&^h;)zTlCER!q_}-$<_2J7gz@Fhah`DDN*(WVUSfbtc$MF| zrLto#g`l5de+2l_;+OG1d$6Ckao$S{g=g4x)$QYP+9^N(o{o_2-Qs#cPS!T@uE0gv zvfk6j7|Wm#ZBL*}3(4STG8?fMVCyG3fS73coPl~6E4J?=4p%{Z-~@CUkrEildy))@ zuX6?SqrPez_Xm&NGENGqe_C2)7!XZ5-+$52Vw3hMf=PO(^@Wi8^M}TdaSQG(&E=i- z{1Y4Y;>e-MQ$90wg4Ex~q9ZC+R%M@uv$0wzX%t=LQ&Q_|ANxly& z$a{I-O%HeJV%ARbMGEg_kc#i(Dfj;$rrtU%sxNB%{VFO70wUcC(%lWx-6btCbV$bl zN2FnJq@}w{x~U+j)MMIg-rF=)&Zq)IrP$*|v0S1H~_*(53_M4GWqE7ai+8gIDZHM~Dxj802>t2XOh z5ur@1ivlIKI6X-xkgRB8NJN#Ex~6z>{Bk}kmtNZ~GBmbu_Txvh_f0`y-K_T%X3=l!jN}psH5~rcoz18f1DQ!6wSmDqX|4xO48k4crROnWnB02usNs!FqRDnZ8RMn23= zbCE{qb~VxbcK547sCDXC$djg1Ae}Kh8~qF$Y0!qc6%b6Vkm1ZzLTU0RM3S`^xPFV-Kc?-bMqpa1D=Tu!y#) z%umXn-+&{=8#sq^;8@82xtkb;F7j^clW?jMZY#lZM}Foa%v_0W4v1*wWLdbVU0ZW+ z;sLGcV-zVh|2FHtfwkYLrchD*+k27{xNONO)hyOh7&j2v0-y}t8ZynC53Kj*Y?{2< z>rlKFj{Jp?htUQX9I<~-m{gatVNzQK%)+itFh;v1h4zZ)lA|W%+k$dV`qs5%I!cB` zBZGO{MfAP9a=7x#aejlqnDi> zfjDEsJ6vyO_chL%aICJj@>t{+X@WMZ?_cX4n%>-kh8F!O9G9RgYS7M5<7c@*tIR7t z?-TT(=a==T`cr8OZqV;jxx&Ky45yP3@H9^jkv&nd^_$|WgVOQAvaXi8s|UsElaMPG zH;$~(#A~hnVcH*aCxHIWsIJ7#F;T}ITHjmGQn@b~}z((%!-Ck=D{>8?kq-b2DHlsYn#7z-FWG{x(p>zUY5s8}r zPRVF)lPI<%-&ee6t2P|}Oxg27{^W2asu}-MNj1*)xyshv(w-{t1Lsrptq#NU31)DT z2!G6d%yI`FTw~#p$30jtJv#@qis1=rsGmJs@`|@Mdl$*h+HUT8azhPNv)u4>uy}rI z9k`m6$>{pLaQk@yk$VLuQaiWuw!p-iFUp^u`DN%*E^j|Wzwlw6t+MiJhvH-Q1JODi zD2;~y;s|X-_}MZTZ{1+BeKk}AwqXYHaHeyNB|=S?nh`_iv^silB536*pZ3sVE|>HD zWQ5Ra8#9?*D2g0P)%6Bgm)K9PlQVUq*Qr)Ep1rFo$!hq)WL(T^N!I5x$!VS=^cRcG zrzv^PLn4KZrO5 zPXSWORvBz*SGB3=pp>5}Iue484%MVcpC|bCQpq|jrVCG*9Oz4;W%dOdP!tqU?cmXw zMY;1uCat}7_Q8*mwLWY*5!f_ka`yNW3Tz~@4i%pdeF>}xMcQOLWpO)$s{KS$8(|I@ zmM*aTY#pnc;>jLGqbr!wv6$u=Y2yu>Sv1B z!pPlk>QW>CtX!YfmxpwL{Re7LUqAS!*%l(&*hyh%{`t@O&{Y5e=>dHIQSkg6-1=QRINl2&ewcdA8EKWRBH((KZv7PiH zp=cwjEDjg{q(DG`;No90Ef3ZQxlmLp}A2kCOqamR%ZRIp==jBJY}c19b} z*P&1FMPV$~_5@#Dm>B53nq;W}3HX<9z}a-bC^xUXrJ;C8Dy4(@vxr{OYqnf_^Jf54 z66c&U{+O3$kD|+>b*EP)VF24Jr;$|l*Ji)Ab`CqDh$3A5rH}*a1vBNQTcN-iX6g(Z z0vz(Q}YE#l(Bu5+>gml9b%P1 zxxPc<6=46>lK*d`_?1QYHr-)h(cxs5fd1%7O^BkXJ+uGS71043%%r-*Dt}*oD)VAz zRJj?yce#rNK$)Yi7(uB()Bo1O%_VW&c`VsWcgAf)K;#Ralx^&g$}ImpU%}{hUN175 z#vYyNE=vgXnP)KSu@XfFZly{{<4E-lkPPK8K z6oFjSFrUZMXgv|#F&kWgVM(OyM>wAnu=P!cnLl%)1uH&fbY?{}G&f0m3bmiphAqnl-e%oR-!jg(#Z-WTSdy=J+tEq)WtuR(Wl!R7P7iC zt!o&eOc!W}=pSM_m?Xblk3Sv#aLC}`drgA&69?PJoN!A}SY)@GJH0T>9&TBz6%fTL;9 zp>ams{1Nx9x@92C3vnY|iIV;Iy+4GS@Ouep_IRUN!!P#rBspvm#407&Q^5Bn@Xz=t z{_dePLoZ==d_+FK8Fn|Ny=s%=7r-3v_2Ml=UL45;Q!%W?DpQ$W@VnqKq-iW>$53?P z!ID+}ao-i;q`ivMBuZ>bx_fbZ@Zo_bN9^KVQ9R{X?+gBauj)BoXcI|HnoW&tyz;Ttk3An1{(SQsLs3L!yrW?y=n`=FA}d9M9eYG#SxSo2?%)~{Z$e~8MCW= z(Tkd%^FLH8zMgDUF|H~Q#XIZa-oPk1666gzK3?OzTD*-=dT>8vBv=be%^u#i8hDDjVRC~_v!kmk!#%@rtU-%vu+%b<`h#*6EP>0B7N0CRw} z=_@8Hn{K%uW!bHk##{8J63R!kO^6|36PvrqzB(uu>Ew-Psr#acnq2)2k+;;)*DA8s zK>`+%ulQ(?l5Z^geH}02_?zldBDlFr z9V-!+w>zWG&ul~#mMQ?bs~YAoGHGu23_{4qJ*mjUN*t4nMYFR_^ZRKeWa;c$Y5p%{ zL7pIsFi4eq$Z$9P39=-N^LrV@-_||LjLVbWD~qoDo-nIja`rmS#N|r4kEsK23DVXM z&849724$Bp1zL5l&6rdRHhEt{^lu>g;hdsZFVi3-zBQaH_}SeKOto7Q`CM)ApcPgCpdV)`2T{W zjhY)Bg}{u7&(k|zb!sBsj4c=M!VSKdhnI9XAE}w&f5;-dZRx+vok^_pCCiv|=zQ$6 z5Wn0v0nM3hj$lQ6YG5;qLaK=rVFVjKeVe))l6wFq4l(JzcV{aPrcYTp&2yr;{0MAW z@|q@@4>_ri=wL3=1vLjnmTIx<#+*enlm+(hh5?>fnUTz{Q_l`(KYIxOXYz}@DcEv` zn4NZ8d+*bLC6Rp&*tGheRlu0^_5a#)WZ@NCKl~NQRpqvpne2gBooqLFSdCBv={Er3 zJkJ+N#^|lb3ZDzI_6rseyBd>b%arjFlp${(SY1#kvOcbf&|xK%{E?kns4F;B##`LH zoFiI8>(eDt$pJSGc(xPB+>AHtEkjrdjh}?%!`^X={T!+KoKLEckmg@Ra@G@O@ja=& z)Bj)=R~bcy+Y@F1|Ch5eo;XXKfkgB43j?$WrG!kg$IxOri4OhsePf80`I~{Bx3`x5 zR#8D$__Z%TwM6tmA$O&Dq?w2CFzg7?p(h?fW^O$2Dgu=+MLxuj&SV&Nj;Hudg>Zl6 zGAkf^p5%q)yNgk_6smP2qWV_DBZXx{+?#n7cwd=s_bx%1zY71$9{tHn*6%u7e~MJM zN>Q9`?UC)~|8i=orA47rCEQ^>aaQd#QA2&rs*3>Rl1TK{92c@6J7)Afq`Rb~wx2bKjDcRBBrYWVM_- zP%2QURyIUP^sM3x0+>uXd8o<|SOuuP%7#!PgA|bL@Blmu-4z%UKA0aK)mIiQl2{#W zKwA%IswrMWn`dQpwzv(O1wQf0m0c_BS6*#YSb3oLw+UUyBng1ZKNSDPS6kBzTmMhk z?>hq-M=~C0$S@pwh)@0Pz_vu>{+^>Yf#MlLzPNgB=CZSQz2)J}&46*2p<`T{3-e)9*QI4vkZ7kz z;p%9p;)qU#V0x%irwm|RGWZUONf8?!aUv0C_j#U*0Jni^{D2kF$-7$<^p>rIinbHy zjmea@KAewA5thIA!25d?Lfj3_KIzWJ7ydRmNB87vwV8bJJ|UMoeTc*p&FUKPu0~Gf zk?T!S({no?9Eu6WX9ob34Dfm zmRoD`=Q!uK`{xZh)8l)%7g6@TZp&cdd#Y-G0O+n)PRy4xyYdLyPW#;69NzcZX7yXA zp1nNeO!=^1A%C9))ctRJ(+%G8gACSInfZhvhFa+k${~mu6=gG&GYY660SZWK_J_;J zF^!_hP>hzK&YvTUU-@ENrhI?V_(F2OMbsEk-5T`Pw(Y{ie9i_Ze&?q~=T|5l$MOLG z?s%i^f81w-DeM9=%f*^_#ElOMv2a^$YySyMmf4WlVOsiNh%|?AT?+RJVXPH!a1dO4 z3CB99i@VaFJ~mjDRGT>#S|MJPA`cI+iVQ8#`m~2&!e*^GE79wi4X8jh!|EnSrm@K# zY&b9A!G{pxM&khY3K=iqBCKi)TvMQ6Q5bEkwTfW0rY`F2#XMN*dLuI`)n-Dqf`zh> zvtL3HQ&?r?)7%B`Y=HW<4|N$Aj;)jqE#VHK`|-1VS_u$u`o|`2leCsAk5?FaHui#^ zEz1O7gdsskTaNs zxAhGtY(PJx*2O?`O6~Tui?nYr>|3J@21Qg#_qfclU^MWF0X&Wvon4J9#@Wqifq* zu71@~t+wQb$ACI}LRlX>LZR5{rEbzbNFl75b$T=|c89V;-1zX;7=irE3;#99`QHIk`@8Z!lDlHA&mNwH{~MokL%P-v%W69Z+%{kXlzosHKw)LsXf zX`*ptch~=#tey{k08Ur!wcODv`}Pr$YjxV6(F4)xD3;n8aim%x*Bp=VNWupGWzM_l zm7=?a*1Xcgs-s(-6&rP4vV?Yt?O3-*|4~h98!+ai>jprV`1#WFg$^K`aZ&YXVdKu+vqq!%!4R4Pz zov^Z$$7Wgk_CRg>^_V(|HuZr!Q%!4<9tEAxjD1ypfz}dNR%X>d-f-g>^{D#H1blZdC1}1G<-AdOK-Ab=`uMrmAN|0pbu;gftNNly+P&zp$X(_Mb zZ)lpDTKRh@i$sh_I|_(XRCF+mo%ErJ+Xc*+#AP=|E|PByLbW&Z$)9Jv{p~VVRl+SS zZMS4(@^Po4NkfEM!xL|CRe?k1Wf0jfoC9(SbV8~R(w`yUwIM!)*SPApGFSc+O&9W` zTJrv`iJ{CRSMXkkED59EL5tixBH{1E8dU^F{of!DTG?6{8 z)?$2Uk=}n}OqBgEsQ%K?0p-*ZZQoRX zIV#f{Cnbo9kwQUwmRuWo?vFwn^+ofBUEjA=rmO(z#NgS0q5gzcA$FaQZ)PIDlIeTHzoumW zsvt9KYRD(=nf+NowoNA#0&uU924%w6?Jd{o@*B>PuJAi@rdaH=CfF)Mx11#G1MJ!S zm3BB{q*Kj4)`3Za+ylMyMd{ZFy`o;qFwjOLPIT1u^yTMgxQ3D{!x_0z7!k->Xd>Qw z!U3i^5j^%_`n-!BHpOw4PWT4y1#vbJn@FF2fW{_IL6*w7{~<&jcv~YOP9N+M7@<}K z84FT%KhDaonGc&L#kaz~F z-HqTkVCMB&{&@j7%%OC>Rn2vOqoi_U&)W4!e-XltW`>J=gKRcwWI+RAmy&%q9Yk$d z3=s5ys2k<)B1falCxueYfx^j??vyf|Shad)-OteHG+vjMQ0elz|2KfHl}KH}`J=|- zzsEmuMchq)m67`f$9wv9dSl0sS@(z8QQ}A0!UGy&H8oK1!Cpn}+@U z3}|)F@Sy2QfpZClSMGT}i}q+e%O5D`q&1aWxv0&KhN`@WgX~GzEZC8i;c$B^v zy!+Wq67`^Kc#bN7c$4W%(V}F9IMqkFTwr;Dd`w)h^Hmht&UarPJa9Ne!E~3Ry$&id z3aesg?;CDejEh(+`uWEcwdkrKnrYpvoI+lP^IU#QT|rUT*vnTJD0oii=@XidZ=4d& z3wNQB51A`_WknBHyBWS2_lsg)Uc?8BpFPsUiBbvLPf473=^S4#)T(m-h!2fPLILP| zQ{LHr72IL3f8HN+{$tCu8CQq>U`;gs= z_cEg-70=}jWGaLAqp^*phI?eohui>*oE}+L^Thz{r9~Of#iL3;DnaAL%R@>*iu+TR ziPR`VH*?F|O{rlfQEadS2C%yKK1p6T@QT{m-9fcDaXQX5ge#;84_Ch|n#?7H+)8&p)7F19c-Q&$xe(Fc`ZOg+?BT z!)XY9h?jRA`s8UxFBgBO(@9ad;teNflt{tPS!R0WPmMURXjt z^@Sy<+ehOKS9SMOfdYiIY<=MKq|u-N!Xs-*b~LK1(v3?aOJ+I`;COqC5DX5C;q!>K z=iCl+yIJm5{_E)(ab66fbas_O?gD6}B97|B=(X)a`%GHSpilkX#eH6Asz>^B-@X zhtoRsn}v&h=7LKY`f9p5G7L_?raJB6K`xkQnXfarY4*Pd{(KEiE~h@@ZoIwL6F2sE zXa26mRU@K$>o9w7K-eJMWUT*s``K`5ZhF{c*{*5Y{EL1o{Buo;QbBal&E%YgXhyhh zbw5`c2HW5XJcQrhGPgvlw*8z)xzTD_v1Pcp>9oQ8MaSzr@QZZ<-45B{s6qOuK`d-9 z!e%qq_cc&uV9oLuMCD|9CH5+(NO=nNhST&7R>d#e_?iF^NN8>A5pz~u!?D?Z-VyV8 zhaGl|M8hZR89S4{IeGxIAtrw9r*O0IR_Nz>D#k}b@}vmqwV_$R4)}IiHifh>#5=rT z=py-eCiH;Uu>3DQLXGUc6wCH6A^i);nFk`Zc?rMD9`>;1tlJ zfRBVyzvw*Su77ELEP0fZJ1XipW_>KrY363hxNN4BV2b+AA@jpwKh@17jpp2Z_U)$o_%(LB z_xApViS2)`%(N)OA)aokwIQGqXAcUWqAf4y-lsdWhhBy;Fb`obw{=hW=~fUS$IcyA zX_JU>>(q+yWEeyX$wg*yPvr>_^1hKpSd*3|@1JXXz0d>Nf@|@3)U8c(4rP6cir6}n zE4zs)R3q-*^(pov>smiNHkBp1hSF&yHdo5T>dnR(6XmlZ>%%BTAsWMC`d?GVF&2sP zpT756?9GiPh1Cch^^4dO(q=gKukOgJ4h&mCre(_iZhzgYE~;lV{U$jiHhc{homcS@7O)tKlKm`mffN zUyd1|-nwhN?5F}?t`oLAbT(pSJX2drGsRUPqME`k`%1QM#pR6h7+Cipga2!z-dRz%* zI5*M1_DY%V@|rL7;o?!`Un@n>GvDbZt8Z}`eJqlao}lu(@@xD!$Ky%18&hA<3KUy0pCjOEwH!pv$R%@@D zS*I|qm`KNnw)$S2mctZ;WjlZ-Xc6GZ0R#P5RPl<`Dd=sF$CGArxH%{Jch$!x%>gl= zZFk1B?Zl5S)5)-Ztn^OOr=S}FWa(afl05T6p^#l$Q|a;u``Z5b^OVdC&q*SB1MPWb zZo8oV{%?~{dx~Xo=fRYgG^05>1@n|HF_GV+Y0}{SQCL*zt&|kfLJ&`+v4_&$ahrBd zoTF7ocox~%JOwJTNOCo@Tsn42V0{Qv?w=0f&KhsR7*u7>??u-iVlKQ$Wx`b53MfN_yY;m-%AiO-PI)-HRLOdJTF)>;qe%i zq%)WIYgc&PM2GkQp(WhSBxUSw__e}`3#kIH>o#Bf;mitG&0b1*w?qzgI{jrCc(0Pn`0~_G(YXH-kI-=R4J&MERwd_aMZb=b=s`Ert6A@CcuYOz&x4n$>d&l{R>HOg9`aL0Pn!6)Z$3@yGWHSYO!_+;u810p}xS%CX(iS(x9O~Q=Qer1>H*a zKee%t(Xw949-bA!_2djP=md95d|de$Zk5HaaDP@Yej$N4D-sa!0O9sSq}FNg-Jx2a z=y@~?KrbdM16zX1hSaEob|)rqHa^yd#~Wn>UmaPynIZawb!-mwc|2gSn}o;3#V#_l zFGV%wS|Bo-j4k{6^~T5QES;#qeSE!(5@bo84kUjMj9~4GhkWPR8hgFRAB0bBHm~z# z&%a(_>n?ZMz4wn)zOBJbAVTIL1c5rk9acAd)uCj6@v5nbF0kdF&vb@;vMe4l1$YKn z;J$y3b}c^|I=x!^>cpz?ee+|efmOinhhgk|5r~Dn0(Dz)L#2n^cb?w7itaihyANl6 zCH}|`d1&*3gpSA&!jWw$v~JiFdHGroR|yZ~q3xBuPT=KK^37*6X@ZX#5y%Y(kqw^n z`!hF1DG3^9BU(go_NrbD|CUBEUjnEf84?9;A+`$UebMkAS&JTD{?R`ewQB^9z08id z!^%egX>F;grf4sXw>a7|QqZn~DyYM?T28wNf^meuekLUw>BLHRzHy~MXqAO5ge@?o z^VVZX?tG6Rlz?ZVW`s*{1)iIR;Tb)1QH>NMCKIlaF~2GE7t)0=5Ng%y^8p^3|JaF< zMbrey5w7tmLCVrN^L<{7VDmh$NzqK2=wk+Cdxe{K?I=WMvA~-t*zTWd>6<30xiP%W zfAZcNMzL0NK_CQ=;yZhO{W9d+;!9XZlN3Cbf|1T>Vi_D7l{C`=|1sKFuZajT#e#c&7({=3W`5ixI)RlL| z8UE@m-S2G_v#;#FBUigHx6CT(s%FglY!7T~)A%mX(avHDS`;y<f7lv*YB@#s*0T1fsQXQH=8uKZ!tY za9N76je6_!jq2Z#ANdN2;Qwec1Q*nYyV%EL71Dn^QUF#=4u?7p`S?e;`l54}_3+4= z9g*|p`2+?6a%%y7tDp059eDqYP43CXb$Pwnc4kN1)cnH>C3tQ)IQue4`g+T1=%L*U zDrDvzWbE*5##dnv+DT_bFDP7k9Q2FGF-x=)u1FDlYrG0U;1!%6dT!^|0OPl^ZJ%t6 zNLA{}fLJYr_KI=4SzBm6AkcqC;JHgdz9)9MfI~5#LQW^CRCSHE{q{s?LWIi8W1-e% zM~Xyv{87zOUEO%vgAfgVTlNbCM6_|4u!)PqrRiL+WN8S0mB`ZAa%a--&-%gbWuUKajp?`6VO<0Zf=<_^! z=gp9d;4He`<&ztz!SRfC-YgTWhGb+OT13jUSGlWOj>es!4(dlFr56(7)? z67YVT8B8vyyp~jMoud* zc>!7@P-^3&$c}r+;441Z?~~q}`Dp#2&6w-X)d$2C5WVS5_O6`pQZ?N`?Z}FXt+(jgziI0AR?SBkQi|~bphvj zxwyrbnat(MlR#F^YE(BsP5*eIEHX!K10UQ24q8|=Y$~gw@^R}R~athQ*i<*G|>m2f6nUs5D zHQgLfQFfZ4#Kp-wWdy=@|Hv1Dejb(QJp8C-O?uNZLq&)O$?!HFaxrSd)>(AI`g=b( z>`b+LwM1DB66~@u9d^deru*6N!yhST3RN`{r?6q$y_Dp+;d9`CRvK~o%!PZ&*!}JrH_M2nB8da0T=oFSYz`l+jVw!$Ot!Q2 z%u~rVvHjr!>CwarNwrpD{NYNiQHs69pW{LOJhL&u{a09_vlqH@Ak{GRT9_kVggKri z!h}#FM80RVMXz0YxJ^oq0+KNL22wlv(s|gMPX79m@IACT6%|;!T}RX2SEIj`@FA5e zl#f=_Q0}4mWxqP|%a}MdzJgP$hBb2dZ@fTy zj?F^^L3sHk5**9Fo0 z=(Sg|>>&YIegXz|^$n!mfWdJ<;cSf7m%v=vz-H?^!f^^_S?! zD{Kvoh3G#(J=eg9T638jl;K;e2Z8xGUo3*_eV=24FdxTMD7AZIaYB)x(S+<#LRO*= z-x}zkGNEh^yMP zec<`xm@F#degb9w({(wu3QO&QnroK+U0mFJf<(Of2QOeAsa*F4EdyYqo&s7GDnQ_n zw=_}?aERZEb~^go?AQ7yC1VMyCiR#Esc!9uOf00w^SsjSxgFo`W1W0^9+qbtT%N_7 zYkA3hF-zJy*>zgAXy_1(z%Lp^=%vE$Nq0E=ojFrKJn!_#>uV|=)T~tn<+)m%Z&?*H z@hS)q9H)a$4jHOqnKctPr47Iy8#JJ;dM0+D12lBa%s71b-NV_?8Dl*jBb^_1#L7Ej zPNKEd*ugKcuy({dA5yXHV`1CYwqO@yuL%}BYUCQ53mGWQ_33sbo4AZOl(_ean`?=1 z-Rklfjaw@6l208b(@8nv-IX9&bS5GOr58#wP7Lo@$4kPs3iIa6#9Q{OHVZuIVXZ*O z^-{M*mfLc~I!X4<8O2n3t%Ze-*lwq{!Q_L~ne{VjiZ#+X$exIvvQf6|d#@!Qbj9^$ zSWC)9D@u6`e|vwOS9y-|@vAEnXboKlaPm$0fxHi#wJ=z4_{yP)pHi!hA0Pq` zTXtXE4h-?mNyp9Y*5wZ>QiKZLZ2`YqtA@d?()u@K4?4q4ktN&@VsefzC0<6_zrk=s zkeA|7A9_u!$qhQJNnqdJZ^EPPs2rz8OkVnqAo^2%WBDgeg-sB`sMjlAdM9$hOBc&0 zg~=jEf`;iL>)R=?={oMq_JVVA7>KRg4L0~&F}S5J6G>^DYUzz+~DkN#}_+-5;NClNS@GItVL^FYGlTk zRrdB%f5oS&rcaxV`)WPEHHUa(@{OA;-?c3p)L9E$yE+Z2XC-pOae=l3Z5{CUuJY|` zrm;5t>N}EAQQcp%;Lt~wQ@0kB^Ci0dNA_Q8uA5ZwB`laIo2Xx@e$wWi|0W$luyQ>P zsV`0SpVHrtq=`X=(!I}aD0Tt`k>#Nxi88-onRv6G9ZA9*&$#viOK>WcXI}{G(z>e6 z7}Iij`BwYv2-HcWX#=xvTT&jK;e)Y%P(_ubMN}ztdYCd!VslDVyc%YSms7$I3nCwX z!Ulf-DksAo`|#HHD+^3-t?GSS>2deSuT_ihR#SLq{?s`OxNV2Am=#isTT;~$)3np5 zH$_-~?wr-??h=E4PQ8{h#gm&6^3oD5E<5$-F><4FbT0f`pQ$U8MnD9CB7 z*z@8dDuG|&B=R?d1O|@ircs?mR{otzQuzK=m!;C-^|2RL`lW06GIM7f^#%zDkwU4m zO-d_*q8Usq-lo3}!_60SF=y0WU#3^~=UMBxF*H6G6ge{S~RJe%{HZD8|uyn&aTzO5jkXH1Tjod3a=niW*W z+$bV)6KM$~_-V`;+r6)bd8ewYD|g{}qi@W06FMUMj=)I`U1ZBKMD%`^c3u)OQMw>_ z`@Ymd{^Xd;&Ak;)j){#1^Qcr+6je&k{R`9Z!DKM%0(QHaTmtJ@X(Mta2)7k^uHR9B z8Y|%xGIDvjvIG~=If^RVuEz;A-ADXw&IyLoHyd6B2~Du_EG@NH?1r12@zfv6mVMq7 zB3}7|EEx-ns_J1yw^mJ%Cg*VGdH*(Q+w!yMdD*R#;cUly8)`pWYaf^|v&|uX1hWk= z>}$CQc#VyxO>FUtIv&~-t8s;A*)?_LXI-Civ_sunW5vrrHyU%Ko;%{%?XE`X3e=g#uZ4pNlr;h z237u6jmN{>psN28m1Bg2F=no7o^WQgNq3j9Z?=7=XROm+XR0UGj+)$(P=RNud_ksDOKnEg{O`dlVMTe)sf+Q=wOPm&IBNKVI(`sdo7oP) z+dZ947IRPUazxE{_}ooSdUxdbYTM|HK22`xc>`q8$yinRamW|j7e7)}$^Li5a0WQn7v%>}V{#U)tC<6>J9!t&4Z7j&zK zrA}Ggx6EwMl^`0WnJu$7*COsS2Neg&zuj>hZ_|)de;Ble$#bIxy$rU3$%CSOicsF8 zPZI_M#Tt&TwMSiXAuH>)=SD`}X9pr-JlDu2t1&}bq@reu3b%O$nN@^+%~Cl)NDHde zctbucQ?-HLlKzXO(Y3dC^14d-4U{ml_EYDGMzY>2$`WE8r_AOdrN1P@cR{Hn3GB0j zccWdJH#u*(aA`0z;1O(6Vw5mo~P)bWKW>7y*H_?Vb2+ zdBd#L$JM4HoUa0PpSa~05FZEh?577o&sdSW?qG4d^hotPD`(#+`kcK{;@FXjfo#_S_yHKMkz+T;&c6)Y)gqhM4mB6-7J6J{=B zyuQ0%A|I|D-5vz4_oSVMGDv;b7Z~KSbK+MAnr%lsB@Q0#( z@@ELIb0x;%&9S0McPDnZGQ23TdIP$@w zkD!!!CXSZaTUYkZ|$hD|<(P#{o%mnsl$8~(?F&4dK2-fU24<3dyz)C(xtXum$s5|(^m~2sD*|>99<#a2do1^wRZ`1R=zVEm3P-UBq zHx*z~;9@F9cpjFY!Uju_Xyu%s%nU1Y4`U8G`#mHLe>53P0OnVs)*?wReU zrM@Q}(BX|A0YaSYjHhN>DZ@sUN{q(}#G&_$j8Y4yjL|e@Ok{4GuPtTeEM)1MDp_vu zY~LJ`*165_qz~t9?lwU08$7)3|MH{>39s$}q0L$`H566O10mB|pMaXXM9@a7qtVcv zVx%7t_XGvF4Pm&R72xN1P})ofpYLqZgc&nhaf^Cfwn9sLeT_-*aE!QFT$&tTZTU9d(87?%oWW%Z>X%)05@ExmN+0 z<@=;v#xFc`TIPQA+u7~io!5-7y4qQ0IbedyOt16nXe+zq!ZV6*uAI`2IyoOJZ7TGR zpd(X~vUqJ6UN`U%vmnw!<~iwKxlvpBznZM{UgV`&vqxVDm3^KyVF$TM|G|6*~w9jGG`x#8*R+eIAT)iipn_4K`^y4Qra92%@AJgKk{zsH#8h11mI0wzL8l4* zI@uN)P1r11d#Q$POHcY|aquzI5@$c&FHYPa$8-xA0SiU?f@3hn0Ikj@wR9TJx{)wT266{HNRw3H*bp>n)t2%t{<>x4`bvMSzrsVxj0u~|WO!Ps zub#;MD0ZPM48o(nb>Q4a04luVaJebk%$Yw)6ms;#i7DG559jw7^K#d{)_eekAw5vBHfb&N!GQ7I zyk!So*hn^Ih3|R2sghGyQe|@q8YqI_$SLj*kEf9pPcF}%Z9LhfVM(2?eL2BcuW?$B z3uSFZ`hd=S74(WlbGd3{pR^b;XX(sae0p6*tg~lk2uC*I?i2bqG=Sz2>kH-wB0w11 z|4ld8RSB{$qyT@t%rMwxzAb$j;kW#T0C({R&JTPvP_n=+RgQ|K3!k&bGBMAcpm)k6 z`AE>#<<|VW$eT6tXOpM_kn3&B$5IpgaAwo9Eqn1eV}-RW8U}U5wQx^|{T59395gH0 zVn>sfU`?x<9=9iUcdNFrEhV>MEOu#f3P?G%XYxMe-vDB&iD!798x|c%b zcO<)YKXqABc()lAzEFklS`tfHMpFps6s5Yn)hX9ney7CQ(iUz(vkino9NoPMEP-MH z`!7hm}>?BuviD}R~*Vpxv_;>|kE6ESb{sCl-!P!C!+shVe z{RlI60R$%;kxMv?6Ma;|eMmK%QO*X#Wj0QNWqWW5lHj+_kJdb)JBTz=m)K>`+=q69{WQT z1w2a-+=dAR0LmUi6w zxQ7QoalDpDQr$|m%MNQeqKx4$9^60WaM&DL@&g*?Xy>iA4i1yg#`AMGrv;LMI7T4# z3E|D|eQ>IVszT++CiBIA4Ee1}FzV-(Q6@dHyWoWKCD&lx^SI0`W@r*CchyLIc2QX= zmCI>9;>+~IF_PLZl84K}Cs)tw3+^M}6VK)Is+Q&FLe)j0A~zrjh$bXSM0+RERmME~~BsnLE zOO~9ul9n7?WC;sA?su!+t@pmE>Y888^qHROnLp-q&vc7CEQDp~Izp3YsBm}VZbH#f zKvHq(PSf{m?)6YzDW;bu<|DG+A@VcC-ID4DCSh@s?lLE+>rH*;AV*dPt?RRwo|tv@ zs9b?AYSMjs(eXC5J=6{a`AQ46;NWu&xqP}LvoG?j4CJVjwMk#p2*h>2X2vyM%j0n^ z#kq|K?`j(r#M}_kpxh;&iK7w@2AB(XXLXX~Q*kXD+y3NPi@YV<9|>EvyBQT%t@%V< zW)J}j#^t#+1lXkS6lv#OW*R+RHjH+NN3d0PpHa;H67r)%Hm53#jA?q2V{i;614RvQ zQjVS2m#D$ZiMwz>yIH^T62jGdl*TVJ>#@Dz#5oT84dOf)ntH?8RCjZq6Sr_V*eU?y z1e`RlZu5hV#A*+d;uFNF!oEm7D^I-v@ zD}CQ?Q5q$zyp^zNtKGdC-%2mtQAnOu14iNl&**6#jKzwFZGIWQ$V6?Zm0k0a14ps5 zOxy6>T-bpHOK#*_ksjDgQN$otb_7E%6JPX(&4ES@`-``+iuvMTGkY^JPxrZaw5Icx zOK6QkG{Ds$V)Nm7qzb}2e4H1ot1Z|lCA<%B>oVAUj6+1FIm2dc?1=Y9aWV4}-sT=4 zpXGBQev!`An-{TPWRUpBjBSeYWpffn7P|#ko@_SATr=OaEydVwC;8^Oo8YM1t|h#+ z_cKHCY|}VOpwG7~4ORRn3fC70X?L^>bcq&>0K{5b+Q^(L`EmV1!uPFtNU!KGj&LoI zk%F8y%a2WFNa>2|thuuk< zZNR%&Po=4EKm0;qX%i3Hqw6&eZejt{rF@`N=Kk5B*oD<{7oD|$yG5p->PNzl)11$K zO#Fnoa0q-Sv}$qinG)33%+@P#F)gz@Di!H*D_YgoWM-UuiB)x zfP;=IZu8ol$u_2lD|yo!D6rpJ3S}tX4=j&Q#PL~uEtcM{DaXFG0u2_Vo4hPaIMx|m zajOUrmy#wa6_8;tE)?_0lwrmGCfI}LopjjM^zcs^xFqz**bkuzIl7lGY-*Kuqe_;|JgGq;S2;`h7+bcI z7saGR{fo}Ue}==`9Dr{m@*89ol0IZcg$O;{Ei#XwmjV^9cblLyy1bH!*)i^9HhX0M?!s+3RK5Ig)q7*TVHFCC=F+tmPD{v?-`U zGr2jFPjXZ(sY$uk9H3al!5a0Cm4e9JnsV3k;43jAdJ9U1Dgd~Cf>2C~6)(?{Q2iL1 zI~XH^@a^ZElNH$#M9sv%4o=}rM?2EhP?xTP zUt%={rR-M>Z2PIQeAUvxKKsHI4^>XQaY8+{@tAm)$B|OBk=yd@7?hpsir2*n>9+<} zY~4^kApG$YxQ@CiALKs2-Ml8Ce@%nU0VK$q!~6_Z&kga&kjo}QOWlhL89ZQ7t?U=Io8HK>C-pP|3wZ-P(ysnnLmUKac4VA6{=MheNL26k-jzQ}@)oWAZ z7_mG+8+Y-+pQElQI)rK!)IJZvVs@NHtGsa&Sy&J&5Ab}TAq!3-c5;Q{BrFr-Pe=E z>r`j^hpT~lS?O(u-WavVNAuFVQ^KgwcsrOy8^hhOvM!Lp(cY(sd1hh-*O4wgh7U&$FFeK3Wx4gNxihz2?MZBa zZ$u<1KZO$!@B{>A0NaUzxcLOjKKQA=cT9oh86FOib#b>CPVS6B9#t51<7M9@7+tzO z=Rw2$zG%V2U!Q42`IY)iar2nfSMCsi1)fZA9Ye{)>e|mx@kY#LKihOdP z$}X218~cj<@UtM!qali3@K1c@cNeO9><}Sz7`!YrKnL;OF-Q~~-k!435d?-;bmFnw zX;t;<=2LHn$2_+=9numat14Y|YWFKEU8rjLxrBD~>%h}O1NsHvDWPKhiaUw$gSDRMmNvUtOlEu>C%s+~8rMV(?jsRa$q6A{F1T{>HF=;>jh4 zRQaDP%8(z1GF@sVgIeCOP2d#C^%ByjWbZFVO60OkwamJb1e!w$O%~=rPww7FQwM{- zOXjn|0viepdP#(l2d^HO@URm*CtVY4&8K?Md7guc2eB!^J7+5ONT<)a%6wuVSo=KyLj|w&7oh3W#h~ifKSRCm`%>dxmf-PRrqx3``73A;VDPdrgIiM?n*3g(#9p zUk-6Z5{p3$jTXY&9kIs(yE;HRAFyN?P`ddSXqrN5TPk9ql|jr_{IQLjYOM|;xY2NVP~|{ zxY6_*o8BIm;89RC6BxS1JGJjt((Q4qQY!7U@WvH14S(t95uZI!r>Qa6i)r*~*{yAJ zcltPUZPiB!d(`+eeN6E~7*tVASG*Vy zSglvvdfFH~;JvOPsMG$&t zj2)b!G$DBCX`(8Udi|4cz;TAd3j=gjgs)qzuUS7uA`7R|%!8V_q;FPJBOODykho3B zqPdlC)fS<0eu2+EuS*8jo}s;Ij?<(86k{5H7M24n!#P_Hm$6iBt#A}KhxVq1Z(c9TM{^V2AxF9Icd$=Mw6?6IebYfq zp-0f6-m{8l5=tU(*?QrbAj(peXvBx13f|WIEb}cj+t|s%AFk7O@#T^B!{FIGAZR2WFdcj4tB- z+s3fg2R_tWQjQ4k#0>j`Q^E5-k%$UREm|x!$J~^3aEFF!J2ya z0#!y>t8n#RY62-}L9GlRO{E>TwdPKkGW1m4BD#gM zoASv9r_=Vg1!J#-!3XJ?llJ9ZlzWd+Aa~=!DW`;WXGZJ_Pv!2-J=abpyH$uz(JaV4 zgQM`5q1X}Q(E+s_d+N|LM%*7hg(dlMR4&`TA>AE)GiLK{W2vV(0{hEUMtG`&MD)Gu zx}1ob-V+P+LkA$IK9d$!%N@X_0bdJQW~a4DA{XPcmJvPWjQm(i&lLTG0Icl!7qwE_ zX#ayoltJVp8YMTH6`K=0=u4jIqiEEy-nu8tn?Yi=M>+f0e8?t)(g!uJ+w?1V-vFs8 zu5B1`lSfnx+%Nwy*yKADO2iKepgts6@_pPaD1FAICJ4;FMYAQV6j=0xHk`J-hTKsa zjq##Z8Xet2Q%uMhnUHR$x`mHaIAFS3C+@FmH4R!z39n`lAmWan$u!0dP#JfR87t3t zeDYR(QpQr!7?|!5CZGPWWQY^0$_~vfbqwU|WiO~Yte&h`c$l7nao%6Nz1}EQzOJ@2 zupmG1=OMt^+c;-tR1@toQvf9{WOe8^=vyE?+BRt7ym997)$_ zlGw@#2cN%vKZ1SV@%oIqbD_)SnzcmUeM=x9d;LOp5Bqy|^nD-r{Ep zpSjk)Gw@281#28P)Oj6ZR{G%{msy{ zVe~c#=qyXKw`O$iRy)LETlhlD&RspHlav8Yzz zOlN+y*h;?KL2Kc!t@)RB$cPoA_VbGNrWT@&pYo04{YAfg{VlZ$+%0A&qU?R{(m5Y7 z*sAE{!kE#L_=~#Lxvm}7D}wzagOZ}LTsLujpx?Ksc|tPod3@)`Vhf_5NFw{6V;U+= z7GF6iRZQ7vh!$h;$ywWC-G&vpcy|hE4G1~fP?ER&k>x7APr7myGiiFp2Zbjlf6USE z`LZbE`@<(VcNciChei+|f|G|(C2V6h-XqJL*h>bzo1hlZy?8ur<>ZO1@dh8do0=XB zjM~38lu60Bu#NVvq0jGwLKo$l_aB1EhT&}dLkFrhG;TYD>pDHu`K{90qz$wwf6QUd zDMu3`^jJYZ?Zmi7XpGdWEOl}hY$MDX+}7xXyOx#KhLJ)`13t*!@unKD=(&RD#WD=0 zExvaK8>=#YZl^?L>H%C~)&nqZ^sPal#e*XvS*J)GLTAIPS=>8YOkbFed+Eqw0bmzFGQ~SfueCEyW@#i|MVY#1 zCkavJYc|1$)4`;t8rI4w=50W2KZ#(1vDNmAi;`=iKCBD_$2gIhH)8K zdsV`68I>)7R~P%OeOx&D(K4#VvxHRhi)OD-&a=AgTS^1%<>WFEt-pu!okVjR{7L30 zD;Fu^Xl1nyWPnUjE_us>>UYwEuf}FuYSv|@POG#>KIHs)DdBx-etbb(XW?g6D(oeHb2J?UB^_`u)>+)mv4WJ@9(9yBZ_Y1V8R+v{ipKH>s?S zwHM9LEPOYg=6}*En-I$DB;`?;adCQZI220!PJnKbN*F2U8@@R4jZeAS$;H0Y|Gvu0 zUNhbinV_!x-!sqNIE5Q@X{CGZ{#BQ0WE2J*T@HY&r`uYZPJaex(%fr{=7AONBWvWY z1KWW1-I2zFiz#=;-wVGKVv7~WjiM}Lr%)OVa3FQ6<(rCw)mhF{cH1tO{#(|V(%K+r zCa|4q`9)RUQoXm^+&t=_TSVfRj9Xvb+#OD1Y>J@^Jqz0a;bFgvvh=Emd^?$XTeO z6~b*05NKMuyW2QI-2VRuEa_qdjyD3@Z|Lp~{aX$&2<~bs(BBiP>@F>Y-t>gr(EWco zZcyeX0fCg0qobv{yN#3Me~pg#Tun=8u!Tsk{_y;N_sJ~+0@)jS^nV7j+DF{0d3gtW I@OSI~09DXMBLDyZ delta 42241 zcmZ6ybwHHS(>83=ARt}R4bsv`H_{DCH`2M>DoFRzDc#+TN-ZhfpfpRz5{tll^E|)z zeV*_8=j{G<&V6R)%*-{{%-7AQweTm{@0FfDd-dqiBh*KS6$p(uY?)^dzcsoNk8vLU zI0+vFo_v4ws9gvfTKJe7IB;8(Z9I$^D}vwQlI#yxgShe(?rqzuz=2 z&Q@#p<2(&O=)fnlPcAiXMG!227v>7p=Xy-v0c z{3r~GHq|HriJs!h_68PO6!$x{b$yCDeU6q(JDfU%5l0=D<9;R4cPh7SJoLmqok$W& z^4rh$?YdpbD^WrkivNo5qCd^3%=g2aYlMDD+=@%WAPtxDw+QPD&495EYxv~*<;;J> z9rr{xcO;7!L;gU~D97xTiSb9<$}G=<79@MN^ReORtKwB!#4L|T$H_`!UyE+5ju$eb zb+8J4WUh-i@hh!lp^p`Y+Q(?Ki)dyS(pMrCOKu4H?urAX|JLt~d`iYQQaf}GU&UYA zG=gNMWbg0YmR8m-mF(WH1T>Xw;=tQ3;g`QgMXR(hC;)F$Khwj^1G7b)0xS{Tey73v z%n6l+IYW;nb9f@ZSJ!EPPJdr1R%f5_>RM}!kgwa4U&ChL!bMTSx}~Y{jNcI}!jRYh z_%3JEJH(t)yM(gC*+tFk`Dhb=!^so1a1rWDLjwN(14@KAvt9Wo!iqIkkF3?{B$R?B zft3PvIAGwI$Q@yD-`x}=TNbr)bP?^BcafIQbl>cj)s^Rfqu5VWVE?`;%uxGqvBO$; zFO0IRkwMM57vAX}4q()oy zdw-2HwM8r6F8?q_xl!*r(Y->QRVEKz8%`X!7_i1jKktZWHI^wosQOfx-K0xgLMyj3 z4M7-Yd6|~LrM{K4o(YP{DBf_%1`?0mXxti>9QEyZ%)l4-%aThD@`1@xC51c8hU||P zgBl$r24k(Al;S@#_6GGBA>CENqQ577$Exm^`Onowp$a5a$Nm8kXPocX_32?wN>$kz zfcb}o6Ji8;skL^@4YhlUuP*#d3xYmXgq9VW3SX7A>)N(UFQ)Gw#_qd4pK$O!NDmZ- zCS6miou0TYu~^h2nsz&CZ)S_+fU4+L#9h?w5Y??f!)iZzzhhtE2!8okbzkv` z-XHSK8zsP=dgo5VA=g@Q9d@zZynq0*^36Bj2Hc`VI^gWc0SX-}@(taMG6s=i$tNx*8y(3fwAkXgl8ED&R*!7a6?cxRywNf7q-q1M1x>7x zENcBLx0HJ=$UQ`sm<;dq_MM|WwbSU`NaN~F%t1_NKSsk*?EhGuu888Hrx0` zCvT%ibn?}Ntj!A>li;zW>4l z&S!2oJsknv}yFY0CC9J+E~dT}5a68P0G2C((!Tx?+CC0ZXgq zH}d-hjyF%oO=E%na#f?V_>%6T*kAu5Ueny!PF~MMr3B3mcSmS+dqLL6b_^}Fr}daQ zi}do0Kf4|eSfS!<6JIt1KW`3EkGdcY^sZ^{Sq7a_>)Gk?mPHVkg7$ zLUIA4$?oCeVRV#+J#M5P?xPz})2EtpcHfOpOY9mD?@ly}ve!KSGXEZFK`-XS^ux)#>P>{t!Xh=21a6)Id$e3kca8UtAGqT7kwcIn) zS$uUi74j$L@mBNo@o^}lHP#H;)EH*Ae}g49=a|s}iJ#Q4g_oaB`ppy%Eh3|%%39TQ zY;@kZfKrzYQzPaE@R%}%qfyj!E#_p}H7*G5DUT{lHI5153E$cDDeg8HuVs{4+NZl& zSfw2)Fkn-OWj$`Tw0mofU5PuI>b&kHbGa_<-17#NfSuhpK0pV=3#+G zc68UMHjhc&EE4C`34Zm)A--vcIT%_L12u^&r<rrxJ9f9F$HffEi5z-u^|9K$#t$0cf#{cGJV z{KlhOX6H7bW~8-}{~O@6C^ zz+n58G>`OwB6cJFj4f%m7cF7kM%Ijru#l;TF_7C5qT=4ZnMFJHhNZ|}Em|}S4P&d3 zZftIYM3*h((0fbv;5Gy~#F5vW>(>@pdt)^rs#qf`Tz!y|?mIIj$f_!)Fy#qVGa*&# zBvG~HKa$Waa0D(EP{mT^GZm_G$kIvdBt>ZdVg7-|d5t|S&to~8InQV!zT{$l^)X&e z1VAo|onq&?osM>s*r)qauktQ)8`ZqsDROCYLc+6Cy(@=t_p?5i>e)exALFTmcc#Hx z2F@%-tVC(rIG{t**^3uC{YIbGYm+Hey(Wk^A*P6lzcTt&Kfne z*n(OX74M(6vnnrfFx|HN&z_D-7uHV!U<5wfc(C=LLH*z&MM;nH2C-sQBxoUO;%F83 z#wEl#m(4Pqe_z~8F`M4Z7turOamXB>G?GwbAm(}whCfb*;m99KY<0tuaJQPVxV9a? zT~E?wGa&gNv^Y9EN%wuSnpv@M)ghs08LSRwTyj& z-}Iwv_HI z+K>JzM&gMaakcRiW@ljp@XkJ74wUNY6?$Bp28PnowVRDP@4b!yrbbodHE*r-))pGv z^_w>dmHo_)!&s*Pw_ESlak=yy%^8evWul#6#=aUV3%^v&kGp_pUXR98O~9DvFmChl zb*GLvGnGe_(Qkt0I5MYdN|E1%1nJcm#fn5Mca?^nlxv@NeT4woTSG9yL`Tdesf!RSPZGx$jRR*ml zG`(H)TpJ1Fshq=vWK3>FBOzInTZ8RCJ@pGn{7P|}bY=MD z*Y+*&)gT%3@w}g|hmoY(ke^h^V@8)d@SxPO#dbM-B~2d4q_;wi)k@nbkY_oV7UoHf z(*sn0nd~suH0K5YVV2ObsXOH^&#;0@nQ4?w7z=naeR_R!Sg+AzZKY%?T6{lE3s(H9 zfmfJ#&a+8jBfdPiSLSARM)#qNmf?+mph~3hxB+@_XHg~}zn!x4IJu=7G~@v7VezUp zrHf^!u`a94Lgw~n)=GXXsF_zZgh+tOChxFXu7!IOH{`e%wK2PVntKH)zQw=-*n~r81W3D?*TT(Xzf&cygeKiv8pJZRZ}}(&2!1#^WlvZ()il2S@hx(%Ir|KVx5FaaYENH^lEcIUu;LYRUY&XM4WFb z#fWd-ZNq#3ROz(gG~X@)%mjA|FX;?%(Rdy00w!CM}B_;yqFfPGJ6j{cRd zyK}OZXPtGcEX(Q4c$AP$vQ}DM8S*_!eu}gs5!2Sw{oym2h!}N$Y}( z=+YKM$rTDakcGg}H|h1psBsFI5oM&uP2M2u+4<1w<=j1n$LKF}O3fw{Z_a7GtEh#_ zs(hZDVJsfVf#`=}|70P#gJUDk@t*j%5f=zQ-F;19Ae# zs$cqj=%S_`3XY(hN)8N4AZ1xzL@gEm^ZT)e`!ZKnPtc({xpka74$iF75?I3-?y2vH z%>l%y;FfgQ82ck;HU*~!B>$;_U&Dw`;D5iLNr%N(`E_SSJd2LyN6K4@FMc$hStZx; z-F)X&U{_y!uCZw0+4b(8v*X(De1xi_tAxG3BFw=P>hiZ^!fXMxVVZNBE1K2p+cs|c z&TA6JBzRNwH9GD#-(sJC@AtR*f=tJN`>2zx0={-FUvAe%8%@CgW8)la=TG(2q@(nU zuA1V4e?TyBOtR%kcc*@w-&&RJV8#JKQTWCc*Q8&;9u zZ%gXbp?12KdG*c#gUyR-Q<|LS$A0yFxp^Q!iDWHkJzS=NK6Rg5?U?tu^u!QDoj&3^ zmDox3`iZu-hbG#w5_41$D<~Oi7Iph{{V6)vq;qjV$hUuDtyW$>2f79QhCAxR!But< zI*!`r<~|`d+M1Cc@T*|+9zd_-Z`XF{Z$}sV``gEARAV8PF;LWVuPqKLm9mwLb?l4H z7`q{>`@PIsW2UkVNoMz|g-v_SblZx{$*Bt6YOnLiQ=(?LU;V*l=@e0koPW_^s)$#1 zj!~2MY^F)4`AmRj$Iq(fOSxZ#BbTUE)1tT7QQ@w>CYhb@g#xPj4gfNbm7^(Q*GzSB zeaNWw^GaZU83q*dOhEuVbD7OY_j?4xGg%wINpiN+e>lxh9?|>$Li^_})vh+Kw$}Q6 ztT{|TM5Msc3F{(hA2oLzPr9li zazL*=U6~q(NptbvB~U+FbakMsC)G%nUT(0+vWVjBaHw=%!sT0U!OP<8rCUNk)xa%h z`sRxxbp!Wuoo(y!6Nh^5D8Jj!aqESz@vSRXwA?+?E47uJBSH)(r&M$iC24D%F*&ew zc2kSpSaF(QJtan2;S96$vv-fo*G2VG`Me2l9-ex0IW|#emP+x*Q=NS}mLXqE&ik^p?+zZg23!!UPw-)PH3~_=M|x z+EnRgv2*n9@%_rK`XG6SPrBAcv-9}T z^1->0YcS#>N&(Q#;`J$QO}o{*3{xC@7h}NJv0dH!1Zvu97?XxGf@p~7Om~~k^Blbk zB*`LiL=1m88dhH1J*=>UHf}vVG``g57~v|>scp1QlrL}Gz)dRN7tAuFauzzrjL-4W z#7n!re>xXt27wdu@;j@B4~PFL#~WqYy(IQdGdDH0f&!li!`kZP-}?0gt1Wk)9*mNf zn8jt*4Q`8w=NAuNpl#W%@Fp`A`tB|;kgU#i;j8IL#$BJdA#8)2k~QCqKH)JuPh&o{ z4NmI#>e$$w{wW?)fx%QDet5TQgm(N@MUD^))MlxI3@?g(vu(6Rsg!!US%U8-wa17i z&5qta-3qXN?fRhV`kpwd{x2shQ$E8QNht^dqrWwH~zK zdNHP&rl6b3yY93cYvsVaYI&g#9@+OiUU0Z|6K_b1-EfhYr`M>K*($E0n>D9MH*fXI zN^+PhJKd;W=#W1M(yY@(RIL=z;}>fs&y!CAOhk$a+cFqiPDHkONTt;j_&3cPz<4%D zh`hI+*P`uksr%X{dl%ZxFKnm2^SZP`Bu3wW-IZ9nwqrp>O7ttsL z(?KiJ$>_`%d8Cr4hb&5zlR&I8z}=W|M03~GuI)Lh2(>c$o=UV6NcA1qpEK~$9-0aG z*R{czm}*8AFz)h3=+1O2QRPj)a;}{_5a88ODHGW=Vm~ZEM!@nM++8Z38Ww9I&t0S~ zKu2OY*mRW0>PG%XY{!*Q=nJ91_w`N?;FH`P(_(}orD_4;#hc>D+E)gE_Z2rKej{Oi#X1{2f zO|(a85>JeVPN*Z=%j?|j=-p;Og_-(Friw*O;5vmUG$Zcf_1+#lFT9*HOO5%m+ArJ! z^L1XVdTDR3GCQ|o_uKo-I;i%9_F=Sk566wvt<0TGjJW42OT2^wt~_7aZUzA(&^$=R z8C9@?8B)LQQewE!cnfgU2>fK>s-0+wA*4O&gm3&(8_J(aEPl9s?CobvGb}et2%tegTu79Y+$`h#=fxt3 z3l}YTr0+KAU6WP1c`!COCVkS(EyVF_!x~pPL}2y|E1#scwJ5CP;q zT^hJM@8Jx971jrCkEi01zMs*+FxK47^MD2kO+QXWVH78ha4A*8p4|FF;z2a%-xWTL zKFRaTis8@_v+pHRkJ{H1KddU6ta#P_%D5-mkeX_q_<~|L@I0=PY&X!pcRxGxReYtZ z*Q--LHjG=X)i_RrjfkPl@$8^+_KyioiKF`FK>RlhixUExmom5XhgX8UOlf7MWPQcD zegEDjHo$Dk&DCBUJ6~B5&d&AxQCd1Cea0P9LHu!*z~HO5bh!S0KGuY+scm4wyTJjC z)P#>x9vZ16vALFSn3HFr*wru%y7aFY{E2c@TLff)) z=>e>2`t!nI3z|17p#oiN0{Uiq#d*h|v|-Ca1T4 zLrJH`H#(*IcH8A5Xamom2stE=cpIhVXZg}Qq`MRTN#L>*m$-NnCLX5ZKZu8p^=2!o zTf{s=4%H^e!`N)v65SI8XNQB~8TMAILQWwvX*JoS`d%7g7+Qdg6mle1bDrBDJr+2fQQK{GK|FDwaWbLo|;>2Z5+1Dyd2a_@M=Tbi&~ zX>NDgur~`b*_qGcE4k5ng5h*8Z<0Ot@DzkyXTXg@D;SuNE@+65}6Ey@JOq6(X=J%C4fIk3wbmAi z|B&TihJ~3H>%5Um5H=T+f2H3V$0Yyi#+p$b6nAcQj!@Xo&Vt1x041+E(R~yiPiyoi zl+^}O=5S<6P7_vJFlZjeocpFQWj>!hm*DhYw93Ecn6$`h#{;ZxT-JtFbY3oLW8}GN zEIqdetx_yK7?+adbyL|u6RZt_*f&umF(^iEn#Uq>xRhGcEa^7xbdRDljttDC-n@~3cHQJFXo~k@~_G`PpgGxET2C!>poXEc~y;oEagh7 zz>6bbfDsQ9jX_BOT&=+4Qrr{UXj6s)xtsxaA4}iI*Ef_mK+dR-_pmv92IHuJ6u(P- z5s}#U9^$08Yk&0Ee7kL`#WG~uH;nM2PVj=oJZLF{Ij5$pcy+hn*8hlYqmh5B`e0Ah zb1!Lma4~DUoXd9Pxj|cy6-T_meEfV4z^h-!#cM(Mv#4I0PIZ5m0N!7H^YqOEX|}~k z!b<>#WHjHvU@atKj!KYKq%DY>sMTTs*9B+Gd|m<6+=@h|p}EZMa!$07=aOwYEUTr~ z%QCX8ru@PQjj}U=b7#1Luf9xJP^6GBscc=3{IKd=ApCKWJP*X_Q&{ERg_#15ir}`E za9KrsMWsp=@P9#>GPpLZ4$9j)y_Ys5DI3xUKl_bfg-}re^AI3RZ+UGD|AD^Zie z;AvLm3$9l{T}RgYK~$3L*=m;Ufy_6(cDxH?ypqx1p1TvQYDOx5azordiBpksMl;n? zKb0S(MOX}0=+0Q@iJr7 zaSrjtO4|Wq@N&Z#(V^10zF>1qdU^GjeZTPgK|&&=(I#oMa!S$J4@~v7RCLcu!c9ws z1IiACeMV~!@8%Y3A6<&bZc*yX`9xiB_?DmGG75K#b&`agFE+#x{S_AUj6b`}Neu`v zTk*6oEf{Ii{{f6QvDI>Woc7fRbh&`2|Kptrw;8g!w>`i$`mt61@CVk=B z15hBQrhcPay#8h3vSndi31QfTDri;Ms0FA&BGfH(b|8BPUpC9ea`qtFJ3B{Q3;+Cz z?wQ=qSEmA~SjjKheOuZx0~ANAR}2=6FAQW>i53EKfq2(rQ5TO6M%g3V{-dydc+=D% z&pCF{ZC3NLm(r@kjX0TeqLsImzX&^zru9Pq_YQ?#JJs1N$kFZ{5|+Nw8WFdfa#U{s zi@sKiIJdo44-xYXrOq-W+11s9+QE0VowoI$EI-GHvn3m3qV>D6jn)EYzn#O*68cO| zwEcDg)=gqP4@@?LRT;jcF=BaB+i}v;JXf#I31jqs?-~x9{@@)r`bS@V!^s-;y99x> zMN(p2!;@v5PUBTGf}#BkZ85KVvzh_CibhDT52*|sZ(57fc+n!ytw=h{WY;e0a>m@d zwRFGY~c`a7`o4d3}ouoCbbd8Rm+bGgngxofos6HfpiS~?;;K3CU zk>umk>NK^XhUog zTujkulUb?o*hAG_DlgJM@-G-b)Y7O&EcRs6;us@&rP5#;|Ffr~EXFCO z&%>T$Z`5J03&TyYEBcZit?A3Ve!3()fYYkg4AGT5HYYS zOHrw;P$F3@(n|TX8XLo8 z&CKa$fN0FecsHC0wcI3X5CqSYanxVsH7Zsa?A_vA`x-D`(pVdLUcuHlaQ&a;^1pLS zKik1Yi|ulr{}z#v6ubKEi;jlKuA}knwqAf=C86#X(}uR(hAiD-M3P%c&+@7C;sB{S zHyOLuP1GAQ$BNGBG9T3E3R{m(ZeraA1m{~@ehFU82pq!%H!_qHVS-?eEC}Yvj&ie8 z4P&GFZrmU>r8ErP*kf%PH6%O{IaHT%8`f;dxIP=!ss2R&M0PUV+uEwSovv=47}#5q zJ%$bA>gU=QBZbH!-b*%z;xRn|Qm$@S5 ztO4&M7&W`=z~Q1yd?-6!i4|XF-P#6NRu&N4D5#lB{~PBi^2S4Hv5|^_`|g$gN8_q0 zv0z;*e0F?uLknWrX1S_blC-4}fcu7xLz?c|BCm^nHs3VcNQw&6r*JpvZ+P!Jss~6) z6$v6|#%vbC1wmG|CyHR@S0eYu8SX?ig1S$Mm}337n9tLrTt_eRmdD~PG*KXVk%^RQ z7{7Q1_s7=@mn>o(1nLckgftyRylSh{VXW$9kW>s78l#~lUNpG-&jLi*&9DN>FS1s_N|Ipg(m6DOu&yNQ zQE*9~-#=59B&CkZ-;*WH$p*8}|AjDk|1ZK&{|~~50})32Lp8n@q?f|}AH9^UEfA?* z73ot??FQ67=~=;$8Sv#?n2yQWcfEl3le!qZF zUW?~P?Nd$kwUn$lFEE+sWwLlo)ucUeOLXAl2@$c}P#P9e=HBc4db``_UkOr6cZzTD zpfqf>24Bf4=mR)2A+a&cJ52DCC=ctdJc^$rKdcoVWAbxM6b2NZwt zgcU`5JTOVZp!#j)u8aXLOYUsQ)l~l^-B@ww(}#o6XtK6s*!Fp|V$;O3+D`B1#hrn}RY0 z=nuJmEr;2ce^KD3P6ZXu{WpUf;u)_1S{OcADO7o>24usbtx1|x35-y1Z|isqEE*#7 z%d#Yn81Wu5Zgsu}$^E;YN&9NHNNeeB2Fes)qf#}VE9ss}z<_2AsJwWIO%23~Pk;&# zZ{3b2X$f44E8%W=03X<1<*SXl@f#xS6S7I5e5BY;#DB{!e~fgs95h{4r&%Q`Eu1OV z%?YIs2x1GiTNM-`!^eNe=~izx{3Gpe8@-uo_*^+1M!d}Gz*n~ldBS-XD4RpEYVD`~ z3CE#4bzn&d9yquW21Yp%m**qSiCA|LQqt;;T0Y9uZutyWFQ%5rNbR+%>fe5oliX5~ zv2seVc%2m}ME6wp1I+|2_w!U5O#!czuXP2dzyRUAxJ87UW~Rg6=;=jl!-P60@;lEi z4QMc6Kx;;R*e@f4KHf0R8s%+B=gAlH5%FyUTFT(@a!~qF(`$=wEgEh;w$HLm{z9S# zqLw`f={2NoM0Ac_%0*zPN_LsOfrWTD^+nn-=qO4WHaa z$0ZT{XvUF!YObi-UsGJG)L-|xXO=*evOnyJ-}Mz82_W@y1hc~aiy9p%o=TDv9cdV+ z!d7;igFOE(i%)~Foh(wnK- zm8X59UL|f5*kcGh_I+!&eeFKf7C}u(TR^>QQt9S;MSsA}aTu~Vo2i_vjRPET8~c#e zgT&C<1Esq3Kn(qPAci_*j(K5AO*M8XaWJ$uT!N2f2Ow-G!e7D;!~RsHeIQ68SXdyR zlJKSNu-x6k?wF7K^EW&mHm~5k%K}}IcRpz1o=7*89-l!^wcH;ANkdm9FmyQ0Djpb4 zqcjgVZ4Oz5tPrM10nN|SVeLP^Fxs1scDrc)tEy`s`5yP?-DmxvwJ^iArurr`XU3Nj?U3RcqO14{Zt0U{{7QwPGm|{Ho+z|bQtGVR-6Z~NAL^NTXucY7XX7SOONu;Z$w}0gPe5i zaOOEED+IS_2&tbM0}8hJ7b_nlnd3>Sq9H~N2a#AgW?~k9TjuN#5L5Gt(ktYJJg@olIKMgW}Vq-X`7eO7X?)qK(s{dJb3%HFd|a_t_+2|O5v9o{;P?NRix|^R_btgHw>5O- zyeT8IVvfIeQgMBH-DfW%EB^N95)ew=_qv1ft28`%Dh?SnLVR0O|AD08NkdVjuk|b0=J_d1jp3t6IS$3FErPBU-@vo#t*Ps2Pua@9e z=0r+AQ{Tq^p|8fW_t|QuoJTS;;C;nE_P5eEblGJFq2oge=fC9KdVc$=5(&U}GAe#d zf}u?`P8*Dx#VXDgxsCSuXXHndh<`+mj?g!(Je(Qo#L@5p&st;a)XNZ8eD+IC0U&E_D@elP@Z#uv6^=$DT+E!47Xvfb< zyp<%>qfowI1HP({x1z(GtuW0_V5}R7%F6@bdRqt8J)LH#nj6n-Hv_t`7d8`-7g|*H zj1>h>^^i#j*)` z=zLr{ll7v0+-#Ofmq?F84gH*D;azt6j-Raz-YhqdejF|S!6=br>$fyF84x7;v8os7R6Fq`3=DvAo%U<_1hU++H;$A-{9YKcqf}Mt(vRwDahl)N1h3y?K5i_=R&&E~H=-|T!+PnnY;;~1Zl{f&H5 z6|%TZj5GwRy}2;m^gBZEMt6i@N8Lqvoo|vMn115YgDt8whcw*HnNocb-iZQ94KC}jkadL5Qy@ihe+eN?XSEBK#&I3*TEGuLTykd|A6~z4b=7f zh^cpN1gw&!s2)jrJjYX*TUT=6*2X6T}{Xc5FOgyt?)c6+) zv><8qLX-@w-Mej+scco}38(%Fpm6_1-g{6aW}||9XosQ_FCJ?ub-dGH4bGpx*4i9Q z5Qi$x#F<)X$3?nLdB@mur>=mi5vSu z%)e;*KIWfB9-US9tjwAu5HCtv?34cJ=jO>ueojM?{5_Zcp$3yfmdbD(e4ewPk?9Vt z;-{TouA&8GYZoiOc%l_|9?YN|z+}LqTI6#xH&z-lkrR5imBaU?2CX8#ar#C-sicPC zX&*Xg@wn)HRT2sQy)h%&bM`2Qy}D6S4!^PrFE8vD{IHRHcZ#w7`5-&o8DUjSxER z{bI1ane@RzX*m;gG^k?uQ1f^BHY1XmVuX@H18^kpSf`fBvtnC{>Kxf|uRA5`fpC)$ zi_o0}2{5^58zj%(y?^axu`7G{{oWkpt)l%KcJD!b-%^1t_+WZ7VnGG+J0&3wjDp;i zC;y1f-n%EdlzE4pGD*qRbB-vpo|q399MbBQX=+e&aqq1+ECcyJ?=u40XI(`xfOGHt~qIf{5e zB|G))xisnvk*}HjpD!i1mkm+`br*6GxOGf*<-BE)Q`ox%)X`AIftF~Y6!3si&;Ysl35ubr2JyPf2jXOoAxPXJ zSclD?9?s9r=Rhl#^nA_=J$HQm8&OsaYFvXpy^BsP_5i;HSuNIlu!EcO!QtH2>4*JA zDoc~p6^Xmf{yWoRH%iL0lvcz~zmmQ_=P6G}0oPPESvZpPxkALDX~~XFW%W}7=s9_^hVK9Ur52- zQpqZ0%VLx0M0wZdu0Y^XLGOTQq4|B$_ZHM8fn%L&jsD{gSSOC5gmP)LBbK|e;0l@~ zZ*mpCZ8Yz!u|4fbi#pE1gpnZNSC1nE-FV0gmKtho#NHJ$7#WET0hhIAzrYzH{-c8eC)3lcA7Prt45 z8uZLzi@^X2es|bqR;f5D&I}3joXRph-7R{(>NnN-FTgEd8w4f4#U)tM@oouLy0hEI z-lrM1v#N4VbCtE16m_K`Oai7^mfK8o1I+X&jv8lKSATgeGYYhQO`cifJ48HPm{)lx z%<3quWKir^TF}({tHKsuSe7PGJ21d8wk)tk0_g^B((To(s~}o_4zjImnWwADDsO0d zcLM1bw;r`tiF9Dk_Z@Y0lOM&Uj)`iB<6qrBEjMo*@Zp(0NYD3Nef?b6nW(ms>Zo0= ze07~Xnrs69zO(6K_oQgO+OY$qP`(t+#*>x#5vFe_wJ*o3iP`-_Z_&5>*2-FNOP)01 z*3b_m!@V9&ZxGuOO|LE`>396Q=uVp1{*CBlaX-H#F1qWOTu%^0I@@){5JcY)-mWdH zjxkv4IXhl*+_adp&nGgnFr%!F=uG-W-(-ovCZ~Ga=9%F_?`N?ueUCK#1^h<$7{-o= zue=ScUgxFS3i~dxU+5Tv^oi-3SdD}49tR*>=*kKsWS_5T$&o?U|1;V*7wK@rZ(`*C z@VUn0bCGBM9qH?EXB*@+>gwq9{i&xEP}P@gh7|bHywddO_GA7;a}`g_#d< z3ku&tukF0HvzH2fP-(FO7-HiqgWChjh{tLH<#1gpu6)i(VXv}dUc2u@9`+IX4glnR zI^_uHlY7oGC;Y=!{CIxN(!)zkS{q@E-6Bf11l+=!R^}WhGn!qyls`~dH+X&sJKs9_ zM+;=)4ZYsesfd-z0xNl7*1h3|biS7tv$QXbUZA4_6O#=PIV{j;m+BZ&y<)t5$St## zY_@5xK^kRoibpCxHPX?H1PD)-+QL#2k>?2%E|gq*vhhLj6ED;Wmb0kuYvvbiT&n@^He4s8E2l zV{SW9N-u*+)_}Czo}P2+dSiFEvHhIW=`Ffk^i&apD^9%Mfat;0Wo1)HJ@j5`f%+iU z^K>t?-W+%sFUqt_GN3wVICzi|1%=-QEAj<6-O(cq_GF2jl#ly{zyYe6T8lJXoU!h! z&onP%y#0f`{Tui`v(Roxqy`F38vqAKXw00wdC!h?H6}`R_EWI=S9dMo1u@Z6 z{B>Hd<|jNmM+NK^3SxBG^Q#4CV3ZsZEa=&q z&`k(QL=VU1+#IwTGJLew%?UoH!ub0{_+i5gF2cSh=~hXPT?u!lgK0D5X7Hj<95EM| zYZXn!rk8$+vBaZ&QI;w!0s447Pl&4qIH}I<Qgglt>y9{o zd9jh%Cdg?H4iiY2f_4`FU}vYGogKuhc^FpUr2XfeJg_Vyy2P35l6N6&EBaqQ5)z}p z@zoOJH~Y7)ihI8;!1FjZfy~pNlVuUI3?_1?RZ$Wg_~kWq@5S8^Q9b0Su&J@P5{lFX zyr!#-%-?vXJ;ziPVY+R?ir1us!2zX515_qA|6E$kpT|nn!oUCbJkS4(^WZ}NAKyZx z`(NWcKD>B=VzQ;!QkL&j{u1D#lcXy@23Uu2G2T%~`Nt2B;ZJpWtNB}y6n8zrwhiK) z@P|NU&v9%wUhO}fKm~ss8Z`^zgmkn(IVOWobvwRj-#<)0QlapVZnfNU3ek*_?P3-^ ze=t4y&-B{hDj8Ra{!-@|)>XD+W=U+`Oy&=fjNKv8wMTgR&*VkuvfP)!6`+K8;;D5m zWYBK~P`D(e9FXoT$Z3wa?nzP*(dGKm4VCeq+v)rQXhAl7lxVzBkQRzld8-8cy#XjQ-=8aF$E~@#lJSm z=>*QEj?L%|Y0@~;;hdvaEO1&dL#+pRAnPRATA;!!X&Ou%S zk7-;i5S}pIK5aXBCjHhHDJGc*w}0LqtGDnzy+_ha@HyF`*8gupM;M&YdGyfbMbA2? zOuZwmXa3#J=?m#}7C!UPBi88)i3p*>V(Fh{51xv6HuiA`tywa`&cPpfEuh# zpbF-v6L1MuMV!YKfFAPY5ZK_Q?v;`UW`gqOJG;|Nv{UhOD)GzS&(l4^Q}t=RC9JB` zU(8ttdqcziM#U{i;-D(ff-Org6aw?(iul3R1-)^uBwm)pz66H_$_Xm+tQV}=X!Bol zV&lrrYix1_Uop^3Md|?oz7zwG5cH8;7c^7$ABxRBr|QM`zU>RhL5c=yEA_oL&)=JI z?qix+jpG|2wZN)IHna!|kT)9Q*I#&43r@td8^|22M- z`Y?VYPcIP#Q!dYe6*heK*dzl6UnQk@>q+8s)qc)wk&GO8)*$wC=~XCB z&XLA;B}LXAy}`oR)?8U+oU2=5(867AKvD9=Sw~&$(Lx4oQ0T;k3PoR6KvF=uyIZ;$mG176?(SB)4_$}u zMq2vL@%xMSuJ_)5_F=JDdkuTfnfW}QC-%;gE?10h9(;qNl|Q_TC7-)*2fVA zlc6{zp>xZRq!t0Cv54A~;i;u7*`QYn89Q+mo?%bBhUP09={ z3?$al3S3%~)#nV#cr1&6<^lC9Y}_+)u5GfDUV2#=asN|%y}!c>H&+TugR1ueP2 zXb0i$b$t313<%KH2@>GANxf%qzX&+m=uZ*2{)?Ic;)vMjUbRJ@r-7cO zaV-M=rAJB`S#3RdXGf`v*@L5QmUO^;-S+9BlN$r1!%YgyTHVDdEqndiLr`06D^xH$ z`Eu3ZT*G`Vu~A=6)bYHn>5p)a~ZL2|QhG^w< z?`fLV+QDx07JgXEY7wKe9>-x^vccj`xa-2T57ca*S1|oI3tc^_z>3!H&4-$kx@C+5 zCC@OTep0?vmETzq{TaOrg803E6EuKl{$Ga1Q^0HD8f=}W=8e`dj>?>m)kyP2xhmwF z_yS2@@h%!Ar{g|2r;1!E+$LPX`qRy9ba+SNi)2R;&N3Ncy}aNm zz+!Xzc-K?&z>KH4 zQ)4iNcOD}lPU^@e4Msb9vO;Nl_Wlg2w>dkpf;E#N30U3^!uhG*&z~2;>Si@N4Xkb^ zPM>(5J>4Z+-2Cq3?FwI?)x=-!Yu>sJi<32VLZVv4RcwA##2H4#7^7@}uRqO6ZopzB zAEJ-|9N8#EK>R8VLP_zS-`29UO1*=(wk=iRr9@qW{r^m-eeWvWCw>e zZ@?=jf*0mM*lvME9f)gK;kf2g%sMW8own9%O#VYTG5b$&T*HoSusKJdorrOj+w*(W z)DBb1D!S~azrWa_>$3-01lf`Kb-f}fw#)MHusY33G__o!flrrixz=jN!?PCdCNu-< zMGM3O0jU@MmuViWHBg!+8#0Nvpfmy}L$N9oxcD&^W^j zxZ(c*nkR)u_QP2Fl|Q*Zr#lqJ(1*)V#!-vhAv*`eHi^y;O&FplxC^iOMy?qlrux$C z`S~qJv*^3zBI1WR2=VOTrhXA6*jcZiTNv$|2FmHr6glqE)>^o zLq4(Mt`MVW;+=+@p*cla+C>`83QS)s&;Vmhf%7)uEULSp_(O_S`h_LtzhM`%o}pC zOv=sxO)7$n#>I*W-ulL-WFY>cO8)lMaB`Ro$+TW;$A0_ca=I^%e6xtb`2mRskF{+aRHM=Gm3jcy(EszYhMH7@<^kXFboF$Oo6L6@iV|5JCDyVAbvzu;7vwk z0F7p&fr9NNKCr#CRhr6F@a0e)j%bvUD)K-?QvgRaHUB|0bReRUePUV-4g< zzH?+K5O^yq3QuKu+m|HH*nO|UphXhiTO;}-QuR(FX>LM0igv~j0VS|mTINwA3$<9N z?KJkf%%(c?dZh3|FU5K!NT-Q`oNjo8n9MI3UL}$OV?nSZBM%oPp`C*kCF?nAxFe_vdB&GV`q!eTgC28p$>=xHwJj=4+L=_~}PUM6YWXq1<7A;!r zMu_)fz{m&d3BHPR?iKU~(enspCoITJy|Fk8=6K@9E4w_aXmEuJ}&&vIExB&SKP5)w2`&=po z`2tr=<TSpQs-@gdT_goM^*n=^i6Fiy@^K2>H}4+&J;rba8syvu zuCXbjfr~yvE(_{DI15b`y$KRX6OZA$jA(xKh<6ucJoDABhvk$DG=^^>8j+O+9yS5n zr;ju%yFbylaIPc3Uc(?7t=Gn(&}gePhPH2SF|j1C&n4bczQU3`W$0i){pJIbkyt?> z%aiQ2G5OJj*0w40$E6dPB zq^%R(K^Upl|C5O6{0|WmcRlC+bLNT>a4el$Y0%Ehxw8yw>Bl4EGLbYrVA^$dGZS3m zMQtb@g4o_%M;}z~W@y#Y*#v_;%$O#4QzbR|I59?Pf)19m*LDqm`bUI1PJN3mktx4|#j^=Et0t1`)}+rcn?URp*SD|sT?(xxfVAp0zmx#6E$ zV4e{u6srgM7^3*Ud<^hcDfXckNoU@PHoe0gvFUV3xrvMV~+2K-!RkZ?k> zn9QjY2c~E{{3lGqR7dE&4=#T1JySZ3o7tSNOx1sH5;>Rq0Dz(f=Z(%$%z@A>4Q?TD z#ncf#UG7YZ7AmN~KbCs!o-uq)OP`>l>)N6KJgBMBr5Cll&Uf9B^%+km+lnSS{*?N_ zFnl$5LtGnS=zFVmaEZn_>F#*O8%-feZvnY6*|m0-x+a}*%V}+6yGWZ@_T5E2Ppp=v za(;94!lAYRFe$&=PzcdhIJ(+d)OwI0CSy>pKh`INoeUPIGRXeQ+V#!h`ty zH@4S0nXUS&r+?uTi8KFq&MRvBf8f-_t@MWhZo21oqx0ojMRAL-(d%*xl1t6gh=1<} zP9AsPptP)%xvWNAq~?rQ)a9XeQuGF`P=r_s)b8LNAYJ&?l2?b4-3;-kDwzjf=3!<6 z#Fb@>T`(C98;IbqKQ*rQ+HL+2A{gn&LtJpLpd~yaPk0}4VFPcH5;)(q`CB-#XeJ1E4m$Gri3;qq&oEi%4-!amnyXe!Y*r&{5#((=kkC z8ZAQN_p7_72d%=l#xUrti#vA*+v?g*0>0!O+ak7UnS6y3;BYW|8Fy6nymcqG3GZ9j zRew?JenMNbIH}V-n*c{uEN;R8(a))IxxzU%5?$p}jI7%y5Er2t05v*154gYG1OqR2 z**({Y_WHvd7p}W6d-$%qqpQZb(#o8iTaUHo{(({GchqN)lS0qm4gy`_n*n517cN`Y zFVaL`RPRZZ>KB}Uu)2=M2FtwR@oj^6(>yoOu}>1%9rB)!jBlBT2|5-$sHvN3qS)P1 zL4%n}cgpsh?#1zIcHmt;CRlZii>;FFL^y;MNRPp>6Av6ap>bvtj}3#^X?)l#rc^2$ zj-3?3njPctAsq*wh+A5}U+jf^;91?n|MvY^Ii5+tIzgt<(8R3ri{hg9{e6uBv|AIj z{$8o;``M`;H4?la&@uZ5LiuJ3kHLY?7$gKRbPNQ2-z8Raf#rVR)6xYl$k`06%qkti zMo}Czg?u2^tk=kdC9#YwtvW^gK{4N}{~>S0d0ow;oxoaWEDUVfa#EnU`Zu2{THQ3; z*9&DzH7nFuTAdC*Nh3&%rN(Cv(?}nl5IG208OJ0aqr>TSwpg!?Nwnrs_z@1c@~jh| z3};xcr!Jc|Q-DxPco^c#L%s#lTGlEv)LRkuX@yql_6~}(W!r>j0sB}xaDP)VMdDiu z!_H%tEa}a{66p)F_DrcM!wv|o066GoV$Yb1{B zJt9g2iRsC@C<2(!;j!ee#?<9K&jJ=kF?rfqd3a&{(P+=GU|CgFS7~8W1%ELU@GsCr zKR^^tNmuA7=ZAdxClF284^A}r%@#kAVKhsZa#|9I`UG+ebgTu&|L#pDYo3g${vP@* z!Ap|dm}3RA!6x7Be*O!sg(}~;S&PNvyLj4TK6cFF9Smj7035NNU9ndQoDs3_0jc5H zHUY@igeT{E>N>E$4&IzQRp8ESSAmdYAzBV+pah-cy%5}LBmQrz%>fC#|Bri>pR-Ok ziK4t~`7|f6I@Y&O79&ZVgw+0vgYhnYx46#TYhH!9L9lG56v3+eC`9?=_MgHz286M1 zKp4yTc(P&)to#dTz&iGvGUnbH$kF#%TzlQ(LpZ3A=W%f3m76eK%dSmSk!Umw8^nO( zIXNH&GqPKWDN|3Xy&yk#J>aNVd>cq}a7~$}Kt&SuQerD%j*=LJv)O;?-qhV%HBxq( zE%?@_98JD9OD<#P=|NN+*i5EMF#2ZuLtq88*^tEkS&n3}dpbA>@Ksag8+@fV=ZXFa z5BFR8+y9LeoI6AMsP%hTOx~b5MmQ6+y4&fuG_}n+~ zN7N(w@q1&h5?#O+F#(rTVX3c?OL+Yyl5CVvkb0igxcISMgrrQe^7$m32fVmirxn)GpVE(d{0C=Wo@?_P=fmVPVs z!fIDqy!A)ZM>$KsKmY0XKfTH#y10i}PLoBG$mJPmr7|cG?Vq22qbl)VCmx!?2 zU#9Cb67~fc#*~rv1tdNND|c3q12TkjKp}7r$WXDz4bA~&fF%|?~80wzHc=<)|q zAPP1Q$N}ZTIUt=5`%Ft#(!{|h6IV#*#9wV3G{b%m5ePn=oPM^Pp1~76(Im25EX$6| zenovFxt_e5G=3`FY9?$o?8&QRw=!Qx4oV=K*bQ{fKLGM@<6uw&%i8_I2UMp}o!xl0 zB{mAWnZfsOrK-qH@_|~vH@FAR_{*s&hHQjiRs2j*A8%Iq{zzl#^If+S5sX6RX9J%A z4dBzuj;tZ&JA-rKy8LwIuJ?&y9Qa=hMip1A>&tvA%o$jVu2EBB6AB5rNG)E0;Q#by zZO?mj%xUpEa2;bLlHK6kH6+$JNVRz#8%!AU@rX)E^#B!W0QjRFiW98hM1liMmUD}i zghR?e)qUVcXk!OCsN)lRV=z;M704>LgqXWYZCnyxF-L9RbQZ+?SfDS6nL<^0RSv&! zY3)0U03`;NcpI{heWN%D*gJBK;FI~q$*Xko)&P8D7@fe$n#I4NY@%Q&+bS5!=26z8 z2~4K^;mvmHEMvK(B-h^h{JO1&D7!+LgRnprOOO0?#S!5S?LLy+0!&uuB{F>39Szpb z`;ldwv)=o*C{A+)m0}RyLQugZ0ABWZ4{AsidiP-?3%=C-#!iS@6GWykx-Y9P*}I4_ zkzH+-&pc3X;o0v{k!7qE4C3Qbe)2g$kh~SE#p<90!-!Sn<@zIwo{4a(zckQJFT;AA z!scr*!eJv}ZsE8eZm#+Drl4B(T@xt5dIxrp7z@} zH!Hz+1(-XF@qyNw<$Kf9{|p zHLg~}8L;W}V$fQ1PzlQK;m(>v_}(nQoi+P#XU$~EFEh7cEd**cKQ-|C$K8jS1;Z7^3J31VCP#r9cc(GU*12rRn5J_{DEpQue|3wHDgOUdt;ydg zpFd`-)V!*k&1i@<$B(Mv5#97>@o*4m{_+%K{*@2z58s==&r9Hi2qZ+7P#r+wuXVoE)d;#O~q7q?J5NZbit#hnQpO__Fb3g$bNdK&;06kIGV?iW`gPcXO3|$P{5`lY%@@}@u zuE1}nWfx8%5HC6Y#Y=OOaJ&@tKj0E0Zs8FKm*S?&eM!+O1Ql8){{@$dlXY}s|A9-a zaJXdpcx+nycA0HGI*KB#;Iyp^lJ44;H6-<{e`yl%g6i6|sED<(UFCJ|##I5ihfa0b z58p~uc5iqMP$gez=X=SgOTaYjo}XBMFsXEV#uy>X;-=WLmFMn!N8{*It*O@+Y^6EK zR+U!A_aeN|KhES5*Ku92Zf#YT`$X)F+D@BxROsOHnd~HS*gcVFndS}Qk(b4L4`jfVhBzTXDs%Nz5+(x^#M5MLRi<*{V$z)$FbFIW$JV;xeDExEIr=ZJg?l zBJCx<0PxMe^iv7bWUAJRL>tIhm4(^`qWoWU75h8Qe71fZkDP0hk+s*95oH4`-B#x8 z1>NmNJ<*{xUyxh$P0y{}TY!_-!qWyG(MOtG>doCPUhjvNUR5ttRSMt^`N3+;cdU-T zSR^&>6bdMvHkd0gO;)|Nk~%a3q(`_U_s|pQs&<$v+53SW!Z%NZhso7NT!Ne033XD@ zEqtsn7azj-TDn&V_J|}y8beO??+knNVu}*+7e8IQ&_FusN!n|X^Z+G8ovcN|nL?FN z=f#8RBej7>Uq`2NT` z?>XFDG&pSt#4ULN{@SBlb53bl>$_(IJR!7EaL0os4M zl0(`aPMWn91XO&!CsxF&^Cgn7h#U?eB$tqiIH(n)-FM!QrR|_P>pH zD^IkdM3tm=Jd|kM>$^#&6UK3qFRo4(2N`i&-k$fvYHf_LkLtTyf=NGKpbjOyu=kj< z;H3Y3fM&rXvYbL|j8cZdC9#A8G1lR*`-IdjxB&qW*Aw*56~qj#l{HK7ua)<=7paaX zDu#zuT;;<(cdvd)O<7G>eO;o1+50Aq#%%{%Ri_ea$Ah|t^jWHQQyen;fqv$Jc^B=? zggq+zpHGXsek2-&oNqlDM2l|8ODm!?fWKSd@{#!(tZmmFc`G+0iq(m~3Q}&Wis9`gtUk8vp=#B76TaSC9L(QN((!Cn zu9sA{Oy)ixO4{T3<6{lAa|CbFc*6W>&fNM-Ft99@p#m8dzDD*lCSheGOaU(#%|U5F z>%!sQGh23rDZa&k_xN;QI@&*XV*$#$KiBi8uLzCqb{^Tjt(-<$499FH)3+e5!|Nqn ze{s@%kM8Q-{9$oh$m+z8VFR`iO1+-}0O@a!ORQTNHkxus$%tQG#VN0#VTTO%Yb#ki+PA4S_a0ESGgxN1(fO#}0xE{66 zS*@N9tx3_G`{Ug>F|GbDHaIQd=AubLsCt;ZIFlRBuKw)7Q3V?SM3 zC>K4x-{HDIUwqG!1GKM69M;U?^DuJg}DBWHTxAw~|z2nj8>chRb5-MYHMb}WD zl6W<6U-3RXT^dK~ix4ncvmfTp3|fl>36C(mf^jxQhvbrlMa;K3goG?%B39emXllHS zq9NV$u;K`fc&f8H%E0wq%0O!I%rRQRo0Iyy$LK@Wu{nd&Bx;XCNq}U zaaCqbHV;(JSc9DP`&Fv<&KHxB->w=sO%hxrpcjP}3n@z|+WRZ@zP%DOj46cQx0|z$ zHO2`h(_x2JHxj$*`ddZZf!zY5rfw0fu-GeQBgvRH?M95wkzh2IfI5mN5VZGLZk$SA zLmHwA-gP&>1CITDR2#lLVM1Q(lEAwujARza0|PA3ntXm;DW&KbEd^Qm69#E~*easG z`YAxF*dF}piqFu<)RscA2ft+m2(N z(JkhyFE9-by_GkAMeN^^DV4r<~7;cA(rwS zf(6jo8$kv1$@P>7$)gtJ=+L2!MOV5zv7+H`#+dt|8Himv;pY|-JGJseYwcGJNDY#B zgwq44b-G9@`ZGl_^uWgtecx)nF7OBs`Zue+WOlX)npHKBHECBjIE-_Vk9!lw5*DcE zI4UMIH=|UtRL%0M#y>kWwF#v>wR9gJ$Y3WCDHn7K0Az9WacQKO6<$Cd%{F+X_D1;EQG>jtTj7XZtcvQZoQE z@~wpGZ>?XXmbI@xyP5y(eeF>+x58ADNt=`!uUPuYwYa>ynEV(EjRjS?8JWzOa0g$4 zLd5EM;d-M?-Va1lJ{FO)S{H))v}Y1}DjZSm7!TuZ*e zidb0+ropob3`n+UdBBFunLr4XI(@%z%0^&z5RfdD79dNX*}(2Li7&$MHi3<{X!#Y` zM5dOlQWD-6Qj7}N^~`O0nC&~MY|1GyXx$SKMiEM8GQBP=jZ2Wi@#cKyC#4KfKcNsE z)(D(@-89^jfTPP8JI}u*ANd}9NPUZV42;Ep>)tS>@glHnC^rq0~RZId)yM1)iT7 zMNnQSQb>R!R3@ri3M<8;3usWTjkSTv%60MZ=*0G6U%0L!DklS-rU!n4nV=ug6QlCF zQ+2W%g_8O62Kvt@NpEuV+mITJrt|wSs?t67DcbVNBeuQYK=unaQ6MvFTgjafDXE?B zm%OcNcHF@yAZ*VAx90n>irUFHPl5VwZ0$kb5v9evRjU3q???7wz^1m(WF1APFdF7; z$=2JhwBgI_>DUNa;aiT6eUzjz%}ph3g$P&6JCl9S*Potz7-#4R9ZmHZ0beyw;jfyb zf_xl@*8V8;{nKx?C`MG5aO2P~WY9PSMYUHdin?=r36n1q%ByNekwExOFVc%r(;bpc zHn)Mnf?50=MyVa2>o-@)OQ`276F z&f>9BxZa3~IJ-rA&*xf)9y^``s7=dT+fBZAiQmQJ0_+pAjJD6|&I)lVt^vf?_(f0w z{nBYuURJtyB5??68{f6jFT8gVj}svuFdQjTdHUZUtJ6%sge(psst^&3HYg%?OlfEZf*2{c@IAIPGX573RM z`Y^0B+s%}kgJXgKFC>IW^kvm(cds7fXCYo9<>5v?9mhW13;g}L85&Zepl&9W+*=L_ z!>&$Ra?`l-8k!$qn&^OkDC7~XZ!h*fg_Dlhs#NQ8>@`DAWQ4eQQ(=R!cwfISWzMZT z4XsiUkj_J3PofoXZcma52JQTYy{hsbR?!#}Qc@Sc*~|-ZzZK#CH7Dkxp-;~!MGRHa zIhU{5!Bt23TWQ1P;M507Oxz?#y|gNmZzH>B>{Wk*7Dt1^G+1^i6u zIVIm)>tqDe3=T)Dx8Z39J9qz0GhkqhVf`n~;O*6`qrta?i6L?6-}R5=_AX7^t`2Tw zUE=P)O|{UD>coB*_BiFg{y|rB_wrRDA2PtvTq`eoWb*c$Veuw6{vJqBT2ut&hTNFi z(jrL^8Rf!R=p&tBd$FX*jXO=rTB8zlzX_cGJTCd2`=cwCbCK%c(xV2;eV~f7iL(4 z;ckl}W#W+C?NBm%nbjBOqUH9$~AS4A$5Af;H#5D_cVZP+JB&H{X` zB#8=lPjzK1E0`Pn^V2qH?Nv{~=}Ez+M(J7Mh1k*QWl>|1XH(kb#c5Q1**@ft(q5ru zI_Z5yRx%-iT&{>KTE%;1dsl-sD%MANCh9Bg!xkVRbBM3nudHQWC%K#w3NYcLA&3I zZ)}OitYBc?d}g7#_f*wiYUUdI&jA(T6_%Wm)ZmOMK$3e=B$yvXQ!yx^asA_v+l!SV zU%Xna?v%F*6ulX%*Rr)$HSDEL@j-#r8<4>hbIeQTENkPPzoyy0DIx0zI$F}Lr*8&M z&(7xkHMnlP9zD03+aAxVY@P0rf)cKosR=+h?|41u>E7ZQl!{-+o?gssM-A?|7D~@K zU)M9~{St98M-P{Ju(qEU8=0vT!Fk#@TlZzkk74>F(?72scZG-@3IGh0L? z@pqn?1%oh&aeJU?@d2RP-7>pMvtF7Hzu()rBeGsfc_^vm@J-8%kvH-z3ZFcE@tsDD zf1$Oht=UmrQ2!x$!GJm;HqB}9o~x;6aAZ7(xauKPU~^7&!I^e02dHyx&3RDECW*ga z*yO(3ze_4~pQ=lCV)E>MT3p4wfj@BXHO^K1Dcm(&?)O$qF#I+=GAGamya}7`_sHrk7!R=R4kFhh{#BLkmy#1hb*h`(qGT; zbXis*lixTvqyxEFiNRB!h~9f4bl0DAf4!bIA$rdt_-^e{Qs`4|l7Wd*o9aXOZQnY& z>t-RDT)R)m;B`=V@g+L0$6E3Cro>_9G!N%x78pQi^nr2g@O0n1p@8?!T5T#pP0#BZ zfuaM=k^iShH0|6kf>Y5agUpdv)NIWyjPgOZiIgl8O#ssw$+gFIpYbAl6~GsMA89W( zm%=O0f8AvIVp6W5sez)&@HhNiEWM)LL`vvr6Tobsl%eallAtN9vyXpDa;<%cCKI zV^U3>EbzR;9{LLX7Lgvw3&kOTh1J^j>#m6nlRFqj-?4G)f+mmhk@Ym(Z>!1yINZ0s7x^DBv&7J2+` zjs-wHi(+wey^NuTum6l`u)RFE_({ectJLQ~w5nmh-+jG}n3;?DKSTjT+{$2Lr$jZ~ z_-35x4gY;1C$c7RZ4BX&+L(PhizvJ%ITB2aBLjN(Ch;>lU=TZXd@R?8`Fp%|@CtuAYm~>iR)g~W1PZYYwr{V=CD)ZgNbyM zU?SZ`3-g9mrTr9;NVN8(M^l(e?&eubbx70EXn0c>JD5pm)q32L(O1l;V3795z+vzK z#RL0|xhNl-!5{_U8`uc};BPFNSay%o*Dc#hwlu;PeH#J#alqX8JwHBh{@N_P-8qA; zV#sVaL#@QgcC3asU9cs|<#q<7wm^OOHsOv%*<0=BG(gUIfMqdcUKZT>>l5Z4c~M6% zj~&(x>&NDSSFMSdVJH>~eJyz^x0KQQI_aO}<0Uq=j~)9!yXnE4;VV|jMlg)9-&o1f zPRxD2;^rb^T`!ogP)%y9n7g4v>CnZD{>=DK7BN&3l$0LmsCDRMssM?4zDGFZ+Qx72 z`{Idof%_RO`#}f4jYqfS+OPX0slDPHt)fPkq6{v|WWzFSeCNj$gTHiD{;=sQ2 zwJWZ3;Y_ne-+F!N2-AO^LI9&m+U1;uSV*VYeluxR0W`81THEspaX-@~8^q~GBVJ!! zQ!3;bx}LFg5-?U2;$L~0?oU6my>Q#;Ysj07zL9c}GNdm;yranur)i875GMs0 zGSBRyd16nG7r{b;lg{fwuNzkFOeSd59Hd@vSY_K!C5Ws%k@N5k5RwA|dA(k~2--~8 z^<&tL1z5saJEc~<$#WA*I397(hX@K+dYQHPcV>Q)Te7Iy?MIpoHZc!QNsRf>KQkQ) z9|d`7DYT5_{a#E;;9oE-JH5>ct8o}MkLJ=`4$VXJ2%PnnFovm=b1fvKPr@B>B^>OE zSkM_`fliuMp8Zg80Xc{J`f`&fE zurXr4_yNCIbI|?pb9@E~ZpxIW3*z7iPW%NdIaca5xG0H0>?FnQSX>F%D<>AYV{21(!R3^@Wj;Ae3a0=Yi=@f;I;h2|tQ^OuLPQoF?^qu! zw0{do5K+E#6CLvse*@k6aa?`yOlV2Eo4!%r4)um4HXr!K(QJqT(?y+R}mE6tH5 zikW-f`J@Xc==-`&oR(6T8&epfhy7;rH4(mGFs_Pb1B1>1%U<+`@0uSItFbM;O8jP{hM1n6j7B|q*qVR&c=6_H}Qrt zl9+9+0SV)rMMS1FV*R}x_s9^A-4Flolq(l5vQc2xxkP)bFD%YKvmQLUaW{%obJ_oI z)cK(f5kMXQW}TZL1)hife9ChV6J!6V+l=lO8-bWf+nqm~cwN^{IMomc(%CCPXvZR4 zU=6ANux}XP5Zi~rsYoL{Q)dU$3$VKwh3rWMefdVNt$XXid@&K)SANVSYhpJy)F+q9 zOwzJ2|Je~a3k;|l^=J0&{DNxKo8S8{8QxC(qtS1&Lt)EC_@sG_6p-7Bv$FV0>ft9Pfd;sCA zlgCb@8kU})Yfe+M_T}nQTste|YZ?@%wF|4Rbc2vH8P4W>gp2v#c4nA}zL>bDlLdOz z5Yoc#<2W?|Lh%KfIy=GYJK?6`}&PEf>pvYLreM zJ~m>11b@agm_t}H1;Sw8K@BnxL@Xbf=%*DO)JiSkn+-oLA=}QCx%MEOylhZ`30N|C zV48rShr51Sa9e3nG+5#?Z)%OoXQG=lr-n;N@vayhWoF40Dex(CyC+Tij`^AE4yc+jwG6$o5Hk3t z#v;ef;CDybQIyRu-edJ{49_gYD{I}KU6g<#G_qUVV5;F*peVVT~g03HgE>sHEpekqiCuAs`^_7lrJ@BY53Qd>a9o#FG^*pc`i z;725G*U9F1U&p=0cH}%G8ijN0T(iVBqk1$sdh}_SvZ5T=*iSELT42}#boXMf?}ujq zd3)SK=!D4_XWaa+7aldv^-0WMJ%UD?cKHPtn)U<>D9+T&+Ll#`g+YbrD81}h)^178 z<-Zi7fBq^&L;kH0#dd`&Lai4NY3JCH0nGx_sn?;hm;pnqs~A>+Fq%uuB%g;edQvaz^??my^L znb>0A<&7X6F?lvOyz=z`2f9OieWoxY602+J62mo|Cc_|t{#q$cn`uCKy7?o%VdtG} zh~((h(w9M*D+G#0lJPS)!JdO8*_WcR>z=>as*A{vJajUl@s$=uhHXx5DxRaq_JC&L zmUP|AhEp3|w&1&)WS)UC_sdgcBxyV3*7t$a%N(;5!^9gWZXv=OFZp>No?qyD9jP3t zdCkd>m7bnK1TH#r)zZ$>VO)hRqNaH^x>+R=MjtIpHJEI23S>Vq#u+_EZ7iNFKFHUS zI(k9BknUVOvhR6z)6UB#s|wS!QUe;zrlFNCZQs>I2VxG!?LQ=(X2xcd#3u$1Op;vimx-_vd^j+IrxP)-hS+9pr` z2=4Oim|X2Ft25OG0lr>RZ{>6(uD^TURZ}xH4<%W4P23Eo*lcV4z$yN64giH0Ii5d& zu6i1jd#9gBPd~RK&cP_^DeuxQj<$1LZQdkNI$xT@(l|?wC*!)EjxISafMB&tsG%?i-NK!fZZ6}R8+aBQ$0x3_MKd2*9v--Ab7GeZ zsF^~~&j$A&cCgLwkuT$1UNKJ8QmQ`y6HpUap5Q^!34IF(!^z_#X@`Mvs7`}Zl%wp1 zx|FLGc1v1eR#~cYWe4Xmc@BqG#wH+bMf>0}#$`1pRB_x;@ZoV#>%bzjTVVU8zKK&8BnJPp+D^pL;f;l|c4 zWrHhzy);{?62o%f=7XKzx(9S*D4m-W4HU~2)cOB;92#O@qfsqMj3?jvZb@b5mjLnF zFm03Vi$Ey`E+L^u#Bv*()~=!46n$AOZ{O-Ul+B!9_69a++WpY{mfx@?#JD=uo)%$j z!;!zt8=ofe68L%~E2r@Yx}r01S1xq0LLlZ&T#epo#^JU1o}SUnPe83@C(7cEL-!J+ zHex@c_Jyi{#aK$t>aTEVf@%5c+uZpUvV|k6IsY7;>b&_ZrI{*2o3Za-?d00-&NGsDpg4ej>?#YSBuY;RNb@8ir+T{mf4DDPj;2fwi>HO z&X4T>%57$BvLe1o9B4hNB`R=w;;M82C21oS9+5&fiSA)IoqebkuUaZNTI%8a;oC}c zE!?BU>tGJ#dvpD|eG^cWoFVgqlgq1e1Uzx|Cf`W5T>B{>TbF|0vwgzt3G4|H-c&bU z9>+q)@trZNE(2JW=3L2#2f1w)e4Edl5%L|J?54chF{i&wdavH^x(>Dq4*WJpkGjRY zN0M1K^?~%$Jl5&p>g^Gil0N3$xK4h?yR0XfoTZlq2xNVGy2+zWzPKbSp3^twz+l+( zW#7l+#I>{DUw{576zS{kW1MTsMRz&3_zSd#HiG2bGz`?Wrxjy^sXq@m4s8WLN`488 z@eZFqujSW1=U4K!-GQ3blWDbcyfh0|M-D||-J00vtwH2P?L0iNX0R${3*$Fhw0yVY z&Elti5BRn`p4>W!Jc%xUvKp>3zaw-PWVg+p1Fv-oM0 zZnLoZsH@$i72hwAJKW-V-il1kv;4cQ5Eu)cb~xM>Q4 z2CCy#Pmf!SucSIMY!;N&5*MmIa7SG?rm-=kH^?)09t7uY5RfblXYC3UjLZ^Eu zBB2gG%4m%!En|d+wyXHROY_W9PF_UYh!d+FgKyUHW59?hVg}D+9V!o&b`CSm}Xz1w1?GjxR7r;T=}np%6n(=?AuM zUD<)cpbC^u-6>)P4BqcfB82a|FVPInY4)!lWu3?G_rIT@@OA%E$ZhIif5tV;JV$D0 z``pDbF!H%p^)IK)7h%j6KnPe@(Z4tb2sRF#<(@YML$!hgdb{SXvoNwv1Q~+OPg#(Z z@Smq7&!zZjB&mEKz-T^BSqk~MY*0~B8x?#?Lfg&Gbu>?N;79g3J$_oNGK7?g>}><5 zZJ3LEy6-F%v}{01@A}=!5s}9IQ-(`jxu*@HEzmuu`8OB3x4AM^LvogMoNt;*fVYf4 zN!wbrQm!_Bu@BY=b80Tn>STRSU*+pGKf}IAm3gVLT!-y(h_t4dL7eanU)k&%O4M&B z8bYT^JsLvJK&T8SnjV)nXN>E8r%7$<+D`P#I;}ujqKbYtYNm*^-N^8fo)2m`AsW*1 z2K{6D76umdzFC}Y3{&xtop`LmFTlvlhUI`7jxH#2lQVh!X=Lwb$ggPOU&aWU)*;Tg zLU9RICf%yE*OaOn<E zGX!eGi3%+F3H^arSCJ(wa?bZ&3ly>y3|*o!dAh>ga{gN-iEnpZ1{7ksT7bEWSAKa_ z4-OaU2LrA>`~9!32o*004DL!Psvmtz;WG005_aECu^xO|+)k_hb@>RdM2PKTXT6z8 zYj|&8Fm--jpkY>tA}W1kYbkaw=7;banYQGy@Dhc867p`z)Y50ji^(y!Ql)}O$I7wY zub&wYfq?ovu8nOMdUVoW-5u}TH~kR`I|Wtz*`5xZjfSgvW>N9$$CDyq&#{@w2Nj6NQf?_5e` zNxk+|IKq>q@S8~X%uwr856o`5n@l>+_HL2)&omY`7rUjLcr=T+pH;@oN5nB1Bp`>o zP|dlZ?s%#>y4M*$XwoM|e8OHJ@5@BPJN)xS@vm->uo_aTjJ!2g;=taaTiNW2; zw?8}af|tQSkMlhsa+NTpE$h1axT6u+O2hH*dL*m0n@mg%|D7Ux*iJ9>mo?jbn)*il{~w#sV|D8&}M4qb1RYTYVPH=#X4PNzUOwZ?wU& zLZ!+<7UT{@n!!%{3pzpiI8R)jqh|t1yjG+=eOmo$tTiN*g zoU)r*&WS1p#yf8^z({lMf}j1OFsZf>4sK@MhZIc6yZ!XniET8i zbr^e()kHCFq}yChxz_Ey{D}(uEMLPeK>%m)oo_PHu)W`~^9G?5PRk0;v2`ZH9nOg7}$Ts>>c8ESd*y!mXum9%GVH3ZF6e+qIP`n)op*+I+A(G424BBV%aXVPMSQD*idC#o*xL*~pqk z>iK63)DZ6OcA%x-7rOgCp1xUlDr-$G_zu#EY%b9&|57W~Hn2=3k1w+9q%wjXTf`oF z4yIp(%Ep{Gu1_4Ivys>U;2&Xw%2Gr%gn+sNVQC!Upj4|j4ZGY&RF@9Wc+Kp4yX>`) z|Gu+OkQthMuhuo3B7y8q2cVkkKptwf^YANU`^F~T%DH?`Nw}nO9EPSI9YIE%#lC!5 zR^R%PD|IrRBPnI?@&Hjvv=|FN$ux=|j-)Z;JS+{eX0x>)4TMeBDVI8+ z?}}G2bH99lU@jb$-7%LQD<>;JU8>d$+E$b9=QU(V6QSZ7#94uB^jjb8+1IhrGvTn> zfslOr&G96XrBZ!GLR2jjQ0pf#5Rctq=+QiIzi1O;$Vfe`@+XYd#w>_N%{p0NVbq2j z&bLj;dK^B|HIVrDR&h;33LfmrKF~;rS>jKNN3Cf(hohf)>Xl|VpjoC*?nFl6+BMZT z>cv`|tVzdIGaq`aIkCVx@Y3o*fp+xk9ALsK3i(+3k#i??h&l|#mgTDlm)9R6Unn14 zL`lAoUhdaXliCvFf3w4>kx~O3itsd1eq{2%b`a##3UT&$7GCx}kK5%4{7D=;((5I- z(WYSY^dUoG1%z(RCCvf(<#oMDu(q5u@ZeE~gqML%-^S=~yNAM&xyo%B2V%K;*x5X; z?)$j<6uD}eZ^t~=j^Si7nzSdsczM0=HcEm8sxJVyXC5Uq;@TR|_Nc0vh4SUz!MMdQ zM>5S%Al7vd_vbR(|1oPyve8(~dKOFTxR4P{0 zQsE1Q>)2YXQX~M|XzvVnUG@`pi28k@GDHgpkg8t8^I;amJEc228JAP8)l`%ydgrxX zBRrUgvi3G?Sv<}dICMlFUO4|bd92A=mJt}xIxtH4$rF#cev{Qp+?ct!k@uMT-3hlG z{bl86tAm7_Q>E$H-8#&u){V8wBCkZn`HQp8;FwpwVxl0=22+HLdw|^ALTVX5lOfH0 zNZoUhYFtK^lUObuSVgIlfITjO4|!mR6)UO6D~^cVUHD5H#@bhvOBl28jmS286AE*sHL=$m4ds`&zO3mI$eMIZ zrBZm_pAw+(*|^NvCisM0Rzz^FD5%i|!kOiL+Q4}-p}nnXqxaMj%3*xc$F!5_>44>c zVeMh#E|5qohc(y$!tQtG9 zFTCQHFzH#4V?F0n%@EkLqdu-2Mi7zf4169vP3Qsm-Pj)YzFC(RIqC&`WP39YJH zikVgp`G{YpIKWM>jCq?Zl`e#91PzOCy{hln77rEPavYb?Xi5>TawkQL8S-U^fAx$2 z-7gq}v0OuOWi3*Z(WvF9_`@A>em9k0ZQE5)gARjdNz}hr4U;;xeIWjJzYEsrevNNH z=`=-vy|wU_;}02g20Ez8%7rz7)2|L(Ln?ZZcV2DtEW_VH#V-X~q!>MmyLRK8)CUPm z8#ld@dw3S!Ui=>a%qS+1F-O6XE6MZ_n^*R!mB+lt@A0bTP}P(2B({}a8q~HoTXc{@PhypKl4K`{(J$$2?<8PE2UX1033UZXvW+QVP9H(#)rDZ) ziEy&NuSRc}I>(ssKzo89Ad2$Y=V&Zyo|*Osp7yi}UI(HV3QAt;du>oUz6+JT4xr5`JJo(SfZRnc%+ z%**v>pK%*D9$a8-ZPinDD$|b-D!1~jRr$+~8VZ1K+Aq^%j_zRjcM^pA=>p%tHWc?4 zI7+<|Ukc?@kq9aG))ZqhO=Z?SC0?)u-g&cn2zg*CJ4!Shq{`XR{B}A6B`?4X5%rD^ ztsm^^kKfMsL_mhxdUb#PbG)Ny<$cdHVUz=E{#xB58?OEO(U^xs+5}4poX6aY1idsi z?jk3~?X}j_WKU4Z1I|_S;qH}Mmz#D4Dwq7c&8JVv+z#7nM5jr|wN@G>vOsOLZ|V5Oc8$Df4(B-uL3oszw_Vy4v!22Tu%4C)^(ALV!90V2#VY+Z3WzvE7cFUC!8Y=O9KxYV1$D3!ynMYy)vH; z2yhF4YA}@(+d&s1NwZqkB7a1PWeMdnu5OuXzwj#lK*a^V|8vFdp|)$?fGLB3h|9{= z%NbuT^acKxW;r5+Ncu*wMBgGGKC*w;LYAwV{YUh-^|PAekFC&|tG$-liCn_d zjKecr9^DE@%^Ujo#@1rXV;Oul*LpiT$ojSxPjL_MlKWD%p$=M?;%XXdEFWhe_1>yS z32Lr7EVDm2E_v1ow@i@!J=8_gxc&*wWD>7n++FH3v?)r1~%hc6Ac44t!ge3jar}U$yRhTmK7|x__l3dsp^hqc04x$ zl!5qOzAod0yeryA3_izUhpd)g9}w&D(3;}E*9y$$JHu@rC8(R0HJ}I4r8b8#P~DMV zZN=#lxfh>;O@&H=p6nVs2^ivK+8n!tSd-8b4lq<5gR^4Xwl;83b0l!CbPimXqwzqF zN+0ccBE&%M6v}MaEpB24I;^r>!|4%PSp9)?x(-=>cgNjZFt`MP1ZzbA%h(K1-%j(p ze*B=QT|XtXF`t;S zl!nWwa7gd;uYXpD?KAV#cT6t4@;N+X$kBE_peto8&v2D*OVh!7sbC&8LKUL%C(gGVPWkU_uBGnDSqHnU}?RF|R_ z!EXPXKLcW}`=m5FI@wX08vyUR0AKJ|Xs*SGt_550Pk^MH*JE}8H-%mB!B=dVz`I@L zDFOeY&^m913Mg4f8vm7QhWEVG0q#m)ceh(C7vFU9@5FmAubK5x6=M#IW?zoCKaUjAfHmDV^AQ>{5!=u&GOO-1++7N(QU z1XXSUbvOe)JQXrMle4d8U{+i5*~9hhm+IUk^0tHwxA2r_zjBsYoFzN;?MtT>?n1H2 z-bYY?jIW}CB?aX6ROWc-AykoV3M4{;ob_ zYs4=no?o=wzi?F!n)98WaDLr(j*OCQuJLl73? zykl>q0K8-!juHt=!j+)IsHw;1M&0FgVP;0b<#VvNH3Jmg4UbKY@XPsN0lGkb`!tHq zhR0tr+=9!DN(d*lac11{%izmEAO;FOuWk-AF$(4*qcZwfx~Frz&H1&uIsgbP=PED- z&TzVWP}BX&zaWp zsw=soFZ8GoUuA`(*fQxN#4N4e362uF9i2_Pxm^pJf_ck^SmhoGk$JiY-9)`jUMdWi z2oMdy3-{Q56^W)|9{Wh=@m$^^T(n|MrnyEHU>aRFxKFc399*GJ!3C6n_&2fTnP;@p z79UH=hP!Q@Jv9U@WoL1?-7?s#>KDfy$h$) zCX4Q5rw>0pdVlK2=xIS^*Jv{g)@tfe{>=N_x!=vde1sk`^HD;$dn-N1P?AeFPWI@K z6}<88Q#M%_s{EJyO#6=cEu>Rv7J@T347axc6$d< zoX3af2S^5J8vg3($!_Q(>bGa_FfNZeg%Ovu5D15Jcn1I>hdcJq@AtK%b|D~FJp;@X zXkp(`ySRGiw(5Or9=T$o)^zD=`@`ucbAqw%yd4HHFu?|5y333I`f19plT30rdVjgtR4DwWlzh= z^AN45yqb|+ z%6Mv4%`)8$;rnCl-JCdA^>lM#L?Qp;ywdRo^6g)jbQb<(;hc?a*e?y6n?r{#nc08U z*!a6wM@_wS_wixJOwln$`GPg;&`mAeY9NIl4YrLC)wy^wlWncGyIHk>Z`ePP{Mq;* zq*za*%1^f}Dx^BzJmQY;5Y3M+x+>)3`-PN|yjfA@n|~oq&*T7*C9&W(4Q?d-hJ29G zn8}X14m=VNTtbaV>>K$~T^0t%e+~ZUK=J8M@EqGRxK7(GwF_PA) zA-y}(Lm%}}z!D*vAy7rf`;l!=3Qc-NjTG-JLGxQOs04+8!sh{hBH1Fe`L;K&A^EP` z0PzKo52%{H<2)h{a~nQ#GIn+86@9D*God;%3p-{IOTOY6(NuL#OJicB{V_}2o0>)h zWk!yOKHmt$1@91((rB%wlduiuO{TEwW%>+sI(og)5mi`&BI;gk-MshOD6cs z;ib4nshLj4no9R;qYQAXMeYrjVI9XtfDAg()NV~AMH@sAmf{`t0&6hN@QT!YTgn``c91X+*~=}a9z^)TX_2dJvgD?EN7&CP83v;FT3^fW` z81oRiuvhI0_W_(WZx^>6O>_GGMXw2OV6rr@UBja=2Frc?`WnQzZ%FKE5@GGj^2%?v zz`85FzuO>~hK{QX=fK0bsZ$rNWQOm1P-mARdts23KELlJ?;B}VT2hv0#rh$O@`Nnu z*!xlsvwrdTl<2wb_z4*#buOXTjq-__8{R2fcJIx{z(^p=S1>B;m=u4W^>uo_^%O&Z{ICF)_x`)y}Ut`E>PT&sw?3=B*($QA;j5- zoqVojU*vo6@V}A%QUU&dqZs^+jB|p~luz zTJ9p@&T_RCIX1ZdN?eC#W;VfZ1_CY_t32E>I&%xy;}h9`lVq&5FPgw&%F@ZdwaI14 zSw5X0JX~#H+!&B@aTo3+0ylRQF<)&xi9QSNIv%W4s%NK?g9bUe;h-<1&l}8pD>*=U z1a!Bzt+ivf3EEm2+Z!p(BrlR0Ca}VOjfWWTgW9IM_64e06?oKBcipA0Ono|^%{n3@ zCk$K%0s9NMIO<9daCvb4<2alALsc29)=F?5g8ievmRl#>5csG=KI1ioi|zbE4VC8Pqy z!FliG=xAZ)ZsX+m|Fp-9U}$iON@DgfDop>=Qy$>p$lcSk|D}k$JVyWM)uWt>f4Khz Dvxnaz diff --git a/powerbi/README.md b/powerbi/README.md index c5fb0576704..d2c67272a54 100644 --- a/powerbi/README.md +++ b/powerbi/README.md @@ -5,9 +5,10 @@ The provided PowerQuery/M function allows you to read a Delta Lake table directl - Read Delta Lake table into PowerBI without having a cluster (Spark, Databricks, Azure Synapse) up and running - Online/Scheduled Refresh in the PowerBI service - Support all storage systems that are supported by PowerBI - - Azure Data Lake Store (tested) + - Azure Data Lake Store Gen2 (tested) - Azure Blob Storage (tested) - Local Folder or Network Share (tested) + - Azure Data Lake Store Gen1 (tested) - AWS S3 (not yet tested) - Local Hadoop / HDFS (partially tested, check `UseFileBuffer` option) - Support for Partition Elimination to leverage the partitioning schema of the Delta Lake table @@ -18,11 +19,11 @@ The provided PowerQuery/M function allows you to read a Delta Lake table directl 2. Once you are in the Power Query Editor use Home -> New Source -> Blank query 3. Go to Home -> Query -> Advanced Editor 4. Paste the code of the custom function: [fn_ReadDeltaTable.pq](fn_ReadDeltaTable.pq) and name the query `fn_ReadDeltaTable` -5. Connect to your storage - e.g. create a PQ query with the following code and call it `Blob_Content` +5. Connect to your storage - e.g. create a PQ query with the following code (paste it via the Advanced Editor) and call it `Blob_Content` ``` let Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), - #"Filtered Rows" = Table.SelectRows(Source, each Text.StartsWith([Name], "powerbi_delta/FactInternetSales_part.delta/")), + #"Filtered Rows" = Table.SelectRows(Source, each Text.StartsWith([Name], "powerbi_delta/FactInternetSales_part.delta/")) in #"Filtered Rows" ``` @@ -44,9 +45,14 @@ These are all returned by default for common Storage connectors like Azure Data ## Parameter DeltaTableOptions An optional record that be specified to control the following options: -- `Version` - a numeric value that defines historic specific version of the Delta Lake table you want to read. This is similar to specifying `VERSION AS OF` when querying the Delta Lake table via SQL. Default is the most recent/current version. -- `UseFileBuffer` - some data sources do not support streaming of binary files and you may receive an error message like **"Parquet.Document cannot be used with streamed binary values."**. To mitigate this issue, you can set `UseFileBuffer=true`. Details about this issue and implications are desribed [here](https://blog.crossjoin.co.uk/2021/03/07/parquet-files-in-power-bi-power-query-and-the-streamed-binary-values-error/) -- `PartitionFilterFunction` - a fuction that is used to filter out partitions before actually reading the files. The function has to take 1 parameter of type `record` and must return a `logical` type (true/false). The record that is passed in can then be used to specify the partition filter. For each file in the delta table the metadata is checked against this function. If it is not matched, it is discarded from the final list of files that make up the Delta Lake table. +### **Version** +A numeric value that defines historic specific version of the Delta Lake table you want to read. This is similar to specifying +`VERSION AS OF` When querying the Delta Lake table via SQL. Default is the most recent/current version. +### **UseFileBuffer** +Some data sources do not support streaming of binary files and you may receive an error message like **"Parquet.Document cannot be used with streamed binary values."**. To mitigate this issue, you can set `UseFileBuffer=true`. Details about this issue and implications are desribed [here](https://blog.crossjoin.co.uk/2021/03/07/parquet-files-in-power-bi-power-query-and-the-streamed-binary-values-error/). +Please be aware that this option can have negative performance impact! +### **PartitionFilterFunction** +A fuction that is used to filter out partitions before actually reading the files. The function has to take 1 parameter of type `record` and must return a `logical` type (true/false). The record that is passed in can then be used to specify the partition filter. For each file in the delta table the metadata is checked against this function. If it is not matched, it is discarded from the final list of files that make up the Delta Lake table. Assuming your Delta Lake table is partitioned by Year and Month and you want to filter for `Year=2021` and `Month="Jan"` your function may look like this: ``` (PartitionValues as record) as logical => @@ -63,19 +69,28 @@ or even more lightweight ``` It supports all possible variations that are supported by Power Query/M so you can also build complex partition filters. -- additional options may be added in the future! +### **IterateFolderContent** +Some data sources (like Azure Data Lake Store Gen1) do not automatically expand all sub-folders to get the single files. To make the function work with those data sources you can set `IterateFolderContent=true`. +Please be aware that this option can have negative performance impact! +### **TimeZoneOffset** +Apache Parquet has no built-in data type for timestamps with offset hence all timestamps are stored physically as UTC. As Delta Lake is also based on Apache Parquet, this also applies here. So, to explicitly change the timezone for all timestamps that are read from the Delta Lake table, you can use `TimeZoneOffset="+02:00"`. The resulting columns will then be of type DateTimeZone with the offset of `+02:00` and the DateTime-value shifted by +2 hours. The parameter supports the following format only: `[+|-][HH:mm]` +### **additional options may be added in the future!** # Known limitations - Time Travel - - currently only supports `VERSION AS OF` - - `TIMESTAMP AS OF` not yet supported + - currently only supports `VERSION AS OF` + - `TIMESTAMP AS OF` not yet supported +- complex data types are currently not supported + - map + - struct + - array # Examples The examples below can be used *as-is* in Power BI desktop. If you are prompted for authentication, just select `Anonymous` for your authentication method. -> Note: In the examples the root folder of the Delta Lake table ends with `.delta`. This is not mandatory and can be any path. +> **Note:** In the examples the root folder of the Delta Lake table ends with `.delta`. This is not mandatory and can be any path. ## Using Delta Lake Time Travel -To use Delta Lake Time Travel you need to specify the `Version`-option as part of the second argument. The following example reads the Version 123 of a Delta Lake table from an Azure Blob Storage. +To use Delta Lake Time Travel you need to specify the `Version`-option as part of the second argument. The following example reads the Version 1 of a Delta Lake table from an Azure Blob Storage. ``` let Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), @@ -86,7 +101,7 @@ in ``` ## Using Delta Lake Partition Elimination -Partition Elimination is a crucial feature when working with large amounts of data. Without it, you would need to read the whole table and discard a majority of +Partition Elimination is a crucial feature when working with large amounts of data. Without it, you would need to read the whole table and discard a majority of the rows afterwards which is not very efficient. This can be accomplished by using the `PartitionFilterFunction`-option as part of the second argument. In the example below our table is partitioned by `SalesTerritoryKey` (integer) and we only want to load data from Sales Territories where the `SalesTerritoryKey` is greater or equal to `5`: ``` let Source = AzureStorage.Blobs("https://gbadls01.blob.core.windows.net/public"), @@ -96,6 +111,18 @@ in DeltaTable ``` +## Reading from Azure Data Lake Store Gen1 +To read diretly from an Azure Data Lake Store Gen1 folder, you need to specify the options `UseFileBuffer=true` and `IterateFolderContent=true`: +``` +let + Source = DataLake.Contents("adl://myadlsgen1.azuredatalakestore.net/DeltaSamples/FactInternetSales_part.delta", [PageSize=null]), + DeltaTable = fn_ReadDeltaTable(Source, [UseFileBuffer = true, IterateFolderContent = true]) +in + DeltaTable +``` + +If you are reading from Azure Data Lake Store **Gen2** with `[HierarchicalNavigation = true]` you can also use `IterateFolderContent=true` which may speed up overall performance - but this can vary from case to case so please test this on your own data first! + # FAQ @@ -106,4 +133,9 @@ in -------------------- **Q:** How can I use [Delta Lake Time Travel](https://databricks.com/blog/2019/02/04/introducing-delta-time-travel-for-large-scale-data-lakes.html)? -**A:** The function supports an optional second parameter to supply generic parameters. To query specific version of the Delta Lake table, you can provide a record with the field `Version` and the value of the version you want to query. For example: `fn_ReadDeltaTable(#"Filtered Rows", [Version=123])` +**A:** The function supports an optional second parameter to supply generic parameters. To query specific version of the Delta Lake table, you can provide a record with the field `Version` and the value of the version you want to query. For example, to read Version 123 of your Delta Table, you can use the following M code: `fn_ReadDeltaTable(DeltaTableFolderContents, [Version=123])` + +-------------------- +**Q:** The data source I am using does not work with the `fn_ReadDeltaTable` function - what can I do? + +**A:** Please open a support ticket/issue in the git repository. diff --git a/powerbi/fn_ReadDeltaTable.pq b/powerbi/fn_ReadDeltaTable.pq index 79b120a1edc..d3ea4abe290 100644 --- a/powerbi/fn_ReadDeltaTable.pq +++ b/powerbi/fn_ReadDeltaTable.pq @@ -6,24 +6,34 @@ let let - DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), - PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", (x) => true), - UseFileBuffer = if DeltaTableOptions = null then false else Record.FieldOrDefault(DeltaTableOptions, "UseFileBuffer", false), + DeltaTableVersion = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "Version", null), + PartitionFilterFunction = if DeltaTableOptions = null then (x) => true else if Record.FieldOrDefault(DeltaTableOptions, "PartitionFilterFunction", null) = null then (x) => true else Record.Field(DeltaTableOptions, "PartitionFilterFunction"), + UseFileBuffer = if DeltaTableOptions = null then false else if Record.FieldOrDefault(DeltaTableOptions, "UseFileBuffer", null) = null then false else Record.Field(DeltaTableOptions, "UseFileBuffer"), + IterateFolderContent = if DeltaTableOptions = null then false else if Record.FieldOrDefault(DeltaTableOptions, "IterateFolderContent", null) = null then false else Record.Field(DeltaTableOptions, "IterateFolderContent"), + TimeZoneOffset = if DeltaTableOptions = null then null else Record.FieldOrDefault(DeltaTableOptions, "TimeZoneOffset", null), + TimeZoneOffsetDuration = Duration.FromText(Text.TrimStart(TimeZoneOffset, "+")), - DeltaTableFolderContent_wFullPath = - let - Source = DeltaTableFolderContent, - #"Added Full_Path" = Table.AddColumn(DeltaTableFolderContent, "Full_Path", each Text.Replace([Folder Path] & [Name], "=", "%3D"), Text.Type), - #"Added File_Name" = Table.AddColumn(#"Added Full_Path", "File_Name", each if Text.Length([Extension]) > 0 then List.Last(Text.Split([Full_Path], Delimiter)) else null, type text), - Buffered = Table.Buffer(#"Added File_Name") - in - Buffered, - - #"Delimiter" = - let - Delimiter = if Text.Contains(DeltaTableFolderContent{0}[Folder Path], "//") then "/" else "\" - in - Delimiter, + Delimiter = if Text.Contains(DeltaTableFolderContent{0}[Folder Path], "//") then "/" else "\", + + DeltaTableFolderContent_wFullPath = + let + Source = DeltaTableFolderContent, + + fn_ReadContentRecursive = (tbl as table) as table => + let + subFolders = Table.SelectRows(tbl, each Value.Is(_[Content], type table)), + binaries = Table.SelectRows(tbl, each Value.Is(_[Content], type binary)), + combinedContent = if Table.RowCount(subFolders) > 0 then Table.Combine({binaries, @fn_ReadContentRecursive(Table.Combine(subFolders[Content]))}) else binaries + in + combinedContent, + + Content = if IterateFolderContent then fn_ReadContentRecursive(Source) else Source, + + #"Added Full_Path" = Table.AddColumn(Content, "Full_Path", each Text.Replace([Folder Path] & [Name], "=", "%3D"), Text.Type), + #"Added File_Name" = Table.AddColumn(#"Added Full_Path", "File_Name", each if Text.Length([Extension]) > 0 then List.Last(Text.Split([Full_Path], Delimiter)) else null, type text), + Buffered = Table.Buffer(#"Added File_Name") + in + Buffered, PQ_DataTypes = let @@ -170,25 +180,31 @@ let #"Filtered Rows" = Table.SelectRows(#"Expanded metaData", each ([schemaString] <> null)), JSON = Table.TransformColumns(#"Filtered Rows",{{"schemaString", Json.Document}}), #"Expanded schemaString" = Table.ExpandRecordColumn(JSON, "schemaString", {"fields"}, {"fields"}), - #"Expanded fields" = Table.ExpandListColumn(#"Expanded schemaString", "fields"), - #"Expanded fields1" = Table.ExpandRecordColumn(#"Expanded fields", "fields", {"name", "type", "nullable", "metadata"}, {"name", "type", "nullable", "metadata"}), - #"Added Custom" = Table.Buffer(Table.AddColumn(#"Expanded fields1", "isPartitionedBy", each List.Contains([partitionColumns], [name]), Logical.Type)), - #"Added Custom1" = Table.AddColumn(#"Added Custom", "PBI_DataType", - each if [type] = "long" then [PBI_DataType=Int64.Type, PBI_Text="Int64.Type", PBI_Transformation=Int64.From] + #"Expanded fieldList" = Table.ExpandListColumn(#"Expanded schemaString", "fields"), + #"Expanded fields" = Table.ExpandRecordColumn(#"Expanded fieldList", "fields", {"name", "type", "nullable", "metadata"}, {"name", "type", "nullable", "metadata"}), + #"Added isPartitionedBy" = Table.Buffer(Table.AddColumn(#"Expanded fields", "isPartitionedBy", each List.Contains([partitionColumns], [name]), Logical.Type)), + #"Added PBI_DataType" = Table.AddColumn(#"Added isPartitionedBy", "PBI_DataType", + each if [type] = "string" then [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From] + else if [type] = "long" then [PBI_DataType=Int64.Type, PBI_Text="Int64.Type", PBI_Transformation=Int64.From] else if [type] = "integer" then [PBI_DataType=Int32.Type, PBI_Text="Int32.Type", PBI_Transformation=Int32.From] else if [type] = "short" then [PBI_DataType=Int16.Type, PBI_Text="Int16.Type", PBI_Transformation=Int16.From] else if [type] = "byte" then [PBI_DataType=Int8.Type, PBI_Text="Int8.Type", PBI_Transformation=Int8.From] else if [type] = "float" then [PBI_DataType=Single.Type, PBI_Text="Single.Type", PBI_Transformation=Single.From] else if [type] = "double" then [PBI_DataType=Double.Type, PBI_Text="Double.Type", PBI_Transformation=Double.From] else if [type] = "string" then [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From] - else if [type] = "timestamp" then [PBI_DataType=DateTime.Type, PBI_Text="DateTime.Type", PBI_Transformation=DateTime.From] + else if [type] = "date" then [PBI_DataType=Date.Type, PBI_Text="Date.Type", PBI_Transformation=Date.From] + else if [type] = "timestamp" and TimeZoneOffset = null then [PBI_DataType=DateTime.Type, PBI_Text="DateTime.Type", PBI_Transformation=DateTime.From] + else if [type] = "timestamp" and TimeZoneOffset <> null then [PBI_DataType=DateTimeZone.Type, PBI_Text="DateTimeZone.Type", PBI_Transformation=(x) as nullable datetimezone => DateTime.AddZone(x + TimeZoneOffsetDuration, Duration.Hours(TimeZoneOffsetDuration), Duration.Minutes(TimeZoneOffsetDuration))] else if [type] = "boolean" then [PBI_DataType=Logical.Type, PBI_Text="Logical.Type", PBI_Transformation=Logical.From] - else [PBI_DataType=Text.Type, PBI_Text="Text.Type", PBI_Transformation=Text.From]), - #"Expanded PBI_DataType" = Table.ExpandRecordColumn(#"Added Custom1", "PBI_DataType", {"PBI_DataType", "PBI_Text", "PBI_Transformation"}, {"PBI_DataType", "PBI_Text", "PBI_Transformation"}), - #"Added Custom2" = Table.AddColumn(#"Expanded PBI_DataType", "ChangeDataType", each {[name], [PBI_DataType]}, type list), - #"Added Custom3" = Table.AddColumn(#"Added Custom2", "TableDataType", each [name] & "=" & (if [nullable] then "nullable " else "") & Text.From([PBI_Text]), type text), - #"Added Custom4" = Table.AddColumn(#"Added Custom3", "ColumnTransformation", each {[name], [PBI_Transformation]}, type list), - #"Buffered Fields" = Table.Buffer(#"Added Custom4") + else if [type] = "binary" then [PBI_DataType=Binary.Type, PBI_Text="Binary.Type", PBI_Transformation=Binary.From] + else if [type] = "null" then [PBI_DataType=Any.Type, PBI_Text="Any.Type", PBI_Transformation=(x) as any => x] + else if Text.StartsWith([type], "decimal") then [PBI_DataType=Number.Type, PBI_Text="Number.Type", PBI_Transformation=Number.From] + else [PBI_DataType=Any.Type, PBI_Text="Any.Type", PBI_Transformation=(x) as any => x]), + #"Expanded PBI_DataType" = Table.ExpandRecordColumn(#"Added PBI_DataType", "PBI_DataType", {"PBI_DataType", "PBI_Text", "PBI_Transformation"}, {"PBI_DataType", "PBI_Text", "PBI_Transformation"}), + #"Added ChangeDataType" = Table.AddColumn(#"Expanded PBI_DataType", "ChangeDataType", each {[name], [PBI_DataType]}, type list), + #"Added TableDataType" = Table.AddColumn(#"Added ChangeDataType", "TableDataType", each "#""" & [name] & """=" & (if [nullable] then "nullable " else "") & Text.From([PBI_Text]), type text), + #"Added ColumnTransformation" = Table.AddColumn(#"Added TableDataType", "ColumnTransformation", each {[name], [PBI_Transformation]}, type list), + #"Buffered Fields" = Table.Buffer(#"Added ColumnTransformation") in #"Buffered Fields", @@ -216,7 +232,8 @@ let #"Added Custom1" = Table.AddColumn(BufferFile, "Data", each Parquet.Document([Content]), Expression.Evaluate("type table [" & Text.Combine(metadata_columns[TableDataType], ", ") & "]", PQ_DataTypes)), #"Removed Columns1" = Table.RemoveColumns(#"Added Custom1",{"Content"}), #"Expanded Data" = Table.ExpandTableColumn(#"Removed Columns1", "Data", Table.SelectRows(metadata_columns, each not [isPartitionedBy])[name]), - #"Reordered Columns" = Table.ReorderColumns(#"Expanded Data", metadata_columns[name]) + #"Changed Type" = Table.TransformColumns(#"Expanded Data",Table.SelectRows(metadata_columns, each [type] = "timestamp")[ColumnTransformation]), + #"Reordered Columns" = Table.ReorderColumns(if TimeZoneOffset = null then #"Expanded Data" else #"Changed Type", metadata_columns[name]) in #"Reordered Columns" From b0f4f69f585cdacf4be4f907969afd146a1f77b4 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 20 Sep 2021 08:27:52 -0700 Subject: [PATCH 075/291] [NEW] [2] Checkpoint Prototype (#142) * got basic checkpointing implementation and test working --- .../standalone/internal/Checkpoints.scala | 80 ++++++++++++++++++- .../standalone/internal/DeltaConfig.scala | 24 ++++++ .../standalone/internal/DeltaLogImpl.scala | 13 +++ .../internal/OptimisticTransactionImpl.scala | 13 ++- .../standalone/internal/SnapshotImpl.scala | 5 +- .../data/CloseableParquetDataIterator.scala | 2 +- .../sources/StandaloneHadoopConf.scala | 2 +- .../internal/OptimisticTransactionSuite.scala | 31 ++++++- 8 files changed, 163 insertions(+), 7 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 008462ad92f..507e30edef4 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -18,12 +18,14 @@ package io.delta.standalone.internal import java.io.FileNotFoundException +import com.github.mjakubowski84.parquet4s.ParquetWriter import scala.util.control.NonFatal +import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, SingleAction} import org.apache.hadoop.fs.Path - import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.internal.util.FileNames._ +import org.apache.parquet.hadoop.metadata.CompressionCodecName /** * Records information about a checkpoint. @@ -33,7 +35,7 @@ import io.delta.standalone.internal.util.FileNames._ * @param parts the number of parts when the checkpoint has multiple parts. None if this is a * singular checkpoint */ -private case class CheckpointMetaData( +private[internal] case class CheckpointMetaData( version: Long, size: Long, parts: Option[Int]) @@ -102,6 +104,20 @@ private[internal] trait Checkpoints { loadMetadataFromFile(0) } + /** + * Creates a checkpoint using snapshotToCheckpoint. By default it uses the current log version. + */ + def checkpoint(snapshotToCheckpoint: SnapshotImpl): Unit = { + if (snapshotToCheckpoint.version < 0) { + // TODO throw DeltaErrors.checkpointNonExistTable(dataPath) + } + val checkpointMetaData = Checkpoints.writeCheckpoint(this, snapshotToCheckpoint) + val json = JsonUtils.toJson(checkpointMetaData) + store.write(LAST_CHECKPOINT, Iterator(json), overwrite = true) + + // TODO: doLogCleanup() + } + /** Loads the checkpoint metadata from the _last_checkpoint file. */ private def loadMetadataFromFile(tries: Int): Option[CheckpointMetaData] = { try { @@ -175,3 +191,63 @@ private[internal] trait Checkpoints { complete.keys.toArray.sorted.lastOption } } + +object Checkpoints { + /** + * Writes out the contents of a [[Snapshot]] into a checkpoint file that + * can be used to short-circuit future replays of the log. + * + * Returns the checkpoint metadata to be committed to a file. We will use the value + * in this file as the source of truth of the last valid checkpoint. + */ + private[delta] def writeCheckpoint( + deltaLog: DeltaLogImpl, + snapshot: SnapshotImpl): CheckpointMetaData = { + + // The writing of checkpoints doesn't go through log store, so we need to check with the + // log store and decide whether to use rename. + val useRename = deltaLog.store.isPartialWriteVisible(deltaLog.logPath) + + var checkpointSize = 0L + var numOfAddFiles = 0L + + // Use the string in the closure as Path is not Serializable. + val path = checkpointFileSingular(snapshot.path, snapshot.version).toString + + // TODO ++ snapshot.removeFiles ++ snapshot.addFiles ++ snapshot.transaction + // TODO SingleAction instead of Action? + // do NOT include commitInfo + // see https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-schema + val actions: Seq[SingleAction] = + (Seq(snapshot.protocolScala, snapshot.metadataScala) ++ snapshot.allFilesScala).map(_.wrap) + + val writerOptions = ParquetWriter.Options( + compressionCodecName = CompressionCodecName.SNAPPY, + timeZone = snapshot.readTimeZone // TODO: this should just be timeZone + ) + val writer = ParquetWriter.writer[SingleAction](path, writerOptions) + + try { + // TODO useRename + + actions.foreach { singleAction => + writer.write(singleAction) + checkpointSize += 1 + if (singleAction.add != null) { + numOfAddFiles += 1 + } + } + } finally { + writer.close() + } + + // TODO: more useRename stuff + + if (numOfAddFiles != snapshot.numOfFiles) { + throw new IllegalStateException( + "State of the checkpoint doesn't match that of the snapshot.") + } + + CheckpointMetaData(snapshot.version, checkpointSize, None) + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala new file mode 100644 index 00000000000..f6fbcb6d68e --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala @@ -0,0 +1,24 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package io.delta.standalone.internal + +object DeltaConfig { + // TODO IS_APPEND_ONLY + + // TODO CHECKPOINT_INTERVAL +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index e60992b78b7..6afe61d75b0 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -60,6 +60,14 @@ private[internal] class DeltaLogImpl private( /** Delta History Manager containing version and commit history. */ protected lazy val history = DeltaHistoryManager(this) + /** Returns the checkpoint interval for this log. Not transactional. */ + // TODO: DeltaConfigs.CHECKPOINT_INTERVAL + def checkpointInterval: Int = metadata.configuration.getOrElse("checkpointInterval", "10").toInt + + /////////////////////////////////////////////////////////////////////////// + // Public Java API Methods + /////////////////////////////////////////////////////////////////////////// + override def getPath: Path = dataPath override def getCommitInfoAt(version: Long): CommitInfoJ = { @@ -95,6 +103,10 @@ private[internal] class DeltaLogImpl private( new OptimisticTransactionImpl(this, snapshot) } + /////////////////////////////////////////////////////////////////////////// + // Internal Methods + /////////////////////////////////////////////////////////////////////////// + /** * Run `body` inside `deltaLogLock` lock using `lockInterruptibly` so that the thread can be * interrupted when waiting for the lock. @@ -132,6 +144,7 @@ private[internal] class DeltaLogImpl private( * can remove data such as DELETE/UPDATE/MERGE. */ def assertRemovable(): Unit = { + // TODO: DeltaConfig.IS_APPEND_ONLY if (metadata.configuration.getOrElse("appendOnly", "false").toBoolean) { throw DeltaErrors.modifyAppendOnlyTableException } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index d620a86f179..be29ce870ac 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -196,7 +196,11 @@ private[internal] class OptimisticTransactionImpl( private def postCommit(commitVersion: Long): Unit = { committed = true - // TODO: checkpoint + if (shouldCheckpoint(commitVersion)) { + // We checkpoint the version to be committed to so that no two transactions will checkpoint + // the same version. + deltaLog.checkpoint(deltaLog.getSnapshotForVersionAsOf(commitVersion)) + } } /////////////////////////////////////////////////////////////////////////// @@ -236,4 +240,11 @@ private[internal] class OptimisticTransactionImpl( body } } + + /** + * Returns true if we should checkpoint the version that has just been committed. + */ + private def shouldCheckpoint(committedVersion: Long): Boolean = { + committedVersion != 0 && committedVersion % deltaLog.checkpointInterval == 0 + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index df6e4ff3e9e..7defd0cc3fa 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -51,7 +51,9 @@ private[internal] class SnapshotImpl( import SnapshotImpl._ /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ - private val readTimeZone = { + // TODO: this should be at the log level + // TODO: rename to timeZone + val readTimeZone = { if (hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID) == null) { TimeZone.getDefault } else { @@ -86,6 +88,7 @@ private[internal] class SnapshotImpl( def allFilesScala: Seq[AddFile] = state.activeFiles.values.toSeq def protocolScala: Protocol = state.protocol def metadataScala: Metadata = state.metadata + def numOfFiles: Long = state.numOfFiles private def load(paths: Seq[Path]): Seq[SingleAction] = { paths.map(_.toString).sortWith(_ < _).par.flatMap { path => diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala index bd9cafa82e9..14e93f768aa 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala @@ -31,7 +31,7 @@ import io.delta.standalone.types.StructType * * @param dataFilePaths paths of files to iterate over, not null * @param schema data schema, not null. Used to read and verify the parquet data - * @param timeZoneId time zone ID for data, can be null. Used to ensure proper Date and Timestamp + * @param readTimeZone time zone ID for data, can be null. Used to ensure proper Date and Timestamp * decoding */ private[internal] case class CloseableParquetDataIterator( diff --git a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala index 50d06515766..c541b34a7a2 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala @@ -21,6 +21,6 @@ package io.delta.standalone.internal.sources */ private[internal] object StandaloneHadoopConf { - /** time zone as which time-based parquet values will be decoded */ + /** time zone as which time-based parquet values will be encoded and decoded */ val PARQUET_DATA_TIME_ZONE_ID = "io.delta.standalone.PARQUET_DATA_TIME_ZONE_ID" } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index e1d017f8936..18b8dda7a21 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -21,7 +21,7 @@ import java.util.{Collections, Optional, UUID} import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog, Operation} -import io.delta.standalone.actions.{AddFile => AddFileJ, Format => FormatJ, Metadata => MetadataJ} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Format => FormatJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ} import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} import io.delta.standalone.internal.util.TestUtils._ import org.apache.hadoop.conf.Configuration @@ -44,6 +44,14 @@ class OptimisticTransactionSuite extends FunSuite { val ManualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + def createAddFileJ(path: String): AddFileJ = { + new AddFileJ(path, Collections.emptyMap(), 100, 100, true, null, null) + } + + def createRemoveFileJ(path: String): RemoveFileJ = { + new RemoveFileJ(path, Optional.of(100L), true, false, null, 0, null) + } + test("basic") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -58,6 +66,27 @@ class OptimisticTransactionSuite extends FunSuite { } } + test("checkpoint") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + (1 to 15).foreach { i => + val meta = if (i == 1) metadata :: Nil else Nil + val txn = log.startTransaction() + val file = createAddFileJ(i.toString) :: Nil + val delete: Seq[ActionJ] = if (i > 1) { + createRemoveFileJ(i - 1 toString) :: Nil + } else { + Nil + } + txn.commit((meta ++ delete ++ file).asJava, ManualUpdate, "test-writer-id") + } + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + assert(log2.snapshot.getVersion == 14) + assert(log2.snapshot.getAllFiles.size == 1) + } + } + // TODO: test prepareCommit > assert not already committed // TODO: test prepareCommit > assert user didn't commit a CommitInfo From 6e1b1ec4d368fe33ea3510951b589f30ec53ac3f Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 21 Sep 2021 08:58:49 -0700 Subject: [PATCH 076/291] [DSW] [3] Conflict Checker Prototype (#144) --- .../exceptions/ConcurrentAppendException.java | 23 +++ .../ConcurrentDeleteReadException.java | 23 +++ .../DeltaConcurrentModificationException.java | 25 +++ .../standalone/internal/Checkpoints.scala | 6 +- .../standalone/internal/ConflictChecker.scala | 154 ++++++++++++++++ .../internal/OptimisticTransactionImpl.scala | 165 +++++++++++++++--- .../internal/exception/DeltaErrors.scala | 55 +++++- .../standalone/internal/isolationLevels.scala | 25 +++ .../internal/OptimisticTransactionSuite.scala | 7 + 9 files changed, 450 insertions(+), 33 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java new file mode 100644 index 00000000000..603024e8f01 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class ConcurrentAppendException extends DeltaConcurrentModificationException { + public ConcurrentAppendException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java new file mode 100644 index 00000000000..f5163c7bb6d --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class ConcurrentDeleteReadException extends DeltaConcurrentModificationException { + public ConcurrentDeleteReadException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java new file mode 100644 index 00000000000..5100a6ed202 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java @@ -0,0 +1,25 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +import java.util.ConcurrentModificationException; + +public class DeltaConcurrentModificationException extends ConcurrentModificationException { + public DeltaConcurrentModificationException(String message) { + super(message); + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 507e30edef4..244f17e6099 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -192,7 +192,7 @@ private[internal] trait Checkpoints { } } -object Checkpoints { +private[internal] object Checkpoints { /** * Writes out the contents of a [[Snapshot]] into a checkpoint file that * can be used to short-circuit future replays of the log. @@ -200,9 +200,7 @@ object Checkpoints { * Returns the checkpoint metadata to be committed to a file. We will use the value * in this file as the source of truth of the last valid checkpoint. */ - private[delta] def writeCheckpoint( - deltaLog: DeltaLogImpl, - snapshot: SnapshotImpl): CheckpointMetaData = { + def writeCheckpoint(deltaLog: DeltaLogImpl, snapshot: SnapshotImpl): CheckpointMetaData = { // The writing of checkpoints doesn't go through log store, so we need to check with the // log store and decide whether to use rename. diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala new file mode 100644 index 00000000000..ac67b32b360 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala @@ -0,0 +1,154 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import io.delta.standalone.expressions.Expression +import io.delta.standalone.internal.actions._ +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.util.FileNames + +/** + * A class representing different attributes of current transaction needed for conflict detection. + * + * @param readPredicates - partition predicates by which files have been queried by the transaction + * @param readFiles - specific files that have been seen by the transaction + * @param readWholeTable - whether the whole table was read during the transaction + * @param readAppIds - appIds that have been seen by the transaction + * @param metadata - table metadata for the transaction + * @param actions - delta log actions that the transaction wants to commit + * @param deltaLog - [[DeltaLogImpl]] corresponding to the target table + */ +private[internal] case class CurrentTransactionInfo( + readPredicates: Seq[Expression], + readFiles: Set[AddFile], + readWholeTable: Boolean, + readAppIds: Set[String], + metadata: Metadata, + actions: Seq[Action], + deltaLog: DeltaLogImpl) + +/** + * Summary of the Winning commit against which we want to check the conflict + * @param actions - delta log actions committed by the winning commit + * @param commitVersion - winning commit version + */ +private[internal] case class WinningCommitSummary(actions: Seq[Action], commitVersion: Long) { + val metadataUpdates: Seq[Metadata] = actions.collect { case a: Metadata => a } + val appLevelTransactions: Seq[SetTransaction] = actions.collect { case a: SetTransaction => a } + val protocol: Seq[Protocol] = actions.collect { case a: Protocol => a } + val commitInfo: Option[CommitInfo] = actions.collectFirst { case a: CommitInfo => a }.map( + ci => ci.copy(version = Some(commitVersion))) + val removedFiles: Seq[RemoveFile] = actions.collect { case a: RemoveFile => a } + val addedFiles: Seq[AddFile] = actions.collect { case a: AddFile => a } + val isBlindAppendOption: Option[Boolean] = commitInfo.flatMap(_.isBlindAppend) + val blindAppendAddedFiles: Seq[AddFile] = if (isBlindAppendOption.getOrElse(false)) { + addedFiles + } else { + Seq() + } + val changedDataAddedFiles: Seq[AddFile] = if (isBlindAppendOption.getOrElse(false)) { + Seq() + } else { + addedFiles + } + val onlyAddFiles: Boolean = actions.collect { case f: FileAction => f } + .forall(_.isInstanceOf[AddFile]) +} + +private[internal] class ConflictChecker( + currentTransactionInfo: CurrentTransactionInfo, + winningCommitVersion: Long, + isolationLevel: IsolationLevel) { + + private val winningCommitSummary: WinningCommitSummary = createWinningCommitSummary() + + def checkConflicts(): Unit = { + // TODO checkProtocolCompatibility() + // TODO checkNoMetadataUpdates() + checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn() + checkForDeletedFilesAgainstCurrentTxnReadFiles() + // TODO checkForDeletedFilesAgainstCurrentTxnDeletedFiles() + // TODO checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn() + } + + /** + * Initializes [[WinningCommitSummary]] for the already committed + * transaction (winning transaction). + */ + private def createWinningCommitSummary(): WinningCommitSummary = { + val deltaLog = currentTransactionInfo.deltaLog + val winningCommitActions = deltaLog.store.read( + FileNames.deltaFile(deltaLog.logPath, winningCommitVersion)).map(Action.fromJson) + WinningCommitSummary(winningCommitActions, winningCommitVersion) + } + + /** + * Check if the new files added by the already committed transactions should have been read by + * the current transaction. + */ + def checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn(): Unit = { + // Fail if new files have been added that the txn should have read. + val addedFilesToCheckForConflicts = isolationLevel match { + case Serializable => + winningCommitSummary.changedDataAddedFiles ++ winningCommitSummary.blindAppendAddedFiles + case SnapshotIsolation => + Seq.empty + } + + // TODO + } + + /** + * Check if [[RemoveFile]] actions added by already committed transactions conflicts with files + * read by the current transaction. + */ + def checkForDeletedFilesAgainstCurrentTxnReadFiles(): Unit = { + // Fail if files have been deleted that the txn read. + val readFilePaths = currentTransactionInfo.readFiles.map( + f => f.path -> f.partitionValues).toMap + val deleteReadOverlap = winningCommitSummary.removedFiles + .find(r => readFilePaths.contains(r.path)) + if (deleteReadOverlap.nonEmpty) { + val filePath = deleteReadOverlap.get.path + val partition = getPrettyPartitionMessage(readFilePaths(filePath)) + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath in $partition") + } + if (winningCommitSummary.removedFiles.nonEmpty && currentTransactionInfo.readWholeTable) { + val filePath = winningCommitSummary.removedFiles.head.path + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath") + } + } + + /////////////////////////////////////////////////////////////////////////// + // Helper Methods + /////////////////////////////////////////////////////////////////////////// + + /** A helper function for pretty printing a specific partition directory. */ + private def getPrettyPartitionMessage(partitionValues: Map[String, String]): String = { + val partitionColumns = currentTransactionInfo.metadata.partitionColumns + if (partitionColumns.isEmpty) { + "the root of the table" + } else { + val partition = partitionColumns.map { name => + s"$name=${partitionValues(name)}" + }.mkString("[", ", ", "]") + s"partition ${partition}" + } + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index be29ce870ac..d7235f9b724 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -16,14 +16,16 @@ package io.delta.standalone.internal +import java.nio.file.FileAlreadyExistsException import java.util.ConcurrentModificationException import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import io.delta.standalone.{CommitResult, Operation, OptimisticTransaction} import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.expressions.Expression -import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, Metadata, Protocol, RemoveFile} +import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMergingUtils, SchemaUtils} @@ -31,6 +33,16 @@ import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMerg private[internal] class OptimisticTransactionImpl( deltaLog: DeltaLogImpl, snapshot: SnapshotImpl) extends OptimisticTransaction { + import OptimisticTransactionImpl._ + + /** + * Tracks the data that could have been seen by recording the partition + * predicates by which files have been queried by this transaction. + */ + protected val readPredicates = new ArrayBuffer[Expression] + + /** Tracks specific files that have been seen by this transaction. */ + protected val readFiles = new scala.collection.mutable.HashSet[AddFile] /** Tracks if this transaction has already committed. */ private var committed = false @@ -41,9 +53,15 @@ private[internal] class OptimisticTransactionImpl( /** Stores the updated protocol (if any) that will result from this txn. */ private var newProtocol: Option[Protocol] = None - // Whether this transaction is creating a new table. + /** Whether this transaction is creating a new table. */ private var isCreatingNewTable: Boolean = false + /** + * Tracks the start time since we started trying to write a particular commit. + * Used for logging duration of retried transactions. + */ + private var commitAttemptStartTime: Long = _ + /** The protocol of the snapshot that this transaction is reading at. */ def protocol: Protocol = newProtocol.getOrElse(snapshot.protocolScala) @@ -66,9 +84,32 @@ private[internal] class OptimisticTransactionImpl( // Try to commit at the next version. var finalActions = prepareCommit(actions) + // Find the isolation level to use for this commit + val noDataChanged = actions.collect { case f: FileAction => f.dataChange }.forall(_ == false) + val isolationLevelToUse = if (noDataChanged) { + // If no data has changed (i.e. its is only being rearranged), then SnapshotIsolation + // provides Serializable guarantee. Hence, allow reduced conflict detection by using + // SnapshotIsolation of what the table isolation level is. + SnapshotIsolation + } else { + Serializable + } + + val isBlindAppend = { + val dependsOnFiles = readPredicates.nonEmpty || readFiles.nonEmpty + val onlyAddFiles = + finalActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) + onlyAddFiles && !dependsOnFiles + } + // TODO blind append check & create commitInfo using engineInfo - val commitVersion = doCommit(snapshot.version + 1, finalActions) + commitAttemptStartTime = System.currentTimeMillis() + + val commitVersion = doCommitRetryIteratively( + snapshot.version + 1, + finalActions, + isolationLevelToUse) postCommit(commitVersion) @@ -76,8 +117,14 @@ private[internal] class OptimisticTransactionImpl( } override def markFilesAsRead( - readPredicates: java.lang.Iterable[Expression]): java.util.List[AddFileJ] = { + _readPredicates: java.lang.Iterable[Expression]): java.util.List[AddFileJ] = { + // TODO: PartitionFiltering::filesForScan // TODO + // val partitionFilters = filters.filter { f => + // DeltaTableUtils.isPredicatePartitionColumnsOnly(f, metadata.partitionColumns, spark) + // } + // TODO readPredicates += ... + // TODO readFiles ++= null } @@ -159,6 +206,44 @@ private[internal] class OptimisticTransactionImpl( finalActions } + /** + * Commit `actions` using `attemptVersion` version number. If there are any conflicts that are + * found, we will retry a fixed number of times. + * + * @return the real version that was committed + */ + protected def doCommitRetryIteratively( + attemptVersion: Long, + actions: Seq[Action], + isolationLevel: IsolationLevel): Long = lockCommitIfEnabled { + var tryCommit = true + var commitVersion = attemptVersion + var attemptNumber = 0 + + while (tryCommit) { + try { + if (attemptNumber == 0) { + doCommit(commitVersion, actions) + } else if (attemptNumber > DELTA_MAX_RETRY_COMMIT_ATTEMPTS) { + val totalCommitAttemptTime = System.currentTimeMillis() - commitAttemptStartTime + throw DeltaErrors.maxCommitRetriesExceededException( + attemptNumber, + commitVersion, + attemptVersion, + actions.length, + totalCommitAttemptTime) + } else { + commitVersion = checkForConflicts(commitVersion, actions, isolationLevel) + doCommit(commitVersion, actions) + } + tryCommit = false + } catch { + case _: FileAlreadyExistsException => attemptNumber += 1 + } + } + commitVersion + } + /** * Commit `actions` using `attemptVersion` version number. * @@ -167,27 +252,21 @@ private[internal] class OptimisticTransactionImpl( * @return the real version that was committed. * @throws IllegalStateException if the attempted commit version is ahead of the current delta log * version - * @throws ConcurrentModificationException if any conflicts are detected */ private def doCommit(attemptVersion: Long, actions: Seq[Action]): Long = lockCommitIfEnabled { - try { - deltaLog.store.write( - FileNames.deltaFile(deltaLog.logPath, attemptVersion), - actions.map(_.json).toIterator - ) - - val postCommitSnapshot = deltaLog.update() - if (postCommitSnapshot.version < attemptVersion) { - throw new IllegalStateException( - s"The committed version is $attemptVersion " + - s"but the current version is ${postCommitSnapshot.version}.") - } - - attemptVersion - } catch { - case _: java.nio.file.FileAlreadyExistsException => - throw new DeltaErrors.DeltaConcurrentModificationException("TODO msg") + deltaLog.store.write( + FileNames.deltaFile(deltaLog.logPath, attemptVersion), + actions.map(_.json).toIterator + ) + + val postCommitSnapshot = deltaLog.update() + if (postCommitSnapshot.version < attemptVersion) { + throw new IllegalStateException( + s"The committed version is $attemptVersion " + + s"but the current version is ${postCommitSnapshot.version}.") } + + attemptVersion } /** @@ -203,6 +282,38 @@ private[internal] class OptimisticTransactionImpl( } } + /** + * Looks at actions that have happened since the txn started and checks for logical + * conflicts with the read/writes. If no conflicts are found return the commit version to attempt + * next. + */ + private def checkForConflicts( + checkVersion: Long, + actions: Seq[Action], + commitIsolationLevel: IsolationLevel): Long = { + val nextAttemptVersion = getNextAttemptVersion + + val currentTransactionInfo = CurrentTransactionInfo( + readPredicates = readPredicates, + readFiles = readFiles.toSet, + readWholeTable = false, // TODO readTheWholeTable + readAppIds = Nil.toSet, // TODO: readTxn.toSet, + metadata = metadata, + actions = actions, + deltaLog = deltaLog) + + (checkVersion until nextAttemptVersion).foreach { otherCommitVersion => + val conflictChecker = new ConflictChecker( + currentTransactionInfo, + otherCommitVersion, + commitIsolationLevel) + + conflictChecker.checkConflicts() + } + + nextAttemptVersion + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// @@ -247,4 +358,14 @@ private[internal] class OptimisticTransactionImpl( private def shouldCheckpoint(committedVersion: Long): Boolean = { committedVersion != 0 && committedVersion % deltaLog.checkpointInterval == 0 } + + /** Returns the next attempt version given the last attempted version */ + private def getNextAttemptVersion: Long = { + deltaLog.update() + deltaLog.snapshot.version + 1 + } +} + +private[internal] object OptimisticTransactionImpl { + val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 517aa33ba82..9d58dac7bba 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -17,12 +17,12 @@ package io.delta.standalone.internal.exception import java.io.{FileNotFoundException, IOException} -import java.util.ConcurrentModificationException -import io.delta.standalone.internal.actions.Protocol -import io.delta.standalone.internal.sources.StandaloneHadoopConf -import org.apache.hadoop.fs.Path +import io.delta.standalone.internal.actions.{CommitInfo, Protocol} import io.delta.standalone.types.StructType +import io.delta.standalone.exceptions._ + +import org.apache.hadoop.fs.Path /** A holder object for Delta errors. */ private[internal] object DeltaErrors { @@ -38,9 +38,6 @@ private[internal] object DeltaErrors { |Standalone Reader/Writer ${clientProtocol.simpleString}. Please upgrade to a newer release. |""".stripMargin) - class DeltaConcurrentModificationException(message: String) - extends ConcurrentModificationException(message) - def deltaVersionsNotContiguousException(deltaVersions: Seq[Long]): Throwable = { new IllegalStateException(s"Versions ($deltaVersions) are not contiguous.") } @@ -182,6 +179,50 @@ private[internal] object DeltaErrors { """.stripMargin, cause) } + def concurrentModificationExceptionMsg( + baseMessage: String, + commit: Option[CommitInfo]): String = { + // TODO + "" + } + + def concurrentAppendException( + conflictingCommit: Option[CommitInfo]): ConcurrentAppendException = { + // TODO: include partition? + val message = DeltaErrors.concurrentModificationExceptionMsg( + s"Files were added by a concurrent update. Please try the operation again.", + conflictingCommit) + new ConcurrentAppendException(message) + } + + def concurrentDeleteReadException( + conflictingCommit: Option[CommitInfo], + file: String): ConcurrentDeleteReadException = { + val message = DeltaErrors.concurrentModificationExceptionMsg( + "This transaction attempted to read one or more files that were deleted" + + s" (for example $file) by a concurrent update. Please try the operation again.", + conflictingCommit) + new ConcurrentDeleteReadException(message) + } + + def maxCommitRetriesExceededException( + attemptNumber: Int, + attemptVersion: Long, + initAttemptVersion: Long, + numActions: Int, + totalCommitAttemptTime: Long): Throwable = { + new IllegalStateException( + s"""This commit has failed as it has been tried $attemptNumber times but did not succeed. + |This can be caused by the Delta table being committed continuously by many concurrent + |commits. + | + |Commit started at version: $initAttemptVersion + |Commit failed at version: $attemptVersion + |Number of actions attempted to commit: $numActions + |Total time spent attempting this commit: $totalCommitAttemptTime ms + """.stripMargin) + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala b/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala new file mode 100644 index 00000000000..d35149a482e --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala @@ -0,0 +1,25 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +sealed trait IsolationLevel { + // TODO override def toString: String = this.getClass.getSimpleName.stripSuffix("$") +} + +case object Serializable extends IsolationLevel + +case object SnapshotIsolation extends IsolationLevel diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 18b8dda7a21..3e17dd273f1 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -132,4 +132,11 @@ class OptimisticTransactionSuite extends FunSuite { // TODO: test doCommit > IllegalStateException // TODO: test doCommit > DeltaConcurrentModificationException + + // TODO: test more ConcurrentAppendException + + // TODO: test more ConcurrentDeleteReadException (including readWholeTable) + + // TODO: test checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn with SnapshotIsolation + // i.e. datachange = false } From 2c7ac31355e3f7dd1fb33e46409a14fd18bb4359 Mon Sep 17 00:00:00 2001 From: Denny Lee Date: Tue, 21 Sep 2021 09:52:32 -0700 Subject: [PATCH 077/291] Update CONTRIBUTING.md (#153) Update connectors to match delta-core contributing.md --- CONTRIBUTING.md | 75 ++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 74 insertions(+), 1 deletion(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 9477ac6dbef..2ae4384e037 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1 +1,74 @@ -We happily welcome contributions to Delta Lake Connectors. We use [GitHub Issues](/../../issues/) to track community reported issues and [GitHub Pull Requests ](/../../pulls/) for accepting changes. +We happily welcome contributions to Delta Lake. We use [GitHub Issues](/../../issues/) to track community reported issues and [GitHub Pull Requests ](/../../pulls/) for accepting changes. + +# Governance +Delta lake governance is conducted by the Technical Steering Committee (TSC), which is currently composed of the following members: + - Michael Armbrust (michael.armbrust@gmail.com) + - Reynold Xin (reynoldx@gmail.com) + - Matei Zaharia (matei@cs.stanford.edu) + +The founding technical charter can be found [here](https://delta.io/wp-content/uploads/2019/12/delta-charter.pdf). + +# Communication +Before starting work on a major feature, please reach out to us via GitHub, Slack, email, etc. We will make sure no one else is already working on it and ask you to open a GitHub issue. +A "major feature" is defined as any change that is > 100 LOC altered (not including tests), or changes any user-facing behavior. +We will use the GitHub issue to discuss the feature and come to agreement. +This is to prevent your time being wasted, as well as ours. +The GitHub review process for major features is also important so that organizations with commit access can come to agreement on design. +If it is appropriate to write a design document, the document must be hosted either in the GitHub tracking issue, or linked to from the issue and hosted in a world-readable location. +Specifically, if the goal is to add a new extension, please read the extension policy. +Small patches and bug fixes don't need prior communication. + +# Coding style +We generally follow the Apache Spark Scala Style Guide. + +# Sign your work +The sign-off is a simple line at the end of the explanation for the patch. Your signature certifies that you wrote the patch or otherwise have the right to pass it on as an open-source patch. The rules are pretty simple: if you can certify the below (from developercertificate.org): + +``` +Developer Certificate of Origin +Version 1.1 + +Copyright (C) 2004, 2006 The Linux Foundation and its contributors. +1 Letterman Drive +Suite D4700 +San Francisco, CA, 94129 + +Everyone is permitted to copy and distribute verbatim copies of this +license document, but changing it is not allowed. + + +Developer's Certificate of Origin 1.1 + +By making a contribution to this project, I certify that: + +(a) The contribution was created in whole or in part by me and I + have the right to submit it under the open source license + indicated in the file; or + +(b) The contribution is based upon previous work that, to the best + of my knowledge, is covered under an appropriate open source + license and I have the right under that license to submit that + work with modifications, whether created in whole or in part + by me, under the same open source license (unless I am + permitted to submit under a different license), as indicated + in the file; or + +(c) The contribution was provided directly to me by some other + person who certified (a), (b) or (c) and I have not modified + it. + +(d) I understand and agree that this project and the contribution + are public and that a record of the contribution (including all + personal information I submit with it, including my sign-off) is + maintained indefinitely and may be redistributed consistent with + this project or the open source license(s) involved. +``` + +Then you just add a line to every git commit message: + +``` +Signed-off-by: Joe Smith +Use your real name (sorry, no pseudonyms or anonymous contributions.) +``` + +If you set your `user.name` and `user.email` git configs, you can sign your commit automatically with git commit -s. From d2fcc2adf15120ea7477c2424478763c9f658465 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 21 Sep 2021 09:52:54 -0700 Subject: [PATCH 078/291] Update CONTRIBUTING.md --- CONTRIBUTING.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 2ae4384e037..05765b1e533 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,4 +1,4 @@ -We happily welcome contributions to Delta Lake. We use [GitHub Issues](/../../issues/) to track community reported issues and [GitHub Pull Requests ](/../../pulls/) for accepting changes. +We happily welcome contributions to Delta Lake Connectors. We use [GitHub Issues](/../../issues/) to track community reported issues and [GitHub Pull Requests ](/../../pulls/) for accepting changes. # Governance Delta lake governance is conducted by the Technical Steering Committee (TSC), which is currently composed of the following members: From 48ee239b3ddfbf1034d672439cc20c87862382fb Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Tue, 21 Sep 2021 11:44:55 -0700 Subject: [PATCH 079/291] Add FieldMetadata to StructField (#130) Add column metadata to the schema via a new type FieldMetadata --- .../delta/standalone/types/FieldMetadata.java | 193 ++++++++++++++++++ .../delta/standalone/types/StructField.java | 33 ++- .../internal/util/DataTypeParser.scala | 87 +++++++- .../internal/DeltaDataReaderSuite.scala | 26 +++ 4 files changed, 329 insertions(+), 10 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java diff --git a/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java b/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java new file mode 100644 index 00000000000..7ea46fe02f8 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This file contains code from the Apache Spark project (original license above). + * It contains modifications, which are licensed as follows: + */ + +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.types; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.stream.Collectors; + +/** + * The metadata for a given {@link StructField}. + */ +public final class FieldMetadata { + private final Map metadata; + + private FieldMetadata(Map metadata) { + this.metadata = metadata; + } + + /** + * @return list of the key-value pairs in {@code this}. + */ + public Map getEntries() { + return Collections.unmodifiableMap(metadata); + } + + /** + * @param key the key to check for + * @return True if {@code this} contains a mapping for the given key, False otherwise + */ + public boolean contains(String key) { + return metadata.containsKey(key); + } + + /** + * @param key the key to check for + * @return the value to which the specified key is mapped, or null if there is no mapping for + * the given key + */ + public Object get(String key) { + return metadata.get(key); + } + + @Override + public String toString() { + return metadata.entrySet() + .stream() + .map(entry -> entry.getKey() + "=" + + (entry.getValue().getClass().isArray() ? + Arrays.toString((Object[]) entry.getValue()) : + entry.getValue().toString())) + .collect(Collectors.joining(", ", "{", "}")); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + FieldMetadata that = (FieldMetadata) o; + if (this.metadata.size() != that.metadata.size()) return false; + return this.metadata.entrySet().stream().allMatch(e -> + e.getValue().equals(that.metadata.get(e.getKey())) || + (e.getValue().getClass().isArray() && + that.metadata.get(e.getKey()).getClass().isArray() && + Arrays.equals( + (Object[]) e.getValue(), + (Object[]) that.metadata.get(e.getKey())))); + } + + @Override + public int hashCode() { + return metadata.entrySet() + .stream() + .mapToInt( entry -> (entry.getValue().getClass().isArray() ? + (entry.getKey() == null ? 0 : entry.getKey().hashCode())^ + Arrays.hashCode((Object[]) entry.getValue()) : + entry.hashCode()) + ).sum(); + } + + /** + * @return a new {@code FieldMetadata.Builder} + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder class for FieldMetadata. + */ + public static class Builder { + private Map metadata = new HashMap(); + + public Builder putNull(String key) { + metadata.put(key, null); + return this; + } + + public Builder putLong(String key, long value) { + metadata.put(key, value); + return this; + } + + public Builder putDouble(String key, double value) { + metadata.put(key, value); + return this; + } + + public Builder putBoolean(String key, boolean value) { + metadata.put(key, value); + return this; + } + + public Builder putString(String key, String value) { + metadata.put(key, value); + return this; + } + + public Builder putMetadata(String key, FieldMetadata value) { + metadata.put(key, value); + return this; + } + + public Builder putLongArray(String key, Long[] value) { + metadata.put(key, value); + return this; + } + + public Builder putDoubleArray(String key, Double[] value) { + metadata.put(key, value); + return this; + } + + public Builder putBooleanArray(String key, Boolean[] value) { + metadata.put(key, value); + return this; + } + + public Builder putStringArray(String key, String[] value) { + metadata.put(key, value); + return this; + } + + public Builder putMetadataArray(String key, FieldMetadata[] value) { + metadata.put(key, value); + return this; + } + + /** + * @return a new {@code FieldMetadata} with the same mappings as {@code this} + */ + public FieldMetadata build() { + return new FieldMetadata(this.metadata); + } + } +} diff --git a/standalone/src/main/java/io/delta/standalone/types/StructField.java b/standalone/src/main/java/io/delta/standalone/types/StructField.java index 56cfb9d6d48..6afdfd203a4 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructField.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructField.java @@ -38,7 +38,10 @@ package io.delta.standalone.types; +import java.util.Collections; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * A field inside a {@link StructType}. @@ -47,6 +50,7 @@ public final class StructField { private final String name; private final DataType dataType; private final boolean nullable; + private final FieldMetadata metadata; /** * @param name the name of this field @@ -57,6 +61,20 @@ public StructField(String name, DataType dataType, boolean nullable) { this.name = name; this.dataType = dataType; this.nullable = nullable; + this.metadata = FieldMetadata.builder().build(); + } + + /** + * @param name the name of this field + * @param dataType the data type of this field + * @param nullable indicates if values of this field can be {@code null} values + * @param metadata metadata for this field + */ + public StructField(String name, DataType dataType, boolean nullable, FieldMetadata metadata) { + this.name = name; + this.dataType = dataType; + this.nullable = nullable; + this.metadata = metadata; } /** @@ -90,12 +108,20 @@ public boolean isNullable() { return nullable; } + /** + * @return the metadata for this field + */ + public FieldMetadata getMetadata() { + return metadata; + } + /** * Builds a readable {@code String} representation of this {@code StructField}. */ protected void buildFormattedString(String prefix, StringBuilder builder) { final String nextPrefix = prefix + " |"; - builder.append(String.format("%s-- %s: %s (nullable = %b)\n", prefix, name, dataType.getTypeName(), nullable)); + builder.append(String.format("%s-- %s: %s (nullable = %b) (metadata =%s)\n", + prefix, name, dataType.getTypeName(), nullable, metadata.toString())); DataType.buildFormattedString(dataType, nextPrefix, builder); } @@ -104,11 +130,12 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; StructField that = (StructField) o; - return name.equals(that.name) && dataType.equals(that.dataType) && nullable == that.nullable; + return name.equals(that.name) && dataType.equals(that.dataType) && nullable == that.nullable + && metadata.equals(that.metadata); } @Override public int hashCode() { - return Objects.hash(name, dataType, nullable); + return Objects.hash(name, dataType, nullable, metadata); } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala index 88a13727b24..7f96f708725 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala @@ -39,11 +39,11 @@ package io.delta.standalone.internal.util import io.delta.standalone.types._ - import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.json4s.JsonDSL._ import org.json4s.JsonAST.JValue +import scala.collection.JavaConverters._ private[standalone] object DataTypeParser { @@ -57,7 +57,7 @@ private[standalone] object DataTypeParser { def fromJson(json: String): DataType = parseDataType(parse(json)) - def parseDataType(json: JValue): DataType = json match { + private def parseDataType(json: JValue): DataType = json match { case JString(name) => nameToType(name) @@ -88,7 +88,7 @@ private[standalone] object DataTypeParser { compact(render(dataTypeToJValue(value))) } - def dataTypeToJValue(dataType: DataType): JValue = dataType match { + private def dataTypeToJValue(dataType: DataType): JValue = dataType match { case array: ArrayType => ("type" -> "array") ~ ("elementType" -> dataTypeToJValue(array.getElementType)) ~ @@ -107,14 +107,39 @@ private[standalone] object DataTypeParser { dataType.getTypeName() } - def structFieldToJValue(field: StructField): JValue = { + private def structFieldToJValue(field: StructField): JValue = { val name = field.getName() val dataType = field.getDataType() val nullable = field.isNullable() + val metadata = field.getMetadata() ("name" -> name) ~ ("type" -> dataTypeToJValue(dataType)) ~ - ("nullable" -> nullable) + ("nullable" -> nullable) ~ + ("metadata" -> metadataValueToJValue(metadata)) + } + + private def metadataValueToJValue(value: Any): JValue = { + value match { + case metadata: FieldMetadata => + JObject(metadata.getEntries().entrySet().asScala.map(e => + (e.getKey(), metadataValueToJValue(e.getValue()))).toList) + case arr: Array[Object] => + JArray(arr.toList.map(metadataValueToJValue)) + case x: Long => + JInt(x) + case x: Double => + JDouble(x) + case x: Boolean => + JBool(x) + case x: String => + JString(x) + case null => + JNull + case other => + throw new IllegalArgumentException( + s"Failed to convert ${value.getClass()} instance to JValue.") + } } /** Given the string representation of a type, return its DataType */ @@ -131,11 +156,11 @@ private[standalone] object DataTypeParser { private def parseStructField(json: JValue): StructField = json match { case JSortedObject( - ("metadata", _: JObject), + ("metadata", metadata: JObject), ("name", JString(name)), ("nullable", JBool(nullable)), ("type", dataType: JValue)) => - new StructField(name, parseDataType(dataType), nullable) + new StructField(name, parseDataType(dataType), nullable, parseFieldMetadata(metadata)) case JSortedObject( ("name", JString(name)), ("nullable", JBool(nullable)), @@ -146,6 +171,54 @@ private[standalone] object DataTypeParser { s"Failed to convert the JSON string '${compact(render(other))}' to a field.") } + private def parseFieldMetadata(metadata: JObject): FieldMetadata = { + val builder = FieldMetadata.builder() + metadata.obj.foreach { + case (key, JInt(value)) => + builder.putLong(key, value.toLong) + case(key, JDouble(value)) => + builder.putDouble(key, value) + case (key, JBool(value)) => + builder.putBoolean(key, value) + case (key, JString(value)) => + builder.putString(key, value) + case (key, o: JObject) => + builder.putMetadata(key, parseFieldMetadata(o)) + case (key, JArray(value)) => + if (value.isEmpty) { + // If it is an empty array, we cannot infer its element type. We put an empty Array[Long]. + builder.putLongArray(key, Array.empty) + } else { + value.head match { + case _: JInt => + builder.putLongArray(key, + value.map(_.asInstanceOf[JInt].num.toLong.asInstanceOf[java.lang.Long]).toArray) + case _: JDouble => + builder.putDoubleArray(key, + value.asInstanceOf[List[JDouble]].map(_.num.asInstanceOf[java.lang.Double]).toArray) + case _: JBool => + builder.putBooleanArray(key, + value.asInstanceOf[List[JBool]].map(_.value.asInstanceOf[java.lang.Boolean]) + .toArray) + case _: JString => + builder.putStringArray(key, value.asInstanceOf[List[JString]].map(_.s).toArray) + case _: JObject => + builder.putMetadataArray(key, + value.asInstanceOf[List[JObject]].map(parseFieldMetadata).toArray) + case other => + throw new IllegalArgumentException( + s"Unsupported ${value.head.getClass()} Array as metadata value.") + } + } + case (key, JNull) => + builder.putNull(key) + case (key, other) => + throw new IllegalArgumentException( + s"Unsupported ${other.getClass()} instance as metadata value.") + } + builder.build() + } + private object JSortedObject { def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { case JObject(seq) => Some(seq.sortBy(_._1)) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala index 42c8f0660fe..579529071be 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala @@ -336,4 +336,30 @@ class DeltaDataReaderSuite extends FunSuite { new IntegerType, new ArrayType(new DoubleType, true), false)) + + test("toJson fromJson for field metadata") { + val emptyMetadata = FieldMetadata.builder().build() + val singleStringMetadata = FieldMetadata.builder().putString("test", "test_value").build() + val singleBooleanMetadata = FieldMetadata.builder().putBoolean("test", true).build() + val singleIntegerMetadata = FieldMetadata.builder().putLong("test", 2L).build() + val singleDoubleMetadata = FieldMetadata.builder().putDouble("test", 2.0).build() + val singleMapMetadata = FieldMetadata.builder().putMetadata("test_outside", + FieldMetadata.builder().putString("test_inside", "test_inside_value").build()).build() + val singleListMetadata = FieldMetadata.builder().putLongArray("test", Array(0L, 1L, 2L)).build() + val multipleEntriesMetadata = FieldMetadata.builder().putString("test", "test_value") + .putDouble("test", 2.0).putLongArray("test", Array(0L, 1L, 2L)).build() + + val field_array = Array( + new StructField("emptyMetadata", new BooleanType, true, emptyMetadata), + new StructField("singleStringMetadata", new BooleanType, true, singleStringMetadata), + new StructField("singleBooleanMetadata", new BooleanType, true, singleBooleanMetadata), + new StructField("singleIntegerMetadata", new BooleanType, true, singleIntegerMetadata), + new StructField("singleDoubleMetadata", new BooleanType, true, singleDoubleMetadata), + new StructField("singleMapMetadata", new BooleanType, true, singleMapMetadata), + new StructField("singleListMetadata", new BooleanType, true, singleListMetadata), + new StructField("multipleEntriesMetadata", new BooleanType, true, multipleEntriesMetadata)) + val struct = new StructType(field_array) + assert(struct == DataTypeParser.fromJson(struct.toJson())) + } } + From 2f0a578e3cadd1d2a2187b99c21655dd6305d880 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 21 Sep 2021 13:46:03 -0700 Subject: [PATCH 080/291] [DSW] [4] Improved commit API prototype with tests (#145) --- .../delta/standalone/actions/CommitInfo.java | 2 +- .../internal/OptimisticTransactionImpl.scala | 34 ++- .../standalone/internal/actions/actions.scala | 54 +++++ .../internal/util/ConversionUtils.scala | 6 +- .../internal/util/SchemaMergingUtils.scala | 1 + .../internal/OptimisticTransactionSuite.scala | 206 ++++++++++++++---- 6 files changed, 255 insertions(+), 48 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index ef57ef6bb37..ddc09db40af 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -136,7 +136,7 @@ public String getOperation() { * @return any relevant operation parameters. e.g. "mode", "partitionBy" */ public Map getOperationParameters() { - return Collections.unmodifiableMap(operationParameters); + return null == operationParameters ? null : Collections.unmodifiableMap(operationParameters); } /** diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index d7235f9b724..24ebca3663a 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -17,7 +17,6 @@ package io.delta.standalone.internal import java.nio.file.FileAlreadyExistsException -import java.util.ConcurrentModificationException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -27,8 +26,7 @@ import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Meta import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.sources.StandaloneHadoopConf -import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMergingUtils, SchemaUtils} +import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils, SchemaMergingUtils, SchemaUtils} private[internal] class OptimisticTransactionImpl( deltaLog: DeltaLogImpl, @@ -71,6 +69,9 @@ private[internal] class OptimisticTransactionImpl( */ def metadata: Metadata = newMetadata.getOrElse(snapshot.metadataScala) + /** The version that this transaction is reading from. */ + private def readVersion: Long = snapshot.version + /////////////////////////////////////////////////////////////////////////// // Public Java API Methods /////////////////////////////////////////////////////////////////////////// @@ -82,7 +83,7 @@ private[internal] class OptimisticTransactionImpl( val actions = actionsJ.asScala.map(ConversionUtils.convertActionJ).toSeq // Try to commit at the next version. - var finalActions = prepareCommit(actions) + var preparedActions = prepareCommit(actions) // Find the isolation level to use for this commit val noDataChanged = actions.collect { case f: FileAction => f.dataChange }.forall(_ == false) @@ -98,17 +99,30 @@ private[internal] class OptimisticTransactionImpl( val isBlindAppend = { val dependsOnFiles = readPredicates.nonEmpty || readFiles.nonEmpty val onlyAddFiles = - finalActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) + preparedActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) onlyAddFiles && !dependsOnFiles } - // TODO blind append check & create commitInfo using engineInfo + val commitInfo = CommitInfo( + System.currentTimeMillis(), + op.getName.toString, + null, + Map.empty, + Some(readVersion).filter(_ >= 0), + Option(isolationLevelToUse.toString), + Some(isBlindAppend), + Some(op.getOperationMetrics.asScala.toMap), + if (op.getUserMetadata.isPresent) Some(op.getUserMetadata.get()) else None, + Some(engineInfo) + ) + + preparedActions = commitInfo +: preparedActions commitAttemptStartTime = System.currentTimeMillis() val commitVersion = doCommitRetryIteratively( snapshot.version + 1, - finalActions, + preparedActions, isolationLevelToUse) postCommit(commitVersion) @@ -154,7 +168,7 @@ private[internal] class OptimisticTransactionImpl( val userCommitInfo = actions.exists(_.isInstanceOf[CommitInfo]) assert(!userCommitInfo, - "CommitInfo actions are created automatically. Users shouldn't try to commit them directly") + "Cannot commit a custom CommitInfo in a transaction.") // If the metadata has changed, add that to the set of actions var finalActions = newMetadata.toSeq ++ actions @@ -368,4 +382,8 @@ private[internal] class OptimisticTransactionImpl( private[internal] object OptimisticTransactionImpl { val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 + + def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { + op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala index 4dd43cc2f7e..9e46b665470 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala @@ -270,6 +270,41 @@ private[internal] object CommitInfo { CommitInfo(version, null, None, None, null, null, None, None, None, None, None, None, None, None, None) } + + def apply( + time: Long, + operation: String, + operationParameters: Map[String, String], + commandContext: Map[String, String], + readVersion: Option[Long], + isolationLevel: Option[String], + isBlindAppend: Option[Boolean], + operationMetrics: Option[Map[String, String]], + userMetadata: Option[String], + engineInfo: Option[String]): CommitInfo = { + val getUserName = commandContext.get("user").flatMap { + case "unknown" => None + case other => Option(other) + } + + CommitInfo( + None, + new Timestamp(time), + commandContext.get("userId"), + getUserName, + operation, + operationParameters, + JobInfo.fromContext(commandContext), + NotebookInfo.fromContext(commandContext), + commandContext.get("clusterId"), + readVersion, + isolationLevel, + isBlindAppend, + operationMetrics, + userMetadata, + engineInfo + ) + } } private[internal] case class JobInfo( @@ -279,8 +314,27 @@ private[internal] case class JobInfo( jobOwnerId: String, triggerType: String) +private[internal] object JobInfo { + def fromContext(context: Map[String, String]): Option[JobInfo] = { + context.get("jobId").map { jobId => + JobInfo( + jobId, + context.get("jobName").orNull, + context.get("runId").orNull, + context.get("jobOwnerId").orNull, + context.get("jobTriggerType").orNull) + } + } +} + private[internal] case class NotebookInfo(notebookId: String) +private[internal] object NotebookInfo { + def fromContext(context: Map[String, String]): Option[NotebookInfo] = { + context.get("notebookId").map { nbId => NotebookInfo(nbId) } + } +} + /** A serialization helper to create a common action envelope. */ private[internal] case class SingleAction( txn: SetTransaction = null, diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index bb37a35cd31..d836310b083 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -144,7 +144,7 @@ private[internal] object ConversionUtils { toJavaStringOptional(internal.userId), toJavaStringOptional(internal.userName), internal.operation, - internal.operationParameters.asJava, + nullableMapAsJava(internal.operationParameters), jobInfoOpt, notebookInfoOpt, toJavaStringOptional(internal.clusterId), @@ -256,7 +256,9 @@ private[internal] object ConversionUtils { external.getUserId, // implicit check this! external.getUserName, // implicit check this! external.getOperation, - external.getOperationParameters.asScala.toMap, + if (external.getOperationParameters != null) { + external.getOperationParameters.asScala.toMap + } else null, None, // TODO: Option[JobInfo] None, // TODO: Option[NotebookInfo] external.getClusterId, // implicit check this! diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala index d6cf81427e1..2a76c3eb40d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala @@ -80,6 +80,7 @@ private[internal] object SchemaMergingUtils { case (x, ys) if ys.length > 1 => s"$x" } + // TODO: AnalysisException ? throw new RuntimeException( s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 3e17dd273f1..f8e4f704a91 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -21,28 +21,41 @@ import java.util.{Collections, Optional, UUID} import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog, Operation} -import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Format => FormatJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.internal.actions._ +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.exception.DeltaErrors.InvalidProtocolVersionException +import io.delta.standalone.internal.util.ConversionUtils import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} import io.delta.standalone.internal.util.TestUtils._ import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path // scalastyle:off funsuite import org.scalatest.FunSuite class OptimisticTransactionSuite extends FunSuite { // scalastyle:on funsuite - -// val schema = new StructType(Array( -// new StructField("col1", new IntegerType(), true), -// new StructField("col2", new StringType(), true))) - - val metadata = new MetadataJ(UUID.randomUUID().toString, null, null, new FormatJ(), - Collections.emptyList(), Collections.emptyMap(), Optional.of(100L), new StructType(Array.empty)) - - val add1 = new AddFileJ("fake/path/1", Collections.emptyMap(), 100, 100, true, null, null) - val add2 = new AddFileJ("fake/path/2", Collections.emptyMap(), 100, 100, true, null, null) - - val ManualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + val writerId = "test-writer-id" + val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + + val A_P1 = "part=1/a" + val B_P1 = "part=1/b" + val C_P1 = "part=1/c" + val C_P2 = "part=2/c" + val D_P2 = "part=2/d" + val E_P3 = "part=3/e" + val F_P3 = "part=3/f" + val G_P4 = "part=4/g" + + private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) def createAddFileJ(path: String): AddFileJ = { new AddFileJ(path, Collections.emptyMap(), 100, 100, true, null, null) @@ -52,33 +65,64 @@ class OptimisticTransactionSuite extends FunSuite { new RemoveFileJ(path, Optional.of(100L), true, false, null, 0, null) } - test("basic") { + implicit def actionSeqToList[T <: Action](seq: Seq[T]): java.util.List[ActionJ] = + seq.map(ConversionUtils.convertAction).asJava + + implicit def addFileSeqToList(seq: Seq[AddFile]): java.util.List[AddFileJ] = + seq.map(ConversionUtils.convertAddFile).asJava + + def withLog( + actions: Seq[Action], + partitionCols: Seq[String] = "part" :: Nil)( + test: DeltaLog => Unit): Unit = { + val schemaFields = partitionCols.map { p => new StructField(p, new StringType()) }.toArray + val schema = new StructType(schemaFields) + // TODO val metadata = Metadata(partitionColumns = partitionCols, schemaString = schema.json) + val metadata = Metadata(partitionColumns = partitionCols) withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - val actions = Seq(metadata, add1, add2) - txn.commit(actions.asJava, ManualUpdate, "test-writer-id") + log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) + log.startTransaction().commit(actions, manualUpdate, writerId) + test(log) + } + } + + test("basic commit") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, writerId) + + // [...] is what is automatically added by OptimisticTransaction + // 0 -> metadata [CommitInfo, Protocol] + // 1 -> addA_P1, addB_P1 [CommitInfo] + // 2 -> removeA_P1 [CommitInfo] val versionLogs = log.getChanges(0, true).asScala.toList - val readActions = versionLogs(0).getActions.asScala - assert(actions.toSet.subsetOf(readActions.toSet)) + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[MetadataJ]) == 1) + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[ProtocolJ]) == 1) + + assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[AddFileJ]) == 2) + assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) + + assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[RemoveFileJ]) == 1) + assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) } } - test("checkpoint") { + test("basic checkpoint") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) (1 to 15).foreach { i => - val meta = if (i == 1) metadata :: Nil else Nil + val meta = if (i == 1) Metadata() :: Nil else Nil val txn = log.startTransaction() - val file = createAddFileJ(i.toString) :: Nil - val delete: Seq[ActionJ] = if (i > 1) { - createRemoveFileJ(i - 1 toString) :: Nil + val file = AddFile(i.toString, Map.empty, 1, 1, dataChange = true) :: Nil + val delete: Seq[Action] = if (i > 1) { + RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil } else { Nil } - txn.commit((meta ++ delete ++ file).asJava, ManualUpdate, "test-writer-id") + txn.commit(meta ++ delete ++ file, manualUpdate, writerId) } val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -87,29 +131,117 @@ class OptimisticTransactionSuite extends FunSuite { } } - // TODO: test prepareCommit > assert not already committed + test("committing twice in the same transaction should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + txn.commit(Metadata() :: Nil, manualUpdate, writerId) + val e = intercept[AssertionError] { + txn.commit(Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Transaction already committed.")) + } + } - // TODO: test prepareCommit > assert user didn't commit a CommitInfo + test("user cannot commit their own CommitInfo") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + val e = intercept[AssertionError] { + log.startTransaction().commit(CommitInfo.empty() :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Cannot commit a custom CommitInfo in a transaction.")) + } + } - // TODO: test prepareCommit > have more than 1 Metadata in transaction + test("commits shouldn't have more than one Metadata") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val e = intercept[AssertionError] { + txn.commit(Metadata() :: Metadata() :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) + } + } - // TODO: test prepareCommit > 1st commit & ensureLogDirectoryExist throws + test("transaction should throw if it cannot read log directory during first commit ") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + dir.setReadOnly() - // TODO: test prepareCommit > 1st commit & commitValidationEnabled & metadataAbsentException + val txn = log.startTransaction() + val e = intercept[java.io.IOException] { + txn.commit(Metadata() :: Nil, manualUpdate, writerId) + } + + val logPath = new Path(log.getPath, "_delta_log") + assert(e.getMessage == s"Cannot create ${logPath.toString}") + } + } + + test("first commit must have a Metadata") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val e = intercept[IllegalStateException] { + txn.commit(Nil, manualUpdate, writerId) + } + assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) + } + } - // TODO: test prepareCommit > 1st commit & !commitValidationEnabled & no metadataAbsentException + test("prevent protocol downgrades") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Protocol(1, 2) :: Nil, manualUpdate, writerId) + val e = intercept[RuntimeException] { + log.startTransaction().commit(Protocol(1, 1) :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Protocol version cannot be downgraded")) + } + } - // TODO: test prepareCommit > protocolDowngradeException (reader) + test("AddFile partition mismatches should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - // TODO: test prepareCommit > protocolDowngradeException (writer) + // Note that Metadata() has no partition schema specified + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + val e = intercept[IllegalStateException] { + log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + + "table's partitioning schema")) + } + } - // TODO: test prepareCommit > commitValidationEnabled & addFilePartitioningMismatchException + test("access with protocol too high") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Protocol(1, 2) :: Nil, manualUpdate, writerId) + val txn = log.startTransaction() + txn.commit(Protocol(1, 3) :: Nil, manualUpdate, writerId) - // TODO: test prepareCommit > !commitValidationEnabled & no addFilePartitioningMismatchException + val e = intercept[InvalidProtocolVersionException] { + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Delta protocol version (1,3) is too new for this version")) + } + } - // TODO: test prepareCommit > assertProtocolWrite + test("can't remove from an append-only table") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val metadata = Metadata(configuration = Map("appendOnly" -> "true")) + log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) - // TODO: test prepareCommit > assertRemovable + val e = intercept[UnsupportedOperationException] { + log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("This table is configured to only allow appends")) + } + } // TODO: test verifyNewMetadata > SchemaMergingUtils.checkColumnNameDuplication From 544ff2f9707c0089f583bfa74a8fc238da3732af Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 22 Sep 2021 14:04:17 -0700 Subject: [PATCH 081/291] [DSW] [5] updated internal LogStore (#146) * refactor ReadOnlyLogStore to LogStore; add test suite --- .../standalone/OptimisticTransaction.java | 1 + .../standalone/expressions/Expression.java | 8 +- .../internal/DeltaHistoryManager.scala | 4 +- .../standalone/internal/DeltaLogImpl.scala | 7 +- .../sources/StandaloneHadoopConf.scala | 3 + .../internal/storage/HDFSLogStore.scala | 145 ++++++++++++++++ .../internal/storage/LogStore.scala | 159 ++++++++++++++++++ .../standalone/internal/LogStoreSuite.scala | 159 ++++++++++++++++++ 8 files changed, 477 insertions(+), 9 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index f247ccb5114..29293516152 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -62,6 +62,7 @@ public interface OptimisticTransaction { void readWholeTable(); /** + * @param id TODO * @return the latest version that has committed for the idempotent transaction with given `id`. */ long txnVersion(String id); diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java index c0c58dcb6f6..a7a3816d6e2 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java @@ -11,22 +11,22 @@ public interface Expression { /** - * Returns the result of evaluating this expression on a given input RowRecord. + * @return the result of evaluating this expression on a given input RowRecord. */ Object eval(RowRecord record); /** - * Returns the [[DataType]] of the result of evaluating this expression. + * @return the [[DataType]] of the result of evaluating this expression. */ DataType dataType(); /** - * Returns the String representation of this expression. + * @return the String representation of this expression. */ String toString(); /** - * Returns a List of the children of this node. Children should not change. + * @return a List of the children of this node. Children should not change. */ List children(); } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala index 45803389b77..680e70babd8 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path import io.delta.standalone.internal.actions.{Action, CommitInfo, CommitMarker} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.FileNames -import io.delta.standalone.internal.storage.ReadOnlyLogStore +import io.delta.standalone.internal.storage.LogStore /** * This class keeps tracks of the version of commits and their timestamps for a Delta table to @@ -154,7 +154,7 @@ private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { * Exposed for tests. */ private def getCommits( - logStore: ReadOnlyLogStore, + logStore: LogStore, logPath: Path, start: Long, end: Long): Array[Commit] = { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 6afe61d75b0..71b5bbb7a5d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -29,7 +29,7 @@ import io.delta.standalone.expressions.{And, Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol} import io.delta.standalone.internal.data.PartitionRowRecord import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore +import io.delta.standalone.internal.storage.{HDFSLogStore, LogStoreProvider} import io.delta.standalone.internal.util.{ConversionUtils, FileNames} import io.delta.standalone.types.StructType @@ -42,10 +42,11 @@ private[internal] class DeltaLogImpl private( val dataPath: Path) extends DeltaLog with Checkpoints + with LogStoreProvider with SnapshotManagement { - /** Used to read (not write) physical log files and checkpoints. */ - lazy val store = new HDFSReadOnlyLogStore(hadoopConf) + /** Used to read and write physical log files and checkpoints. */ + lazy val store = createLogStore(hadoopConf) /** Direct access to the underlying storage system. */ private lazy val fs = logPath.getFileSystem(hadoopConf) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala index c541b34a7a2..8af33b0c9b5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala @@ -23,4 +23,7 @@ private[internal] object StandaloneHadoopConf { /** time zone as which time-based parquet values will be encoded and decoded */ val PARQUET_DATA_TIME_ZONE_ID = "io.delta.standalone.PARQUET_DATA_TIME_ZONE_ID" + + /** TODO */ + val LOG_STORE_CLASS_KEY = "io.delta.standalone.LOG_STORE_CLASS_KEY" } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala new file mode 100644 index 00000000000..c99baae37ee --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala @@ -0,0 +1,145 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.storage + +import java.io.{BufferedReader, FileNotFoundException, InputStreamReader, IOException} +import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.FileAlreadyExistsException +import java.util.{EnumSet, UUID} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import io.delta.standalone.internal.exception.DeltaErrors +import org.apache.commons.io.IOUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileContext, FileStatus, Options, Path, RawLocalFileSystem} +import org.apache.hadoop.fs.CreateFlag.CREATE +import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} + +/** + * The [[LogStore]] implementation for HDFS, which uses Hadoop [[FileContext]] APIs to + * provide the necessary atomic and durability guarantees: + * + * 1. Atomic visibility of files: `FileContext.rename` is used write files which is atomic for HDFS. + * + * 2. Consistent file listing: HDFS file listing is consistent. + */ +private[internal] class HDFSLogStore(hadoopConf: Configuration) extends LogStore { + + val noAbstractFileSystemExceptionMessage = "No AbstractFileSystem" + + override def read(path: Path): Seq[String] = { + val fs = path.getFileSystem(hadoopConf) + val stream = fs.open(path) + try { + val reader = new BufferedReader(new InputStreamReader(stream, UTF_8)) + IOUtils.readLines(reader).asScala.map(_.trim) + } finally { + stream.close() + } + } + + override def listFrom(path: Path): Iterator[FileStatus] = { + val fs = path.getFileSystem(hadoopConf) + if (!fs.exists(path.getParent)) { + throw new FileNotFoundException(s"No such file or directory: ${path.getParent}") + } + val files = fs.listStatus(path.getParent) + files.filter(_.getPath.getName >= path.getName).sortBy(_.getPath.getName).iterator + } + + override def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit = { + val isLocalFs = path.getFileSystem(hadoopConf).isInstanceOf[RawLocalFileSystem] + if (isLocalFs) { + // We need to add `synchronized` for RawLocalFileSystem as its rename will not throw an + // exception when the target file exists. Hence we must make sure `exists + rename` in + // `writeInternal` for RawLocalFileSystem is atomic in our tests. + synchronized { + writeInternal(path, actions, overwrite) + } + } else { + // rename is atomic and also will fail when the target file exists. Not need to add the extra + // `synchronized`. + writeInternal(path, actions, overwrite) + } + } + + private def writeInternal(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { + val fc: FileContext = try { + getFileContext(path) + } catch { + case e: IOException if e.getMessage.contains(noAbstractFileSystemExceptionMessage) => + val newException = DeltaErrors.incorrectLogStoreImplementationException(e) + throw newException + } + if (!overwrite && fc.util.exists(path)) { + // This is needed for the tests to throw error with local file system + throw new FileAlreadyExistsException(path.toString) + } + + val tempPath = createTempPath(path) + var streamClosed = false // This flag is to avoid double close + var renameDone = false // This flag is to save the delete operation in most of cases. + val stream = fc.create( + tempPath, EnumSet.of(CREATE), CreateOpts.checksumParam(ChecksumOpt.createDisabled())) + + try { + actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) + stream.close() + streamClosed = true + try { + val renameOpt = if (overwrite) Options.Rename.OVERWRITE else Options.Rename.NONE + fc.rename(tempPath, path, renameOpt) + renameDone = true + // TODO: this is a workaround of HADOOP-16255 - remove this when HADOOP-16255 is resolved + tryRemoveCrcFile(fc, tempPath) + } catch { + case e: org.apache.hadoop.fs.FileAlreadyExistsException => + throw new FileAlreadyExistsException(path.toString) + } + } finally { + if (!streamClosed) { + stream.close() + } + if (!renameDone) { + fc.delete(tempPath, false) + } + } + } + + private def createTempPath(path: Path): Path = { + new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}.tmp") + } + + private def getFileContext(path: Path): FileContext = { + FileContext.getFileContext(path.toUri, hadoopConf) + } + + private def tryRemoveCrcFile(fc: FileContext, path: Path): Unit = { + try { + val checksumFile = new Path(path.getParent, s".${path.getName}.crc") + if (fc.util.exists(checksumFile)) { + // checksum file exists, deleting it + fc.delete(checksumFile, true) + } + } catch { + case NonFatal(_) => // ignore, we are removing crc file as "best-effort" + } + } + +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala new file mode 100644 index 00000000000..887ec6ab7c5 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala @@ -0,0 +1,159 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.storage + +import io.delta.standalone.data.{CloseableIterator => ClosebleIteratorJ} +import io.delta.standalone.storage.{LogStore => LogStoreJ} +import io.delta.standalone.internal.sources.StandaloneHadoopConf + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} + +/** + * General interface for all critical file system operations required to read and write the + * [[io.delta.standalone.DeltaLog]]. The correctness of the [[io.delta.standalone.DeltaLog]] is + * predicated on the atomicity and durability guarantees of the implementation of this interface. + * Specifically, + * + * 1. Atomic visibility of files: Any file written through this store must + * be made visible atomically. In other words, this should not generate partial files. + * + * 2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final + * destination. + * + * 3. Consistent listing: Once a file has been written in a directory, all future listings for + * that directory must return that file. + */ +private[internal] trait LogStore { // TODO: rename and refactor + + /** Read the given `path` */ + def read(path: String): Seq[String] = read(new Path(path)) + + /** Read the given `path` */ + def read(path: Path): Seq[String] + + /** + * List the paths in the same directory that are lexicographically greater or equal to + * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. + */ + def listFrom(path: String): Iterator[FileStatus] = listFrom(new Path(path)) + + /** + * List the paths in the same directory that are lexicographically greater or equal to + * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. + */ + def listFrom(path: Path): Iterator[FileStatus] + + /** + * Write the given `actions` to the given `path` without overwriting any existing file. + * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file + * already exists. Furthermore, implementation must ensure that the entire file is made + * visible atomically, that is, it should not generate partial files. + */ + final def write(path: String, actions: Iterator[String]): Unit = write(new Path(path), actions) + + /** + * Write the given `actions` to the given `path` with or without overwrite as indicated. + * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file + * already exists and overwrite = false. Furthermore, implementation must ensure that the + * entire file is made visible atomically, that is, it should not generate partial files. + */ + def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit + + /** Resolve the fully qualified path for the given `path`. */ + def resolvePathOnPhysicalStorage(path: Path): Path = { + throw new UnsupportedOperationException() + } + + /** + * Whether a partial write is visible when writing to `path`. + * + * As this depends on the underlying file system implementations, we require the input of `path` + * here in order to identify the underlying file system, even though in most cases a log store + * only deals with one file system. + * + * The default value is only provided here for legacy reasons, which will be removed. + * Any LogStore implementation should override this instead of relying on the default. + */ + def isPartialWriteVisible(path: Path): Boolean = true +} + +private[internal] object LogStore extends LogStoreProvider + +private[internal] trait LogStoreProvider { + + val defaultLogStoreClassName = classOf[HDFSLogStore].getName + + def createLogStore(hadoopConf: Configuration): LogStore = { + val logStoreClassName = + hadoopConf.get(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, defaultLogStoreClassName) + + // scalastyle:off classforname + val logStoreClass = + Class.forName(logStoreClassName, true, Thread.currentThread().getContextClassLoader) + // scalastyle:on classforname + + if (classOf[LogStoreJ].isAssignableFrom(logStoreClass)) { + val logStoreImpl = logStoreClass.getConstructor(classOf[Configuration]) + .newInstance(hadoopConf).asInstanceOf[LogStoreJ] + new LogStoreAdaptor(logStoreImpl, hadoopConf) + } else { + logStoreClass.getConstructor(classOf[Configuration]).newInstance(hadoopConf) + .asInstanceOf[LogStore] + } + } +} + +/** + * An adapter from external Java instances of [[LogStoreJ]] to internal Scala instances of + * [[LogStore]]. + */ +private[internal] class LogStoreAdaptor( + logStoreImpl: LogStoreJ, + hadoopConf: Configuration) extends LogStore { + + override def read(path: Path): Seq[String] = { + var iter: ClosebleIteratorJ[String] = null + try { + iter = logStoreImpl.read(path, hadoopConf) + val contents = iter.asScala.toArray + contents + } finally { + if (iter != null) { + iter.close() + } + } + } + + override def write(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { + logStoreImpl.write(path, actions.asJava, overwrite, hadoopConf) + } + + override def listFrom(path: Path): Iterator[FileStatus] = { + logStoreImpl.listFrom(path, hadoopConf).asScala + } + + override def resolvePathOnPhysicalStorage(path: Path): Path = { + logStoreImpl.resolvePathOnPhysicalStorage(path, hadoopConf) + } + + override def isPartialWriteVisible(path: Path): Boolean = { + logStoreImpl.isPartialWriteVisible(path, hadoopConf) + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala new file mode 100644 index 00000000000..9683d5fe73c --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala @@ -0,0 +1,159 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.io.File + +import io.delta.standalone.data.{CloseableIterator => CloseableIteratorJ} +import io.delta.standalone.storage.{LogStore => LogStoreJ} +import io.delta.standalone.internal.sources.StandaloneHadoopConf +import io.delta.standalone.internal.storage.{HDFSLogStore, LogStore, LogStoreAdaptor, LogStoreProvider} +import io.delta.standalone.internal.util.GoldenTableUtils._ +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +/** + * Instead of using Spark in this project to WRITE data and log files for tests, we have + * io.delta.golden.GoldenTables do it instead. During tests, we then refer by name to specific + * golden tables that that class is responsible for generating ahead of time. This allows us to + * focus on READING only so that we may fully decouple from Spark and not have it as a dependency. + * + * See io.delta.golden.GoldenTables for documentation on how to ensure that the needed files have + * been generated. + */ +abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { + // scalastyle:on funsuite + + def logStoreClassName: Option[String] + + def hadoopConf: Configuration = { + val conf = new Configuration() + if (logStoreClassName.isDefined) { + conf.set(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, logStoreClassName.get) + } + conf + } + + test("instantiation") { + val expectedClassName = logStoreClassName.getOrElse(LogStore.defaultLogStoreClassName) + assert(createLogStore(hadoopConf).getClass.getName == expectedClassName) + } + + test("read") { + withGoldenTable("log-store-read") { tablePath => + val logStore = createLogStore(hadoopConf) + + val deltas = Seq(0, 1).map(i => new File(tablePath, i.toString)).map(_.getCanonicalPath) + assert(logStore.read(deltas.head) == Seq("zero", "none")) + assert(logStore.read(deltas(1)) == Seq("one")) + } + } + + test("listFrom") { + withGoldenTable("log-store-listFrom") { tablePath => + val logStore = createLogStore(hadoopConf) + + val deltas = Seq(0, 1, 2, 3, 4) + .map(i => new File(tablePath, i.toString)) + .map(_.toURI) + .map(new Path(_)) + + assert(logStore.listFrom(deltas.head).map(_.getPath.getName) + .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) + assert(logStore.listFrom(deltas(1)).map(_.getPath.getName) + .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) + assert(logStore.listFrom(deltas(2)).map(_.getPath.getName) + .filterNot(_ == "_delta_log").toArray === Seq(2, 3).map(_.toString)) + assert(logStore.listFrom(deltas(3)).map(_.getPath.getName) + .filterNot(_ == "_delta_log").toArray === Seq(3).map(_.toString)) + assert(logStore.listFrom(deltas(4)).map(_.getPath.getName) + .filterNot(_ == "_delta_log").toArray === Nil) + } + } +} + +/** + * Test providing a system-defined (standalone.internal.storage) LogStore. + */ +class HDFSLogStoreSuite extends LogStoreSuiteBase { + override def logStoreClassName: Option[String] = Some(classOf[HDFSLogStore].getName) +} + +/** + * Test not providing a LogStore classname, in which case [[LogStoreProvider]] will use + * the default value. + */ +class DefaultLogStoreSuite extends LogStoreSuiteBase { + override def logStoreClassName: Option[String] = None +} + +/** + * Test having the user provide their own LogStore. + */ +class UserDefinedLogStoreSuite extends LogStoreSuiteBase { + // The actual type of LogStore created will be LogStoreAdaptor. + override def logStoreClassName: Option[String] = Some(classOf[LogStoreAdaptor].getName) + + override def hadoopConf: Configuration = { + val conf = new Configuration() + conf.set(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, classOf[UserDefinedLogStore].getName) + conf + } +} + +/** + * Sample user-defined log store implementing [[LogStoreJ]] + */ +class UserDefinedLogStore(override val initHadoopConf: Configuration) + extends LogStoreJ(initHadoopConf) { + + private val mockImpl = new HDFSLogStore(initHadoopConf) + + override def read(path: Path, hadoopConf: Configuration): CloseableIteratorJ[String] = { + val iter = mockImpl.read(path).iterator + new CloseableIteratorJ[String] { + override def close(): Unit = {} + override def hasNext: Boolean = iter.hasNext + override def next(): String = iter.next + } + } + + override def write( + path: Path, + actions: java.util.Iterator[String], + overwrite: java.lang.Boolean, + hadoopConf: Configuration): Unit = { + mockImpl.write(path, actions.asScala, overwrite) + } + + override def listFrom(path: Path, hadoopConf: Configuration): java.util.Iterator[FileStatus] = { + mockImpl.listFrom(path).asJava + } + + override def resolvePathOnPhysicalStorage(path: Path, hadoopConf: Configuration): Path = { + mockImpl.resolvePathOnPhysicalStorage(path) + } + + override def isPartialWriteVisible(path: Path, hadoopConf: Configuration): java.lang.Boolean = { + mockImpl.isPartialWriteVisible(path) + } +} From 91a2b1bb4c66093a9a7058a4a25e0b25741ca7b8 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 27 Sep 2021 10:08:04 -0700 Subject: [PATCH 082/291] [DSW] [6] updateMetadata() prototype with tests (#147) * implement updateMetadata() function with tests * remove isCommitLockEnabled --- .../io/delta/standalone/actions/Protocol.java | 16 ++ .../internal/OptimisticTransactionImpl.scala | 117 +++++++------ .../standalone/internal/actions/actions.scala | 10 +- .../storage/HDFSReadOnlyLogStore.scala | 145 ---------------- .../internal/storage/ReadOnlyLogStore.scala | 83 --------- .../internal/util/SchemaUtils.scala | 15 ++ .../standalone/internal/DeltaLogSuite.scala | 2 +- .../internal/OptimisticTransactionSuite.scala | 160 +++++++++++++----- .../internal/ReadOnlyLogStoreSuite.scala | 70 -------- 9 files changed, 222 insertions(+), 396 deletions(-) delete mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala delete mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala delete mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java index 1a421ccede9..93b79b9a18d 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java @@ -1,5 +1,7 @@ package io.delta.standalone.actions; +import java.util.Objects; + public class Protocol implements Action { private final int minReaderVersion; private final int minWriterVersion; @@ -16,5 +18,19 @@ public int getMinReaderVersion() { public int getMinWriterVersion() { return minWriterVersion; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Protocol protocol = (Protocol) o; + return minReaderVersion == protocol.minReaderVersion && + minWriterVersion == protocol.minWriterVersion; + } + + @Override + public int hashCode() { + return Objects.hash(minReaderVersion, minWriterVersion); + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 24ebca3663a..838473e9505 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -142,8 +142,43 @@ private[internal] class OptimisticTransactionImpl( null } - override def updateMetadata(metadata: MetadataJ): Unit = { + override def updateMetadata(metadataJ: MetadataJ): Unit = { + assert(newMetadata.isEmpty, + "Cannot change the metadata more than once in a transaction.") + + var latestMetadata = ConversionUtils.convertMetadataJ(metadataJ) + + if (readVersion == -1 || isCreatingNewTable) { + latestMetadata = withGlobalConfigDefaults(latestMetadata) + isCreatingNewTable = true + newProtocol = Some(Protocol()) + } + + latestMetadata = if (snapshot.metadataScala.schemaString == latestMetadata.schemaString) { + // Shortcut when the schema hasn't changed to avoid generating spurious schema change logs. + // It's fine if two different but semantically equivalent schema strings skip this special + // case - that indicates that something upstream attempted to do a no-op schema change, and + // we'll just end up doing a bit of redundant work in the else block. + latestMetadata + } else { + // TODO getJson() + // val fixedSchema = + // SchemaUtils.removeUnenforceableNotNullConstraints(metadata.schema).getJson() + // metadata.copy(schemaString = fixedSchema) + + latestMetadata + } + + // Remove the protocol version properties + val noProtocolVersionConfig = latestMetadata.configuration.filter { + case (Protocol.MIN_READER_VERSION_PROP, _) => false + case (Protocol.MIN_WRITER_VERSION_PROP, _) => false + case _ => true + } + latestMetadata = latestMetadata.copy(configuration = noProtocolVersionConfig) + verifyNewMetadata(latestMetadata) + newMetadata = Some(latestMetadata) } override def readWholeTable(): Unit = { @@ -166,9 +201,8 @@ private[internal] class OptimisticTransactionImpl( private def prepareCommit(actions: Seq[Action]): Seq[Action] = { assert(!committed, "Transaction already committed.") - val userCommitInfo = actions.exists(_.isInstanceOf[CommitInfo]) - assert(!userCommitInfo, - "Cannot commit a custom CommitInfo in a transaction.") + val customCommitInfo = actions.exists(_.isInstanceOf[CommitInfo]) + assert(!customCommitInfo, "Cannot commit a custom CommitInfo in a transaction.") // If the metadata has changed, add that to the set of actions var finalActions = newMetadata.toSeq ++ actions @@ -193,18 +227,14 @@ private[internal] class OptimisticTransactionImpl( } } + val protocolOpt = finalActions.collectFirst{ case p: Protocol => p } + if (protocolOpt.isDefined) { + assert(protocolOpt.get == Protocol(), s"Invalid Protocol ${protocolOpt.get.simpleString}. " + + s"Currently only Protocol readerVersion 1 and writerVersion 2 is supported.") + } + val partitionColumns = metadata.partitionColumns.toSet finalActions.foreach { - case newVersion: Protocol => - require(newVersion.minReaderVersion > 0, "The reader version needs to be greater than 0") - require(newVersion.minWriterVersion > 0, "The writer version needs to be greater than 0") - if (!isCreatingNewTable) { - val currentVersion = snapshot.protocolScala - if (newVersion.minReaderVersion < currentVersion.minReaderVersion || - newVersion.minWriterVersion < currentVersion.minWriterVersion) { - throw DeltaErrors.protocolDowngradeException(currentVersion, newVersion) - } - } case a: AddFile if partitionColumns != a.partitionValues.keySet => throw DeltaErrors.addFilePartitioningMismatchException( a.partitionValues.keySet.toSeq, partitionColumns.toSeq) @@ -229,7 +259,7 @@ private[internal] class OptimisticTransactionImpl( protected def doCommitRetryIteratively( attemptVersion: Long, actions: Seq[Action], - isolationLevel: IsolationLevel): Long = lockCommitIfEnabled { + isolationLevel: IsolationLevel): Long = deltaLog.lockInterruptibly { var tryCommit = true var commitVersion = attemptVersion var attemptNumber = 0 @@ -267,22 +297,23 @@ private[internal] class OptimisticTransactionImpl( * @throws IllegalStateException if the attempted commit version is ahead of the current delta log * version */ - private def doCommit(attemptVersion: Long, actions: Seq[Action]): Long = lockCommitIfEnabled { - deltaLog.store.write( - FileNames.deltaFile(deltaLog.logPath, attemptVersion), - actions.map(_.json).toIterator - ) + private def doCommit(attemptVersion: Long, actions: Seq[Action]): Long = + deltaLog.lockInterruptibly { + deltaLog.store.write( + FileNames.deltaFile(deltaLog.logPath, attemptVersion), + actions.map(_.json).toIterator + ) + + val postCommitSnapshot = deltaLog.update() + if (postCommitSnapshot.version < attemptVersion) { + throw new IllegalStateException( + s"The committed version is $attemptVersion " + + s"but the current version is ${postCommitSnapshot.version}.") + } - val postCommitSnapshot = deltaLog.update() - if (postCommitSnapshot.version < attemptVersion) { - throw new IllegalStateException( - s"The committed version is $attemptVersion " + - s"but the current version is ${postCommitSnapshot.version}.") + attemptVersion } - attemptVersion - } - /** * Perform post-commit operations */ @@ -343,27 +374,7 @@ private[internal] class OptimisticTransactionImpl( case e: RuntimeException => throw DeltaErrors.invalidPartitionColumn(e) } - // TODO: this function is still incomplete - val needsProtocolUpdate = Protocol.checkProtocolRequirements(metadata, protocol) - - if (needsProtocolUpdate.isDefined) { - newProtocol = needsProtocolUpdate - } - } - - private def isCommitLockEnabled: Boolean = { -// TODO: -// spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED).getOrElse( -// deltaLog.store.isPartialWriteVisible(deltaLog.logPath)) - true - } - - private def lockCommitIfEnabled[T](body: => T): T = { - if (isCommitLockEnabled) { - deltaLog.lockInterruptibly(body) - } else { - body - } + Protocol.checkMetadataProtocolProperties(metadata, protocol) } /** @@ -378,6 +389,12 @@ private[internal] class OptimisticTransactionImpl( deltaLog.update() deltaLog.snapshot.version + 1 } + + /** Creates new metadata with global Delta configuration defaults. */ + private def withGlobalConfigDefaults(metadata: Metadata): Metadata = { + // TODO + metadata + } } private[internal] object OptimisticTransactionImpl { @@ -385,5 +402,5 @@ private[internal] object OptimisticTransactionImpl { def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap - } + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala index 9e46b665470..d6ed74328ea 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala @@ -71,19 +71,11 @@ private[internal] object Protocol { val MIN_READER_VERSION_PROP = "delta.minReaderVersion" val MIN_WRITER_VERSION_PROP = "delta.minWriterVersion" - /** - * Verify that the protocol version of the table satisfies the version requirements of all the - * configurations to be set for the table. Returns the minimum required protocol if not. - */ - def checkProtocolRequirements(metadata: Metadata, protocol: Protocol): Option[Protocol] = { + def checkMetadataProtocolProperties(metadata: Metadata, protocol: Protocol): Unit = { assert(!metadata.configuration.contains(MIN_READER_VERSION_PROP), s"Should not have the " + s"protocol version ($MIN_READER_VERSION_PROP) as part of table properties") assert(!metadata.configuration.contains(MIN_WRITER_VERSION_PROP), s"Should not have the " + s"protocol version ($MIN_WRITER_VERSION_PROP) as part of table properties") - - // TODO: requiredMinimumProtocol(...) - - Some(protocol) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala deleted file mode 100644 index 46835f76ca5..00000000000 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSReadOnlyLogStore.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Copyright (2020) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal.storage - -import java.io.{BufferedReader, FileNotFoundException, InputStreamReader, IOException} -import java.nio.charset.StandardCharsets.UTF_8 -import java.nio.file.FileAlreadyExistsException -import java.util.{EnumSet, UUID} - -import scala.collection.JavaConverters._ -import scala.util.control.NonFatal - -import io.delta.standalone.internal.exception.DeltaErrors -import org.apache.commons.io.IOUtils -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileContext, FileStatus, Options, Path, RawLocalFileSystem} -import org.apache.hadoop.fs.CreateFlag.CREATE -import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} - -/** - * The [[LogStore]] implementation for HDFS, which uses Hadoop [[FileContext]] API's to - * provide the necessary atomic and durability guarantees: - * - * 1. Atomic visibility of files: `FileContext.rename` is used write files which is atomic for HDFS. - * - * 2. Consistent file listing: HDFS file listing is consistent. - */ -private[internal] class HDFSReadOnlyLogStore(hadoopConf: Configuration) extends ReadOnlyLogStore { - - val noAbstractFileSystemExceptionMessage = "No AbstractFileSystem" - - override def read(path: Path): Seq[String] = { - val fs = path.getFileSystem(hadoopConf) - val stream = fs.open(path) - try { - val reader = new BufferedReader(new InputStreamReader(stream, UTF_8)) - IOUtils.readLines(reader).asScala.map(_.trim) - } finally { - stream.close() - } - } - - override def listFrom(path: Path): Iterator[FileStatus] = { - val fs = path.getFileSystem(hadoopConf) - if (!fs.exists(path.getParent)) { - throw new FileNotFoundException(s"No such file or directory: ${path.getParent}") - } - val files = fs.listStatus(path.getParent) - files.filter(_.getPath.getName >= path.getName).sortBy(_.getPath.getName).iterator - } - - override def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit = { - val isLocalFs = path.getFileSystem(hadoopConf).isInstanceOf[RawLocalFileSystem] - if (isLocalFs) { - // We need to add `synchronized` for RawLocalFileSystem as its rename will not throw an - // exception when the target file exists. Hence we must make sure `exists + rename` in - // `writeInternal` for RawLocalFileSystem is atomic in our tests. - synchronized { - writeInternal(path, actions, overwrite) - } - } else { - // rename is atomic and also will fail when the target file exists. Not need to add the extra - // `synchronized`. - writeInternal(path, actions, overwrite) - } - } - - private def writeInternal(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { - val fc: FileContext = try { - getFileContext(path) - } catch { - case e: IOException if e.getMessage.contains(noAbstractFileSystemExceptionMessage) => - val newException = DeltaErrors.incorrectLogStoreImplementationException(e) - throw newException - } - if (!overwrite && fc.util.exists(path)) { - // This is needed for the tests to throw error with local file system - throw new FileAlreadyExistsException(path.toString) - } - - val tempPath = createTempPath(path) - var streamClosed = false // This flag is to avoid double close - var renameDone = false // This flag is to save the delete operation in most of cases. - val stream = fc.create( - tempPath, EnumSet.of(CREATE), CreateOpts.checksumParam(ChecksumOpt.createDisabled())) - - try { - actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) - stream.close() - streamClosed = true - try { - val renameOpt = if (overwrite) Options.Rename.OVERWRITE else Options.Rename.NONE - fc.rename(tempPath, path, renameOpt) - renameDone = true - // TODO: this is a workaround of HADOOP-16255 - remove this when HADOOP-16255 is resolved - tryRemoveCrcFile(fc, tempPath) - } catch { - case e: org.apache.hadoop.fs.FileAlreadyExistsException => - throw new FileAlreadyExistsException(path.toString) - } - } finally { - if (!streamClosed) { - stream.close() - } - if (!renameDone) { - fc.delete(tempPath, false) - } - } - } - - private def createTempPath(path: Path): Path = { - new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}.tmp") - } - - private def getFileContext(path: Path): FileContext = { - FileContext.getFileContext(path.toUri, hadoopConf) - } - - private def tryRemoveCrcFile(fc: FileContext, path: Path): Unit = { - try { - val checksumFile = new Path(path.getParent, s".${path.getName}.crc") - if (fc.util.exists(checksumFile)) { - // checksum file exists, deleting it - fc.delete(checksumFile, true) - } - } catch { - case NonFatal(_) => // ignore, we are removing crc file as "best-effort" - } - } - -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala deleted file mode 100644 index 55da34fa669..00000000000 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/ReadOnlyLogStore.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Copyright (2020) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal.storage - -import org.apache.hadoop.fs.{FileStatus, Path} - -/** - * General interface for all critical file system operations required to read and write the - * [[io.delta.standalone.DeltaLog]]. The correctness of the [[io.delta.standalone.DeltaLog]] is - * predicated on the atomicity and durability guarantees of the implementation of this interface. - * Specifically, - * - * 1. Atomic visibility of files: Any file written through this store must - * be made visible atomically. In other words, this should not generate partial files. - * - * 2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final - * destination. - * - * 3. Consistent listing: Once a file has been written in a directory, all future listings for - * that directory must return that file. - */ -private[internal] trait ReadOnlyLogStore { // TODO: rename and refactor - - /** Read the given `path` */ - def read(path: String): Seq[String] = read(new Path(path)) - - /** Read the given `path` */ - def read(path: Path): Seq[String] - - /** - * List the paths in the same directory that are lexicographically greater or equal to - * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. - */ - def listFrom(path: String): Iterator[FileStatus] = listFrom(new Path(path)) - - /** - * List the paths in the same directory that are lexicographically greater or equal to - * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. - */ - def listFrom(path: Path): Iterator[FileStatus] - - /** - * Write the given `actions` to the given `path` without overwriting any existing file. - * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file - * already exists. Furthermore, implementation must ensure that the entire file is made - * visible atomically, that is, it should not generate partial files. - */ - final def write(path: String, actions: Iterator[String]): Unit = write(new Path(path), actions) - - /** - * Write the given `actions` to the given `path` with or without overwrite as indicated. - * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file - * already exists and overwrite = false. Furthermore, implementation must ensure that the - * entire file is made visible atomically, that is, it should not generate partial files. - */ - def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit - - /** - * Whether a partial write is visible when writing to `path`. - * - * As this depends on the underlying file system implementations, we require the input of `path` - * here in order to identify the underlying file system, even though in most cases a log store - * only deals with one file system. - * - * The default value is only provided here for legacy reasons, which will be removed. - * Any LogStore implementation should override this instead of relying on the default. - */ - def isPartialWriteVisible(path: Path): Boolean = true -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala index 14260df9edd..32a3b06f485 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala @@ -17,6 +17,7 @@ package io.delta.standalone.internal.util import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.types.StructType private[internal] object SchemaUtils { @@ -33,6 +34,20 @@ private[internal] object SchemaUtils { names.find(_.contains("\n")).foreach(col => throw DeltaErrors.invalidColumnName(col)) } + /** + * Go through the schema to look for unenforceable NOT NULL constraints. By default we'll throw + * when they're encountered, but if this is suppressed through SQLConf they'll just be silently + * removed. + * + * Note that this should only be applied to schemas created from explicit user DDL - in other + * scenarios, the nullability information may be inaccurate and Delta should always coerce the + * nullability flag to true. + */ + def removeUnenforceableNotNullConstraints(schema: StructType): StructType = { + // TODO + schema + } + private object ParquetSchemaConverter { def checkFieldName(name: String): Unit = { // ,;{}()\n\t= and space are special characters in Parquet schema diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index f6e74ac67ee..bd5a19851f3 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -239,7 +239,7 @@ class DeltaLogSuite extends FunSuite { withLogForGoldenTable("deltalog-invalid-protocol-version") { _ => } } - assert(e.getMessage ===new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, + assert(e.getMessage === new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, Protocol(99)).getMessage) } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index f8e4f704a91..1bb51c265b7 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -16,17 +16,16 @@ package io.delta.standalone.internal -import java.util.{Collections, Optional, UUID} - import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import io.delta.standalone.{DeltaLog, Operation} -import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.exception.DeltaErrors.InvalidProtocolVersionException import io.delta.standalone.internal.util.ConversionUtils -import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} +import io.delta.standalone.types.{StringType, StructField, StructType} import io.delta.standalone.internal.util.TestUtils._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -57,14 +56,6 @@ class OptimisticTransactionSuite extends FunSuite { private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) - def createAddFileJ(path: String): AddFileJ = { - new AddFileJ(path, Collections.emptyMap(), 100, 100, true, null, null) - } - - def createRemoveFileJ(path: String): RemoveFileJ = { - new RemoveFileJ(path, Optional.of(100L), true, false, null, 0, null) - } - implicit def actionSeqToList[T <: Action](seq: Seq[T]): java.util.List[ActionJ] = seq.map(ConversionUtils.convertAction).asJava @@ -88,6 +79,26 @@ class OptimisticTransactionSuite extends FunSuite { } } + /** + * @tparam T expected exception type + */ + def testMetadata[T <: Throwable : ClassTag]( + metadata: Metadata, + expectedExceptionMessageSubStr: String): Unit = { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val e1 = intercept[T] { + log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) + } + assert(e1.getMessage.contains(expectedExceptionMessageSubStr)) + + val e2 = intercept[T] { + log.startTransaction().updateMetadata(ConversionUtils.convertMetadata(metadata)) + } + assert(e2.getMessage.contains(expectedExceptionMessageSubStr)) + } + } + test("basic commit") { withLog(addA_P1 :: addB_P1 :: Nil) { log => log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, writerId) @@ -165,7 +176,7 @@ class OptimisticTransactionSuite extends FunSuite { } } - test("transaction should throw if it cannot read log directory during first commit ") { + test("transaction should throw if it cannot read log directory during first commit") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) dir.setReadOnly() @@ -191,42 +202,45 @@ class OptimisticTransactionSuite extends FunSuite { } } - test("prevent protocol downgrades") { + test("AddFile with different partition schema compared to metadata should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Protocol(1, 2) :: Nil, manualUpdate, writerId) - val e = intercept[RuntimeException] { - log.startTransaction().commit(Protocol(1, 1) :: Nil, manualUpdate, writerId) + + // Note that Metadata() has no partition schema specified and addA_P1 does + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + val e = intercept[IllegalStateException] { + log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, writerId) } - assert(e.getMessage.contains("Protocol version cannot be downgraded")) + assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + + "table's partitioning schema")) } } - test("AddFile partition mismatches should fail") { + // TODO test create a table with protocol too high + test("Can't create table with invalid protocol version") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - // Note that Metadata() has no partition schema specified - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) - val e = intercept[IllegalStateException] { - log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, writerId) + Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => + val e = intercept[AssertionError] { + log.startTransaction().commit(Metadata() :: protocol :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Invalid Protocol")) } - assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + - "table's partitioning schema")) } } - test("access with protocol too high") { + test("can't change protocol to invalid version") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Protocol(1, 2) :: Nil, manualUpdate, writerId) - val txn = log.startTransaction() - txn.commit(Protocol(1, 3) :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Protocol() :: Nil, manualUpdate, writerId) - val e = intercept[InvalidProtocolVersionException] { - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => + val e = intercept[AssertionError] { + log.startTransaction().commit(protocol :: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains("Invalid Protocol")) } - assert(e.getMessage.contains("Delta protocol version (1,3) is too new for this version")) } } @@ -243,17 +257,87 @@ class OptimisticTransactionSuite extends FunSuite { } } - // TODO: test verifyNewMetadata > SchemaMergingUtils.checkColumnNameDuplication + test("can't update metadata more than once in a transaction") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + val e = intercept[AssertionError] { + txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + } + + assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) + } + } + + test("Protocol Action should be automatically added to transaction for new table") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + assert(log.getChanges(0, true).asScala.next().getActions.contains(new ProtocolJ(1, 2))) + } + } + + test("updateMetadata removes Protocol properties from metadata config") { + // Note: These Protocol properties are not currently exposed to the user. However, they + // might be in the future, and nothing is stopping the user now from seeing these + // properties in Delta OSS and adding them to the config map here. + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val metadata = Metadata(configuration = Map( + Protocol.MIN_READER_VERSION_PROP -> "1", + Protocol.MIN_WRITER_VERSION_PROP -> "2" + )) + txn.updateMetadata(ConversionUtils.convertMetadata(metadata)) + txn.commit(Nil, manualUpdate, writerId) + + val writtenConfig = log.update().getMetadata.getConfiguration + assert(!writtenConfig.containsKey(Protocol.MIN_READER_VERSION_PROP)) + assert(!writtenConfig.containsKey(Protocol.MIN_WRITER_VERSION_PROP)) + } + } - // TODO: test verifyNewMetadata > SchemaUtils.checkFieldNames(dataSchema) > ParquetSchemaConverter + test("commit new metadataa with Protocol properties should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + val newMetadata = Metadata(configuration = Map( + Protocol.MIN_READER_VERSION_PROP -> "1", + Protocol.MIN_WRITER_VERSION_PROP -> "2" + )) - // TODO: test verifyNewMetadata > SchemaUtils.checkFieldNames(dataSchema) > invalidColumnName + val e = intercept[AssertionError] { + log.startTransaction().commit(newMetadata:: Nil, manualUpdate, writerId) + } + assert(e.getMessage.contains(s"Should not have the protocol version " + + s"(${Protocol.MIN_READER_VERSION_PROP}) as part of table properties")) + } + } - // TODO: test verifyNewMetadata > ...checkFieldNames(partitionColumns) > ParquetSchemaConverter + test("can't have duplicate column names") { + // TODO: just call myStruct.getJson() + // scalastyle:off + val schemaStr = """{"type":"struct","fields":[{"name":"col1","type":"integer","nullable":true,"metadata":{}},{"name":"col1","type":"integer","nullable":true,"metadata":{}}]}""" + // scalastyle:on + testMetadata[RuntimeException](Metadata(schemaString = schemaStr), "Found duplicate column(s)") + } - // TODO: test verifyNewMetadata > ...checkFieldNames(partitionColumns) > invalidColumnName + test("column names (both data and partition) must be acceptable by parquet") { + // TODO: just call myStruct.getJson() + // test DATA columns + // scalastyle:off + val schemaStr1 = """{"type":"struct","fields":[{"name":"bad;column,name","type":"integer","nullable":true,"metadata":{}}]}""" + // scalastyle:on + testMetadata[RuntimeException](Metadata(schemaString = schemaStr1), + """Attribute name "bad;column,name" contains invalid character(s)""") + + // test PARTITION columns + testMetadata[RuntimeException](Metadata(partitionColumns = "bad;column,name" :: Nil), + "Found partition columns having invalid character(s)") + } - // TODO: test verifyNewMetadata > Protocol.checkProtocolRequirements + // TODO: test updateMetadata > unenforceable not null constraints removed from metadata schemaStr // TODO: test commit // - commitInfo is actually added to final actions diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala deleted file mode 100644 index 92a90f5959d..00000000000 --- a/standalone/src/test/scala/io/delta/standalone/internal/ReadOnlyLogStoreSuite.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright (2020) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal - -import java.io.File - -import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore -import io.delta.standalone.internal.util.GoldenTableUtils._ -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -import org.scalatest.FunSuite - -/** - * Instead of using Spark in this project to WRITE data and log files for tests, we have - * io.delta.golden.GoldenTables do it instead. During tests, we then refer by name to specific - * golden tables that that class is responsible for generating ahead of time. This allows us to - * focus on READING only so that we may fully decouple from Spark and not have it as a dependency. - * - * See io.delta.golden.GoldenTables for documentation on how to ensure that the needed files have - * been generated. - */ -class ReadOnlyLogStoreSuite extends FunSuite { - // scalastyle:on funsuite - - test("read") { - withGoldenTable("log-store-read") { tablePath => - val readStore = new HDFSReadOnlyLogStore(new Configuration()) - - val deltas = Seq(0, 1).map(i => new File(tablePath, i.toString)).map(_.getCanonicalPath) - assert(readStore.read(deltas.head) == Seq("zero", "none")) - assert(readStore.read(deltas(1)) == Seq("one")) - } - } - - test("listFrom") { - withGoldenTable("log-store-listFrom") { tablePath => - val readStore = new HDFSReadOnlyLogStore(new Configuration()) - val deltas = Seq(0, 1, 2, 3, 4) - .map(i => new File(tablePath, i.toString)) - .map(_.toURI) - .map(new Path(_)) - - assert(readStore.listFrom(deltas.head).map(_.getPath.getName) - .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) - assert(readStore.listFrom(deltas(1)).map(_.getPath.getName) - .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) - assert(readStore.listFrom(deltas(2)).map(_.getPath.getName) - .filterNot(_ == "_delta_log").toArray === Seq(2, 3).map(_.toString)) - assert(readStore.listFrom(deltas(3)).map(_.getPath.getName) - .filterNot(_ == "_delta_log").toArray === Seq(3).map(_.toString)) - assert(readStore.listFrom(deltas(4)).map(_.getPath.getName) - .filterNot(_ == "_delta_log").toArray === Nil) - } - } -} From 8e5c79c41b5fcb6f5cb56073c802ced9daedf145 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 28 Sep 2021 15:47:44 -0700 Subject: [PATCH 083/291] Add Snapshot.scan API (#156) --- .../java/io/delta/standalone/DeltaScan.java | 52 +++++++++++++++++++ .../java/io/delta/standalone/Snapshot.java | 17 +++++- .../standalone/expressions/Expression.java | 44 ++++++++++++++++ .../standalone/internal/DeltaScanImpl.scala | 52 +++++++++++++++++++ .../standalone/internal/SnapshotImpl.scala | 13 ++++- 5 files changed, 175 insertions(+), 3 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/DeltaScan.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Expression.java create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala diff --git a/standalone/src/main/java/io/delta/standalone/DeltaScan.java b/standalone/src/main/java/io/delta/standalone/DeltaScan.java new file mode 100644 index 00000000000..0c14d7b7be8 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/DeltaScan.java @@ -0,0 +1,52 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone; + +import io.delta.standalone.actions.AddFile; +import io.delta.standalone.data.CloseableIterator; +import io.delta.standalone.expressions.Expression; + +import java.util.Optional; + +public interface DeltaScan { + + /** + * Creates a {@link CloseableIterator} which can iterate over files belonging to this snapshot. + * + * It provides no iteration ordering guarantee among files. + * + * Files returned are guaranteed to satisfy the predicate, if any, returned by + * {@link #getPushedPredicate()}. + * + * @return a {@link CloseableIterator} to iterate over files. + */ + CloseableIterator getFiles(); + + /** + * @return portion of the input predicate that can be evaluated by Delta Standalone using only + * metadata. Files returned by {@link #getFiles()} are guaranteed to satisfy the pushed + * predicate, and the caller doesn’t need to apply them again on the returned files. + */ + Optional getPushedPredicate(); + + /** + * @return portion of the input predicate that may not be fully applied. Files returned by + * {@link #getFiles()} are not guaranteed to satisfy the residual predicate, and the + * caller should still apply them on the returned files. + */ + Optional getResidualPredicate(); +} diff --git a/standalone/src/main/java/io/delta/standalone/Snapshot.java b/standalone/src/main/java/io/delta/standalone/Snapshot.java index 1e14acfb633..83715d24c39 100644 --- a/standalone/src/main/java/io/delta/standalone/Snapshot.java +++ b/standalone/src/main/java/io/delta/standalone/Snapshot.java @@ -13,14 +13,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.delta.standalone; -import java.util.List; +package io.delta.standalone; import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.Metadata; import io.delta.standalone.data.CloseableIterator; import io.delta.standalone.data.RowRecord; +import io.delta.standalone.expressions.Expression; + +import java.util.List; /** * {@link Snapshot} provides APIs to access the Delta table state (such as table metadata, active @@ -31,6 +33,17 @@ */ public interface Snapshot { + /** + * @return a {@link DeltaScan} of the files in this snapshot. + */ + DeltaScan scan(); + + /** + * @param predicate the predicate to be used to filter the files in this snapshot. + * @return a {@link DeltaScan} of the files in this snapshot. + */ + DeltaScan scan(Expression predicate); + /** * @return all of the files present in this snapshot */ diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java new file mode 100644 index 00000000000..1830a1a56b8 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java @@ -0,0 +1,44 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.DataType; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * An expression in Delta Standalone. + */ +public interface Expression { + + /** + * @param record the input record to evaluate. + * @return the result of evaluating this expression on the given input {@link RowRecord}. + */ + Object eval(RowRecord record); + + /** + * @return the {@link DataType} of the result of evaluating this expression. + */ + DataType dataType(); + + /** + * @return the String representation of this expression. + */ + String toString(); + + /** + * @return the names of columns referenced by this expression. + */ + default Set references() { + Set result = new HashSet<>(); + children().forEach(child -> result.addAll(child.references())); + return result; + } + + /** + * @return a {@link List} of the children of this node. Children should not change. + */ + List children(); +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala new file mode 100644 index 00000000000..4622090b69f --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala @@ -0,0 +1,52 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.Optional + +import io.delta.standalone.DeltaScan +import io.delta.standalone.actions.{AddFile => AddFileJ} +import io.delta.standalone.data.CloseableIterator +import io.delta.standalone.expressions.Expression + +/** + * Scala implementation of Java interface [[DeltaScan]]. + * + * TODO this is currently a naive implementation, since + * a) it takes in the in-memory AddFiles. + * b) it uses the metadata.partitionColumns, but the metadata won't be known until the log files + * are scanned + */ +final class DeltaScanImpl( + files: java.util.List[AddFileJ], + expr: Option[Expression] = None) extends DeltaScan { + + override def getFiles: CloseableIterator[AddFileJ] = new CloseableIterator[AddFileJ] { + private val iter = files.iterator + + override def hasNext: Boolean = iter.hasNext + + override def next(): AddFileJ = iter.next() + + override def close(): Unit = { } + } + + override def getPushedPredicate: Optional[Expression] = Optional.empty() + + override def getResidualPredicate: Optional[Expression] = Optional.ofNullable(expr.orNull) + +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 14afd9552c1..8c2734fa9e4 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -24,9 +24,10 @@ import scala.collection.JavaConverters._ import com.github.mjakubowski84.parquet4s.ParquetReader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import io.delta.standalone.Snapshot +import io.delta.standalone.{DeltaScan, Snapshot} import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} +import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors @@ -63,6 +64,11 @@ private[internal] class SnapshotImpl( // Public API Methods /////////////////////////////////////////////////////////////////////////// + override def scan(): DeltaScan = new DeltaScanImpl(activeFiles) + + override def scan(predicate: Expression): DeltaScan = + new DeltaScanImpl(activeFiles, Some(predicate)) + override def getAllFiles: java.util.List[AddFileJ] = activeFiles override def getMetadata: MetadataJ = ConversionUtils.convertMetadata(state.metadata) @@ -218,4 +224,9 @@ private class InitialSnapshotImpl( Map.empty[URI, AddFile], 0L, 0L, 1L, 1L) } + + override def scan(): DeltaScan = new DeltaScanImpl(Nil.asJava) + + override def scan(predicate: Expression): DeltaScan = + new DeltaScanImpl(Nil.asJava, Some(predicate)) } From 23894b04dab196b273b502d6afa543068b8d000e Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 29 Sep 2021 09:12:25 -0700 Subject: [PATCH 084/291] [DSW] [7] checkpoint prototype v2 (with tests; with metadata cleanup) (#148) --- .../BufferingLogDeletionIterator.scala | 166 +++++++++++++++ .../standalone/internal/Checkpoints.scala | 70 +++++-- .../standalone/internal/DeltaConfig.scala | 6 + .../standalone/internal/DeltaLogImpl.scala | 39 +++- .../standalone/internal/MetadataCleanup.scala | 92 ++++++++ .../internal/OptimisticTransactionImpl.scala | 8 +- .../standalone/internal/SnapshotImpl.scala | 38 ++-- .../internal/SnapshotManagement.scala | 1 + .../internal/actions/InMemoryLogReplay.scala | 20 +- .../internal/exception/DeltaErrors.scala | 6 - .../standalone/internal/util/Clock.scala | 100 +++++++++ .../internal/util/ManualClock.scala | 70 +++++++ .../BufferLogDeletionIteratorSuite.scala | 21 ++ .../standalone/internal/DeltaLogSuite.scala | 6 + .../internal/DeltaRetentionSuite.scala | 198 ++++++++++++++++++ .../internal/DeltaRetentionSuiteBase.scala | 44 ++++ .../internal/OptimisticTransactionSuite.scala | 53 +++-- .../standalone/internal/util/TestUtils.scala | 11 + 18 files changed, 884 insertions(+), 65 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/BufferingLogDeletionIterator.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/Clock.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/ManualClock.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala diff --git a/standalone/src/main/scala/io/delta/standalone/internal/BufferingLogDeletionIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/BufferingLogDeletionIterator.scala new file mode 100644 index 00000000000..55b0e29c5d0 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/BufferingLogDeletionIterator.scala @@ -0,0 +1,166 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +/** + * An iterator that helps select old log files for deletion. It takes the input iterator of log + * files from the earliest file, and returns should-be-deleted files until the given maxTimestamp + * or maxVersion to delete is reached. Note that this iterator may stop deleting files earlier + * than maxTimestamp or maxVersion if it finds that files that need to be preserved for adjusting + * the timestamps of subsequent files. Let's go through an example. Assume the following commit + * history: + * + * +---------+-----------+--------------------+ + * | Version | Timestamp | Adjusted Timestamp | + * +---------+-----------+--------------------+ + * | 0 | 0 | 0 | + * | 1 | 5 | 5 | + * | 2 | 10 | 10 | + * | 3 | 7 | 11 | + * | 4 | 8 | 12 | + * | 5 | 14 | 14 | + * +---------+-----------+--------------------+ + * + * As you can see from the example, we require timestamps to be monotonically increasing with + * respect to the version of the commit, and each commit to have a unique timestamp. If we have + * a commit which doesn't obey one of these two requirements, we adjust the timestamp of that + * commit to be one millisecond greater than the previous commit. + * + * Given the above commit history, the behavior of this iterator will be as follows: + * - For maxVersion = 1 and maxTimestamp = 9, we can delete versions 0 and 1 + * - Until we receive maxVersion >= 4 and maxTimestamp >= 12, we can't delete versions 2 and 3. + * This is because version 2 is used to adjust the timestamps of commits up to version 4. + * - For maxVersion >= 5 and maxTimestamp >= 14 we can delete everything + * The semantics of time travel guarantee that for a given timestamp, the user will ALWAYS get the + * same version. Consider a user asks to get the version at timestamp 11. If all files are there, + * we would return version 3 (timestamp 11) for this query. If we delete versions 0-2, the + * original timestamp of version 3 (7) will not have an anchor to adjust on, and if the time + * travel query is re-executed we would return version 4. This is the motivation behind this + * iterator implementation. + * + * The implementation maintains an internal "maybeDelete" buffer of files that we are unsure of + * deleting because they may be necessary to adjust time of future files. For each file we get + * from the underlying iterator, we check whether it needs time adjustment or not. If it does need + * time adjustment, then we cannot immediately decide whether it is safe to delete that file or + * not and therefore we put it in each the buffer. Then we iteratively peek ahead at the future + * files and accordingly decide whether to delete all the buffered files or retain them. + * + * @param underlying The iterator which gives the list of files in ascending version order + * @param maxTimestamp The timestamp until which we can delete (inclusive). + * @param maxVersion The version until which we can delete (inclusive). + * @param versionGetter A method to get the commit version from the file path. + */ +class BufferingLogDeletionIterator( + underlying: Iterator[FileStatus], + maxTimestamp: Long, + maxVersion: Long, + versionGetter: Path => Long) extends Iterator[FileStatus] {/** + * Our output iterator + */ + private val filesToDelete = new mutable.Queue[FileStatus]() + /** + * Our intermediate buffer which will buffer files as long as the last file requires a timestamp + * adjustment. + */ + private val maybeDeleteFiles = new mutable.ArrayBuffer[FileStatus]() + private var lastFile: FileStatus = _ + private var hasNextCalled: Boolean = false + + private def init(): Unit = { + if (underlying.hasNext) { + lastFile = underlying.next() + maybeDeleteFiles.append(lastFile) + } + } + + init() + + /** Whether the given file can be deleted based on the version and retention timestamp input. */ + private def shouldDeleteFile(file: FileStatus): Boolean = { + file.getModificationTime <= maxTimestamp && versionGetter(file.getPath) <= maxVersion + } + + /** + * Files need a time adjustment if their timestamp isn't later than the lastFile. + */ + private def needsTimeAdjustment(file: FileStatus): Boolean = { + versionGetter(lastFile.getPath) < versionGetter(file.getPath) && + lastFile.getModificationTime >= file.getModificationTime + } + + /** + * Enqueue the files in the buffer if the last file is safe to delete. Clears the buffer. + */ + private def flushBuffer(): Unit = { + if (maybeDeleteFiles.lastOption.exists(shouldDeleteFile)) { + filesToDelete.enqueue(maybeDeleteFiles: _*) + } + maybeDeleteFiles.clear() + } + + /** + * Peeks at the next file in the iterator. Based on the next file we can have three + * possible outcomes: + * - The underlying iterator returned a file, which doesn't require timestamp adjustment. If + * the file in the buffer has expired, flush the buffer to our output queue. + * - The underlying iterator returned a file, which requires timestamp adjustment. In this case, + * we add this file to the buffer and fetch the next file + * - The underlying iterator is empty. In this case, we check the last file in the buffer. If + * it has expired, then flush the buffer to the output queue. + * Once this method returns, the buffer is expected to have 1 file (last file of the + * underlying iterator) unless the underlying iterator is fully consumed. + */ + private def queueFilesInBuffer(): Unit = { + var continueBuffering = true + while (continueBuffering) { + if (!underlying.hasNext) { + flushBuffer() + return + } + + var currentFile = underlying.next() + require(currentFile != null, "FileStatus iterator returned null") + if (needsTimeAdjustment(currentFile)) { + currentFile = new FileStatus( + currentFile.getLen, currentFile.isDirectory, currentFile.getReplication, + currentFile.getBlockSize, lastFile.getModificationTime + 1, currentFile.getPath) + maybeDeleteFiles.append(currentFile) + } else { + flushBuffer() + maybeDeleteFiles.append(currentFile) + continueBuffering = false + } + lastFile = currentFile + } + } + + override def hasNext: Boolean = { + hasNextCalled = true + if (filesToDelete.isEmpty) queueFilesInBuffer() + filesToDelete.nonEmpty + } + + override def next(): FileStatus = { + if (!hasNextCalled) throw new NoSuchElementException() + hasNextCalled = false + filesToDelete.dequeue() + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 244f17e6099..d9a4424bdcd 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -17,6 +17,7 @@ package io.delta.standalone.internal import java.io.FileNotFoundException +import java.util.UUID import com.github.mjakubowski84.parquet4s.ParquetWriter import scala.util.control.NonFatal @@ -104,6 +105,11 @@ private[internal] trait Checkpoints { loadMetadataFromFile(0) } + /** + * Creates a checkpoint using the default snapshot. + */ + def checkpoint(): Unit = checkpoint(snapshot) + /** * Creates a checkpoint using snapshotToCheckpoint. By default it uses the current log version. */ @@ -115,7 +121,7 @@ private[internal] trait Checkpoints { val json = JsonUtils.toJson(checkpointMetaData) store.write(LAST_CHECKPOINT, Iterator(json), overwrite = true) - // TODO: doLogCleanup() + doLogCleanup() } /** Loads the checkpoint metadata from the _last_checkpoint file. */ @@ -207,41 +213,77 @@ private[internal] object Checkpoints { val useRename = deltaLog.store.isPartialWriteVisible(deltaLog.logPath) var checkpointSize = 0L - var numOfAddFiles = 0L + var numOfFiles = 0L // Use the string in the closure as Path is not Serializable. val path = checkpointFileSingular(snapshot.path, snapshot.version).toString - // TODO ++ snapshot.removeFiles ++ snapshot.addFiles ++ snapshot.transaction - // TODO SingleAction instead of Action? - // do NOT include commitInfo - // see https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-schema - val actions: Seq[SingleAction] = - (Seq(snapshot.protocolScala, snapshot.metadataScala) ++ snapshot.allFilesScala).map(_.wrap) + val actions: Seq[SingleAction] = ( + Seq(snapshot.metadataScala, snapshot.protocolScala) ++ + snapshot.setTransactions ++ + snapshot.allFilesScala ++ + snapshot.tombstonesScala).map(_.wrap) + + val writtenPath = + if (useRename) { + val p = new Path(path) + // Two instances of the same task may run at the same time in some cases (e.g., + // speculation, stage retry), so generate the temp path here to avoid two tasks + // using the same path. + val tempPath = new Path(p.getParent, s".${p.getName}.${UUID.randomUUID}.tmp") + tempPath.toString + } else { + path + } val writerOptions = ParquetWriter.Options( compressionCodecName = CompressionCodecName.SNAPPY, timeZone = snapshot.readTimeZone // TODO: this should just be timeZone ) - val writer = ParquetWriter.writer[SingleAction](path, writerOptions) + val writer = ParquetWriter.writer[SingleAction](writtenPath, writerOptions) try { - // TODO useRename - actions.foreach { singleAction => writer.write(singleAction) checkpointSize += 1 if (singleAction.add != null) { - numOfAddFiles += 1 + numOfFiles += 1 } } + } catch { + case e: org.apache.hadoop.fs.FileAlreadyExistsException if !useRename => + val p = new Path(writtenPath) + if (p.getFileSystem(deltaLog.hadoopConf).exists(p)) { + // The file has been written by a zombie task. We can just use this checkpoint file + // rather than failing a Delta commit. + } else { + throw e + } } finally { writer.close() } - // TODO: more useRename stuff + if (useRename) { + val src = new Path(writtenPath) + val dest = new Path(path) + val fs = dest.getFileSystem(deltaLog.hadoopConf) + var renameDone = false + try { + if (fs.rename(src, dest)) { + renameDone = true + } else { + // There should be only one writer writing the checkpoint file, so there must be + // something wrong here. + throw new IllegalStateException(s"Cannot rename $src to $dest") + } + } finally { + if (!renameDone) { + fs.delete(src, false) + } + } + } - if (numOfAddFiles != snapshot.numOfFiles) { + if (numOfFiles != snapshot.numOfFiles) { throw new IllegalStateException( "State of the checkpoint doesn't match that of the snapshot.") } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala index f6fbcb6d68e..1e6c83ce543 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala @@ -21,4 +21,10 @@ object DeltaConfig { // TODO IS_APPEND_ONLY // TODO CHECKPOINT_INTERVAL + + // TODO: LOG_RETENTION + + // TODO: TOMBSTONE_RETENTION + + // TODO: ENABLE_EXPIRED_LOG_CLEANUP } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 71b5bbb7a5d..feb10b9b611 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -29,8 +29,8 @@ import io.delta.standalone.expressions.{And, Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol} import io.delta.standalone.internal.data.PartitionRowRecord import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.storage.{HDFSLogStore, LogStoreProvider} -import io.delta.standalone.internal.util.{ConversionUtils, FileNames} +import io.delta.standalone.internal.storage.LogStoreProvider +import io.delta.standalone.internal.util.{Clock, ConversionUtils, FileNames, SystemClock} import io.delta.standalone.types.StructType /** @@ -39,9 +39,11 @@ import io.delta.standalone.types.StructType private[internal] class DeltaLogImpl private( val hadoopConf: Configuration, val logPath: Path, - val dataPath: Path) + val dataPath: Path, + val clock: Clock) extends DeltaLog with Checkpoints + with MetadataCleanup with LogStoreProvider with SnapshotManagement { @@ -49,11 +51,23 @@ private[internal] class DeltaLogImpl private( lazy val store = createLogStore(hadoopConf) /** Direct access to the underlying storage system. */ - private lazy val fs = logPath.getFileSystem(hadoopConf) + protected lazy val fs = logPath.getFileSystem(hadoopConf) // TODO: There is a race here where files could get dropped when increasing the // retention interval... - private def metadata = if (snapshot == null) Metadata() else snapshot.metadataScala + protected def metadata = if (snapshot == null) Metadata() else snapshot.metadataScala + + /** How long to keep around logically deleted files before physically deleting them. */ + def tombstoneRetentionMillis: Long = + // TODO DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata)) + // 1 week + metadata.configuration.getOrElse("deletedFileRetentionDuration", "604800000").toLong + + /** + * Tombstones before this timestamp will be dropped from the state and the files can be + * garbage collected. + */ + def minFileRetentionTimestamp: Long = clock.getTimeMillis() - tombstoneRetentionMillis /** Use ReentrantLock to allow us to call `lockInterruptibly`. */ private val deltaLogLock = new ReentrantLock() @@ -161,11 +175,22 @@ private[standalone] object DeltaLogImpl { apply(hadoopConf, new Path(dataPath, "_delta_log")) } - def apply(hadoopConf: Configuration, rawPath: Path): DeltaLogImpl = { + def forTable(hadoopConf: Configuration, dataPath: String, clock: Clock): DeltaLogImpl = { + apply(hadoopConf, new Path(dataPath, "_delta_log"), clock) + } + + def forTable(hadoopConf: Configuration, dataPath: Path, clock: Clock): DeltaLogImpl = { + apply(hadoopConf, new Path(dataPath, "_delta_log"), clock) + } + + def apply( + hadoopConf: Configuration, + rawPath: Path, + clock: Clock = new SystemClock): DeltaLogImpl = { val fs = rawPath.getFileSystem(hadoopConf) val path = fs.makeQualified(rawPath) - new DeltaLogImpl(hadoopConf, path, path.getParent) + new DeltaLogImpl(hadoopConf, path, path.getParent, clock) } /** diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala new file mode 100644 index 00000000000..5ae830bd169 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala @@ -0,0 +1,92 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.{Calendar, TimeZone} + +import io.delta.standalone.internal.util.FileNames.{checkpointPrefix, isCheckpointFile, isDeltaFile, checkpointVersion, deltaVersion} + +import org.apache.commons.lang.time.DateUtils +import org.apache.hadoop.fs.{FileStatus, Path} + +private[internal] trait MetadataCleanup { + self: DeltaLogImpl => + + /** Whether to clean up expired log files and checkpoints. */ + def enableExpiredLogCleanup: Boolean = + // TODO: DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.fromMetaData(metadata) + metadata.configuration.getOrElse("enableExpiredLogCleanup", "true").toBoolean + + /** + * Returns the duration in millis for how long to keep around obsolete logs. We may keep logs + * beyond this duration until the next calendar day to avoid constantly creating checkpoints. + */ + def deltaRetentionMillis: Long = { + // TODO DeltaConfigs.getMilliSeconds(DeltaConfigs.LOG_RETENTION.fromMetaData(metadata)) + // 30 days + metadata.configuration.getOrElse("logRetentionDuration", "2592000000").toLong + } + + def doLogCleanup(): Unit = { + if (enableExpiredLogCleanup) { + cleanUpExpiredLogs() + } + } + + /** Clean up expired delta and checkpoint logs. Exposed for testing. */ + def cleanUpExpiredLogs(): Unit = { + val fileCutOffTime = truncateDay(clock.getTimeMillis() - deltaRetentionMillis).getTime + listExpiredDeltaLogs(fileCutOffTime.getTime).map(_.getPath).foreach { path => + // recursive = false + fs.delete(path, false) + } + } + + /** + * Returns an iterator of expired delta logs that can be cleaned up. For a delta log to be + * considered as expired, it must: + * - have a checkpoint file after it + * - be older than `fileCutOffTime` + */ + private def listExpiredDeltaLogs(fileCutOffTime: Long): Iterator[FileStatus] = { + val latestCheckpoint = lastCheckpoint + if (latestCheckpoint.isEmpty) return Iterator.empty + val threshold = latestCheckpoint.get.version - 1L + val files = store.listFrom(checkpointPrefix(logPath, 0)) + .filter(f => isCheckpointFile(f.getPath) || isDeltaFile(f.getPath)) + def getVersion(filePath: Path): Long = { + if (isCheckpointFile(filePath)) { + checkpointVersion(filePath) + } else { + deltaVersion(filePath) + } + } + + new BufferingLogDeletionIterator(files, fileCutOffTime, threshold, getVersion) + } + + /** Truncates a timestamp down to the previous midnight and returns the time and a log string */ + private def truncateDay(timeMillis: Long): Calendar = { + val date = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + date.setTimeInMillis(timeMillis) + + // TODO: this is using org.apache.commons.lang2.6 instead of org.apache.commons.lang3 + DateUtils.truncate( + date, + Calendar.DAY_OF_MONTH) + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 838473e9505..441c0ef8309 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -104,7 +104,7 @@ private[internal] class OptimisticTransactionImpl( } val commitInfo = CommitInfo( - System.currentTimeMillis(), + deltaLog.clock.getTimeMillis(), op.getName.toString, null, Map.empty, @@ -118,7 +118,7 @@ private[internal] class OptimisticTransactionImpl( preparedActions = commitInfo +: preparedActions - commitAttemptStartTime = System.currentTimeMillis() + commitAttemptStartTime = deltaLog.clock.getTimeMillis() val commitVersion = doCommitRetryIteratively( snapshot.version + 1, @@ -206,11 +206,13 @@ private[internal] class OptimisticTransactionImpl( // If the metadata has changed, add that to the set of actions var finalActions = newMetadata.toSeq ++ actions + val metadataChanges = finalActions.collect { case m: Metadata => m } assert(metadataChanges.length <= 1, "Cannot change the metadata more than once in a transaction.") metadataChanges.foreach(m => verifyNewMetadata(m)) + finalActions = newProtocol.toSeq ++ finalActions if (snapshot.version == -1) { @@ -269,7 +271,7 @@ private[internal] class OptimisticTransactionImpl( if (attemptNumber == 0) { doCommit(commitVersion, actions) } else if (attemptNumber > DELTA_MAX_RETRY_COMMIT_ATTEMPTS) { - val totalCommitAttemptTime = System.currentTimeMillis() - commitAttemptStartTime + val totalCommitAttemptTime = deltaLog.clock.getTimeMillis() - commitAttemptStartTime throw DeltaErrors.maxCommitRetriesExceededException( attemptNumber, commitVersion, diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 7defd0cc3fa..c94e2a22e59 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import io.delta.standalone.Snapshot import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} -import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, SingleAction} +import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.sources.StandaloneHadoopConf @@ -45,6 +45,7 @@ private[internal] class SnapshotImpl( val path: Path, val version: Long, val logSegment: LogSegment, + val minFileRetentionTimestamp: Long, val deltaLog: DeltaLogImpl, val timestamp: Long) extends Snapshot { @@ -85,7 +86,9 @@ private[internal] class SnapshotImpl( // Internal-Only Methods /////////////////////////////////////////////////////////////////////////// - def allFilesScala: Seq[AddFile] = state.activeFiles.values.toSeq + def allFilesScala: Seq[AddFile] = state.activeFiles.toSeq + def tombstonesScala: Seq[RemoveFile] = state.tombstones.toSeq + def setTransactions: Seq[SetTransaction] = state.setTransactions def protocolScala: Protocol = state.protocol def metadataScala: Metadata = state.metadata def numOfFiles: Long = state.numOfFiles @@ -110,7 +113,7 @@ private[internal] class SnapshotImpl( */ protected lazy val state: State = { val logPathURI = path.toUri - val replay = new InMemoryLogReplay(hadoopConf) + val replay = new InMemoryLogReplay(hadoopConf, minFileRetentionTimestamp) val files = (logSegment.deltas ++ logSegment.checkpoints).map(_.getPath) // assert that the log belongs to table @@ -137,16 +140,19 @@ private[internal] class SnapshotImpl( State( replay.currentProtocolVersion, replay.currentMetaData, + replay.getSetTransactions, replay.getActiveFiles, + replay.getTombstones, replay.sizeInBytes, replay.getActiveFiles.size, replay.numMetadata, - replay.numProtocol + replay.numProtocol, + replay.getTombstones.size, + replay.getSetTransactions.size ) } - private lazy val activeFiles = - state.activeFiles.values.map(ConversionUtils.convertAddFile).toList.asJava + private lazy val activeFiles = state.activeFiles.map(ConversionUtils.convertAddFile).toList.asJava /** * Asserts that the client is up to date with the protocol and allowed @@ -185,20 +191,28 @@ private[internal] object SnapshotImpl { * * @param protocol The protocol version of the Delta table * @param metadata The metadata of the table + * @param setTransactions The streaming queries writing to this table * @param activeFiles The files in this table + * @param tombstones The unexpired tombstones * @param sizeInBytes The total size of the table (of active files, not including tombstones) * @param numOfFiles The number of files in this table * @param numOfMetadata The number of metadata actions in the state. Should be 1 * @param numOfProtocol The number of protocol actions in the state. Should be 1 + * @param numOfRemoves The number of tombstones in the state + * @param numOfSetTransactions Number of streams writing to this table */ case class State( protocol: Protocol, metadata: Metadata, - activeFiles: scala.collection.immutable.Map[URI, AddFile], + setTransactions: Seq[SetTransaction], + activeFiles: Iterable[AddFile], + tombstones: Iterable[RemoveFile], sizeInBytes: Long, numOfFiles: Long, numOfMetadata: Long, - numOfProtocol: Long) + numOfProtocol: Long, + numOfRemoves: Long, + numOfSetTransactions: Long) } /** @@ -212,13 +226,9 @@ private class InitialSnapshotImpl( override val hadoopConf: Configuration, val logPath: Path, override val deltaLog: DeltaLogImpl) - extends SnapshotImpl(hadoopConf, logPath, -1, LogSegment.empty(logPath), deltaLog, -1) { + extends SnapshotImpl(hadoopConf, logPath, -1, LogSegment.empty(logPath), -1, deltaLog, -1) { override lazy val state: SnapshotImpl.State = { - SnapshotImpl.State( - Protocol(), - Metadata(), - Map.empty[URI, AddFile], - 0L, 0L, 1L, 1L) + SnapshotImpl.State(Protocol(), Metadata(), Nil, Nil, Nil, 0L, 0L, 1L, 1L, 0L, 0L) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala index 8aa32ca8986..5e2204fef70 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala @@ -226,6 +226,7 @@ private[internal] trait SnapshotManagement { self: DeltaLogImpl => logPath, segment.version, segment, + minFileRetentionTimestamp, this, lastCommitTimestamp) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/InMemoryLogReplay.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/InMemoryLogReplay.scala index 5a9a6598df3..a93bbc316c0 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/InMemoryLogReplay.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/InMemoryLogReplay.scala @@ -26,8 +26,8 @@ import io.delta.standalone.internal.SnapshotImpl.canonicalizePath * Replays a history of action, resolving them to produce the current state * of the table. The protocol for resolution is as follows: * - The most recent [[AddFile]] and accompanying metadata for any `path` wins. - * - [[RemoveFile]] deletes a corresponding [[AddFile]] and is NOT retained. No tombstones are - * kept. + * - [[RemoveFile]] deletes a corresponding [[AddFile]] and is retained as a + * tombstone until `minFileRetentionTimestamp` has passed. * - The most recent [[Metadata]] wins. * - The most recent [[Protocol]] version wins. * - For each path, this class should always output only one [[FileAction]] (either [[AddFile]] or @@ -35,13 +35,16 @@ import io.delta.standalone.internal.SnapshotImpl.canonicalizePath * * This class is not thread safe. */ -private[internal] class InMemoryLogReplay(hadoopConf: Configuration) { +private[internal] class InMemoryLogReplay( + hadoopConf: Configuration, + minFileRetentionTimestamp: Long) { var currentProtocolVersion: Protocol = null var currentVersion: Long = -1 var currentMetaData: Metadata = null var sizeInBytes: Long = 0 var numMetadata: Long = 0 var numProtocol: Long = 0 + private val transactions = new scala.collection.mutable.HashMap[String, SetTransaction]() private val activeFiles = new scala.collection.mutable.HashMap[URI, AddFile]() private val tombstones = new scala.collection.mutable.HashMap[URI, RemoveFile]() @@ -50,6 +53,8 @@ private[internal] class InMemoryLogReplay(hadoopConf: Configuration) { s"Attempted to replay version $version, but state is at $currentVersion") currentVersion = version actions.foreach { + case a: SetTransaction => + transactions(a.appId) = a case a: Metadata => currentMetaData = a numMetadata += 1 @@ -78,5 +83,12 @@ private[internal] class InMemoryLogReplay(hadoopConf: Configuration) { } } - def getActiveFiles: Map[URI, AddFile] = activeFiles.toMap + def getSetTransactions: Seq[SetTransaction] = transactions.values.toSeq + + def getActiveFiles: Iterable[AddFile] = activeFiles.values + + def getTombstones: Iterable[RemoveFile] = { + tombstones.values.filter(_.delTimestamp > minFileRetentionTimestamp) + } + } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 9d58dac7bba..cba0f85526f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -126,12 +126,6 @@ private[internal] object DeltaErrors { "Couldn't find Metadata while committing the first version of the Delta table.") } - def protocolDowngradeException(oldProtocol: Protocol, newProtocol: Protocol): Throwable = { - // TODO: class ProtocolDowngradeException ? - new RuntimeException("Protocol version cannot be downgraded from " + - s"${oldProtocol.simpleString} to ${newProtocol.simpleString}") - } - def addFilePartitioningMismatchException( addFilePartitions: Seq[String], metadataPartitions: Seq[String]): Throwable = { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/Clock.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/Clock.scala new file mode 100644 index 00000000000..1aa41ec2ace --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/Clock.scala @@ -0,0 +1,100 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +/** + * An interface to represent clocks, so that they can be mocked out in unit tests. + */ +private[internal] trait Clock { + /** @return Current system time, in ms. */ + def getTimeMillis(): Long + + // scalastyle:off line.size.limit + /** + * Current value of high resolution time source, in ns. + * + * This method abstracts the call to the JRE's `System.nanoTime()` call. As with that method, the + * value here is not guaranteed to be monotonically increasing, but rather a higher resolution + * time source for use in the calculation of time intervals. The characteristics of the values + * returned may very from JVM to JVM (or even the same JVM running on different OSes or CPUs), but + * in general it should be preferred over [[getTimeMillis()]] when calculating time differences. + * + * Specifically for Linux on x64 architecture, the following links provide useful information + * about the characteristics of the value returned: + * + * http://btorpey.github.io/blog/2014/02/18/clock-sources-in-linux/ + * https://stackoverflow.com/questions/10921210/cpu-tsc-fetch-operation-especially-in-multicore-multi-processor-environment + * + * TL;DR: on modern (2.6.32+) Linux kernels with modern (AMD K8+) CPUs, the values returned by + * `System.nanoTime()` are consistent across CPU cores *and* packages, and provide always + * increasing values (although it may not be completely monotonic when the system clock is + * adjusted by NTP daemons using time slew). + */ + // scalastyle:on line.size.limit + def nanoTime(): Long + + /** + * Wait until the wall clock reaches at least the given time. Note this may not actually wait for + * the actual difference between the current and target times, since the wall clock may drift. + */ + def waitTillTime(targetTime: Long): Long +} + +/** + * A clock backed by the actual time from the OS as reported by the `System` API. + */ +private[internal] class SystemClock extends Clock { + + val minPollTime = 25L + + /** + * @return the same time (milliseconds since the epoch) + * as is reported by `System.currentTimeMillis()` + */ + override def getTimeMillis(): Long = System.currentTimeMillis() + + /** + * @return value reported by `System.nanoTime()`. + */ + override def nanoTime(): Long = System.nanoTime() + + /** + * @param targetTime block until the current time is at least this value + * @return current system time when wait has completed + */ + override def waitTillTime(targetTime: Long): Long = { + var currentTime = System.currentTimeMillis() + + var waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong + + while (true) { + currentTime = System.currentTimeMillis() + waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + val sleepTime = math.min(waitTime, pollTime) + Thread.sleep(sleepTime) + } + -1 + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ManualClock.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ManualClock.scala new file mode 100644 index 00000000000..d4906fa93ee --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ManualClock.scala @@ -0,0 +1,70 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import java.util.concurrent.TimeUnit + +/** + * A `Clock` whose time can be manually set and modified. Its reported time does not change + * as time elapses, but only as its time is modified by callers. This is mainly useful for + * testing. + * + * For this implementation, `getTimeMillis()` and `nanoTime()` always return the same value + * (adjusted for the correct unit). + * + * @param time initial time (in milliseconds since the epoch) + */ +private[internal] class ManualClock(private var time: Long) extends Clock { + + /** + * @return `ManualClock` with initial time 0 + */ + def this() = this(0L) + + override def getTimeMillis(): Long = synchronized { + time + } + + override def nanoTime(): Long = TimeUnit.MILLISECONDS.toNanos(getTimeMillis()) + + /** + * @param timeToSet new time (in milliseconds) that the clock should represent + */ + def setTime(timeToSet: Long): Unit = synchronized { + time = timeToSet + notifyAll() + } + + /** + * @param timeToAdd time (in milliseconds) to add to the clock's time + */ + def advance(timeToAdd: Long): Unit = synchronized { + time += timeToAdd + notifyAll() + } + + /** + * @param targetTime block until the clock time is set or advanced to at least this time + * @return current time reported by the clock when waiting finishes + */ + override def waitTillTime(targetTime: Long): Long = synchronized { + while (time < targetTime) { + wait(10) + } + getTimeMillis() + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala new file mode 100644 index 00000000000..1a66598928b --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala @@ -0,0 +1,21 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +class BufferingLogDeletionIteratorSuite { + // TODO +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index bd5a19851f3..fa36d3c0886 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -58,6 +58,8 @@ class DeltaLogSuite extends FunSuite { } } + // TODO: another checkpoint test + test("snapshot") { def getDirDataFiles(tablePath: String): Array[File] = { val correctTablePath = @@ -146,6 +148,8 @@ class DeltaLogSuite extends FunSuite { } } + // TODO: update should pick up checkpoints + test("handle corrupted '_last_checkpoint' file") { withLogImplForGoldenTable("corrupted-last-checkpoint") { log1 => assert(log1.lastCheckpoint.isDefined) @@ -191,6 +195,8 @@ class DeltaLogSuite extends FunSuite { } } + // TODO: do not relativize paths in RemoveFiles + test("delete and re-add the same file in different transactions") { withLogForGoldenTable("delete-re-add-same-file-different-transactions") { log => assert(log.snapshot().getAllFiles.size() == 2) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala new file mode 100644 index 00000000000..4fd53e85e29 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala @@ -0,0 +1,198 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.io.File + +import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, RemoveFile} +import io.delta.standalone.internal.util.ManualClock +import io.delta.standalone.internal.util.TestUtils._ +import io.delta.standalone.Operation +import org.apache.hadoop.conf.Configuration + + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +// scalastyle:off removeFile +class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { + // scalastyle:on funsuite + + val writerId = "test-writer-id" + val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + val manualLogCleanupMetadata = Metadata(configuration = Map("enableExpiredLogCleanup" -> "false")) + + protected def getLogFiles(dir: File): Seq[File] = + getDeltaFiles(dir) ++ getCheckpointFiles(dir) + + test("delete expired logs") { + withTempDir { dir => + val clock = new ManualClock(System.currentTimeMillis()) + val log = DeltaLogImpl.forTable(new Configuration(), dir.getCanonicalPath, clock) + val logPath = new File(log.logPath.toUri) + (1 to 5).foreach { i => + val txn = log.startTransaction() + val metadata = if (i == 1) manualLogCleanupMetadata :: Nil else Nil + val file = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil + val delete: Seq[Action] = if (i > 1) { + RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil + } else { + Nil + } + txn.commit(metadata ++ delete ++ file, manualUpdate, writerId) + } + + val initialFiles = getLogFiles(logPath) + // Shouldn't clean up, no checkpoint, no expired files + log.cleanUpExpiredLogs() + + assert(initialFiles === getLogFiles(logPath)) + + // TODO clock.advance(intervalStringToMillis(DeltaConfigs.LOG_RETENTION.defaultValue) + + // intervalStringToMillis("interval 1 day")) + clock.advance(log.deltaRetentionMillis + 1000*60*60*24) // 1 day + + // Shouldn't clean up, no checkpoint, although all files have expired + log.cleanUpExpiredLogs() + assert(initialFiles === getLogFiles(logPath)) + + log.checkpoint() + + val expectedFiles = Seq("04.json", "04.checkpoint.parquet") + // after checkpointing, the files should be cleared + log.cleanUpExpiredLogs() + val afterCleanup = getLogFiles(logPath) + assert(initialFiles !== afterCleanup) + assert(expectedFiles.forall(suffix => afterCleanup.exists(_.getName.endsWith(suffix))), + s"${afterCleanup.mkString("\n")}\n didn't contain files with suffixes: $expectedFiles") + } + } + + test("delete expired logs 2") { + withTempDir { dir => + val clock = new ManualClock(System.currentTimeMillis()) + val conf = new Configuration() + val log = DeltaLogImpl.forTable(conf, dir.getCanonicalPath, clock) + val logPath = new File(log.logPath.toUri) + + // write 000.json to 009.json + (0 to 9).foreach { i => + val txn = log.startTransaction() + val metadata = if (i == 0) manualLogCleanupMetadata :: Nil else Nil + txn.commit(metadata :+ AddFile(i.toString, Map.empty, 1, 1, true), manualUpdate, writerId) + } + + assert(log.update().version == 9) + assert(getDeltaFiles(logPath).size == 10) + assert(getCheckpointFiles(logPath).isEmpty) + + // Local filesystem will truncate the logFile last modified timestamps to the nearest second. + // This allows for contiguous log & checkpoint files to have the same timestamp. + // e.g. 00.json, 00.checkpoint, 01.json. 01.checkpoint have lastModified time 1630107078000. + // This breaks assumptions made in [[BufferingLogDeletionIterator]]. + // This will never happen in production, so let's just fix the timestamps + val now = clock.getTimeMillis() + getLogFiles(logPath).sortBy(_.getName).zipWithIndex.foreach { case (file, idx) => + file.setLastModified(now + 1000 * idx) + } + + // to expire log files, advance by the retention duration, then another day (since we + // truncate) + clock.advance(log.deltaRetentionMillis + 2*1000*60*60*24 + 1000*100) + // now, 000.json to 009.json have all expired + + // write 010.json and 010.checkpoint + log.startTransaction() + .commit(AddFile("10", Map.empty, 1, 1, true) :: Nil, manualUpdate, writerId) + + getLogFiles(logPath) + .filter(_.getName.contains("10.")) + .foreach(_.setLastModified(clock.getTimeMillis())) + + // Finally, clean up expired logs. this should delete 000.json to 009.json + log.cleanUpExpiredLogs() + + assert(log.update().version == 10) + assert(getDeltaFiles(logPath).size == 1) + assert(getCheckpointFiles(logPath).size == 1) + + val afterAutoCleanup = getLogFiles(logPath) + val expectedFiles = Seq("10.json", "10.checkpoint.parquet") + assert(expectedFiles.forall(suffix => afterAutoCleanup.exists(_.getName.endsWith(suffix))), + s"${afterAutoCleanup.mkString("\n")}\n didn't contain files with suffixes: $expectedFiles") + } + } + + test("Can set enableExpiredLogCleanup") { + withTempDir { tempDir => + val log = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath) + log.startTransaction().commit( + Metadata(configuration = Map("enableExpiredLogCleanup" -> "true")) :: Nil, + manualUpdate, writerId) + assert(log.enableExpiredLogCleanup) + + log.startTransaction().commit(manualLogCleanupMetadata :: Nil, manualUpdate, writerId) + assert(!log.enableExpiredLogCleanup) + + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + assert(log.enableExpiredLogCleanup) + } + } + + test( + "RemoveFiles persist across checkpoints as tombstones if retention time hasn't expired") { + withTempDir { tempDir => + val clock = new ManualClock(System.currentTimeMillis()) + val log1 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + + val txn1 = log1.startTransaction() + val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) + txn1.commit(files1 :+ manualLogCleanupMetadata, manualUpdate, writerId) + val txn2 = log1.startTransaction() + val files2 = (1 to 4).map(f => RemoveFile(f.toString, Some(clock.getTimeMillis()))) + txn2.commit(files2, manualUpdate, writerId) + log1.checkpoint() + + val log2 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + assert(log2.snapshot.tombstonesScala.size === 4) + assert(log2.snapshot.allFilesScala.size === 6) + } + } + + test("RemoveFiles get deleted during checkpoint if retention time has passed") { + withTempDir { tempDir => + val clock = new ManualClock(System.currentTimeMillis()) + val log1 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + + val txn = log1.startTransaction() + val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) + txn.commit(files1 :+ manualLogCleanupMetadata, manualUpdate, writerId) + val txn2 = log1.startTransaction() + val files2 = (1 to 4).map(f => RemoveFile(f.toString, Some(clock.getTimeMillis()))) + txn2.commit(files2, manualUpdate, writerId) + + // TODO: intervalStringToMillis(DeltaConfigs.TOMBSTONE_RETENTION.defaultValue) + 1000000L) + clock.advance(log1.tombstoneRetentionMillis + 1000000L) + + log1.checkpoint() + + val log2 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + assert(log2.snapshot.tombstonesScala.size === 0) + assert(log2.snapshot.allFilesScala.size === 6) + } + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala new file mode 100644 index 00000000000..bf867e66541 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala @@ -0,0 +1,44 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.io.File + +import io.delta.standalone.{DeltaLog, OptimisticTransaction} +import io.delta.standalone.internal.actions.Metadata +import io.delta.standalone.internal.util.{ConversionUtils, FileNames} +import org.apache.hadoop.fs.Path + +trait DeltaRetentionSuiteBase { + + protected def getDeltaFiles(dir: File): Seq[File] = + dir.listFiles().filter(_.getName.endsWith(".json")) + + protected def getCheckpointFiles(dir: File): Seq[File] = + dir.listFiles().filter(f => FileNames.isCheckpointFile(new Path(f.getCanonicalPath))) + + /** + * Start a txn that disables automatic log cleanup. Some tests may need to manually clean up logs + * to get deterministic behaviors. + */ + // TODO: this is dependent on withGlobalConfigDefaults + // protected def startTxnWithManualLogCleanup(log: DeltaLog): OptimisticTransaction = { + // val txn = log.startTransaction() + // txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + // txn + // } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 1bb51c265b7..daffbdd92ae 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -23,10 +23,10 @@ import io.delta.standalone.{DeltaLog, Operation} import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.exception.DeltaErrors.InvalidProtocolVersionException import io.delta.standalone.internal.util.ConversionUtils import io.delta.standalone.types.{StringType, StructField, StructType} import io.delta.standalone.internal.util.TestUtils._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -56,12 +56,6 @@ class OptimisticTransactionSuite extends FunSuite { private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) - implicit def actionSeqToList[T <: Action](seq: Seq[T]): java.util.List[ActionJ] = - seq.map(ConversionUtils.convertAction).asJava - - implicit def addFileSeqToList(seq: Seq[AddFile]): java.util.List[AddFileJ] = - seq.map(ConversionUtils.convertAddFile).asJava - def withLog( actions: Seq[Action], partitionCols: Seq[String] = "part" :: Nil)( @@ -142,6 +136,10 @@ class OptimisticTransactionSuite extends FunSuite { } } + /////////////////////////////////////////////////////////////////////////// + // prepareCommit() tests + /////////////////////////////////////////////////////////////////////////// + test("committing twice in the same transaction should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -176,6 +174,7 @@ class OptimisticTransactionSuite extends FunSuite { } } + // DeltaLog::ensureLogDirectoryExists test("transaction should throw if it cannot read log directory during first commit") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -191,7 +190,7 @@ class OptimisticTransactionSuite extends FunSuite { } } - test("first commit must have a Metadata") { + test("initial commit without metadata should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() @@ -216,7 +215,6 @@ class OptimisticTransactionSuite extends FunSuite { } } - // TODO test create a table with protocol too high test("Can't create table with invalid protocol version") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -257,6 +255,10 @@ class OptimisticTransactionSuite extends FunSuite { } } + /////////////////////////////////////////////////////////////////////////// + // updateMetadata() tests + /////////////////////////////////////////////////////////////////////////// + test("can't update metadata more than once in a transaction") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -298,7 +300,7 @@ class OptimisticTransactionSuite extends FunSuite { } } - test("commit new metadataa with Protocol properties should fail") { + test("commit new metadata with Protocol properties should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) @@ -315,6 +317,10 @@ class OptimisticTransactionSuite extends FunSuite { } } + /////////////////////////////////////////////////////////////////////////// + // verifyNewMetadata() tests + /////////////////////////////////////////////////////////////////////////// + test("can't have duplicate column names") { // TODO: just call myStruct.getJson() // scalastyle:off @@ -339,15 +345,22 @@ class OptimisticTransactionSuite extends FunSuite { // TODO: test updateMetadata > unenforceable not null constraints removed from metadata schemaStr - // TODO: test commit - // - commitInfo is actually added to final actions - // - isBlindAppend == true - // - isBlindAppend == false - // - different operation names + // TODO: test updateMetadata > withGlobalConfigDefaults - // TODO: test doCommit > IllegalStateException + /////////////////////////////////////////////////////////////////////////// + // commit() tests + /////////////////////////////////////////////////////////////////////////// - // TODO: test doCommit > DeltaConcurrentModificationException + // - TODO commitInfo is actually added to final actions (with correct engineInfo) + // - TODO isBlindAppend == true cases + // - TODO isBlindAppend == false case + // - TODO different operation names + + /////////////////////////////////////////////////////////////////////////// + // checkForConflicts() tests + /////////////////////////////////////////////////////////////////////////// + + // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) // TODO: test more ConcurrentAppendException @@ -355,4 +368,10 @@ class OptimisticTransactionSuite extends FunSuite { // TODO: test checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn with SnapshotIsolation // i.e. datachange = false + + // TODO: test Checkpoint > partialWriteVisible (==> useRename) + + // TODO: test Checkpoint > !partialWriteVisible (==> !useRename) + + // TODO: test Checkpoint > correctly checkpoints all action types } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala b/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala index b26c3c2c151..48e32841492 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/util/TestUtils.scala @@ -20,6 +20,11 @@ import java.io.File import java.nio.file.Files import java.util.UUID +import scala.collection.JavaConverters._ + +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ} +import io.delta.standalone.internal.actions.{Action, AddFile} + import org.apache.commons.io.FileUtils object TestUtils { @@ -35,4 +40,10 @@ object TestUtils { } } + implicit def actionSeqToList[T <: Action](seq: Seq[T]): java.util.List[ActionJ] = + seq.map(ConversionUtils.convertAction).asJava + + implicit def addFileSeqToList(seq: Seq[AddFile]): java.util.List[AddFileJ] = + seq.map(ConversionUtils.convertAddFile).asJava + } From c44bf631d96b0819aa0f411a1ac834a6aad11083 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 29 Sep 2021 10:38:08 -0700 Subject: [PATCH 085/291] [DSW] [9] Production commit API (#149) * add Expression::references() func; implement OptimisticTxn::markFilesAsRead(); isBlindAppend unit tests * Update OptimisticTransactionSuite.scala * whitespace javadoc changes --- .../standalone/OptimisticTransaction.java | 12 +-- .../delta/standalone/expressions/Column.java | 8 ++ .../standalone/internal/DeltaLogImpl.scala | 6 +- .../internal/OptimisticTransactionImpl.scala | 39 ++++++-- .../standalone/internal/actions/actions.scala | 5 + .../standalone/internal/DeltaLogSuite.scala | 2 +- .../standalone/internal/ExpressionSuite.scala | 78 +++++++++------ .../internal/OptimisticTransactionSuite.scala | 96 +++++++++++++------ 8 files changed, 168 insertions(+), 78 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 29293516152..eddc14a3cf7 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -14,9 +14,9 @@ public interface OptimisticTransaction { * the given `lastVersion`. In the case of a conflict with a concurrent writer this * method will throw an exception. * - * @param actions Set of actions to commit. - * @param op Details of operation that is performing this transactional commit. - * @param engineInfo String used to identify the writer engine. It should resemble + * @param actions Set of actions to commit. + * @param op Details of operation that is performing this transactional commit. + * @param engineInfo String used to identify the writer engine. It should resemble * "{engineName}-{engineVersion}". * @return a {@link CommitResult}, wrapping the table version that was committed. */ @@ -39,7 +39,7 @@ public interface OptimisticTransaction { * readFiles were changed by TXN2. Thus there are no logical conflicts and TXN1 can commit at * table version N+1. * - * @param readPredicates Predicates used to determine which files were read. + * @param readPredicates Predicates used to determine which files were read. * @return Files matching the given predicates. */ List markFilesAsRead(Iterable readPredicates); @@ -52,7 +52,7 @@ public interface OptimisticTransaction { * IMPORTANT: It is the responsibility of the caller to ensure that files currently * present in the table are still valid under the new metadata. * - * @param metadata The new metadata for the delta table. + * @param metadata The new metadata for the delta table. */ void updateMetadata(Metadata metadata); @@ -62,7 +62,7 @@ public interface OptimisticTransaction { void readWholeTable(); /** - * @param id TODO + * @param id TODO * @return the latest version that has committed for the idempotent transaction with given `id`. */ long txnVersion(String id); diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Column.java b/standalone/src/main/java/io/delta/standalone/expressions/Column.java index ee7a83f415f..69a66049ba7 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Column.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Column.java @@ -1,5 +1,8 @@ package io.delta.standalone.expressions; +import java.util.Collections; +import java.util.Set; + import io.delta.standalone.data.RowRecord; import io.delta.standalone.types.*; @@ -48,6 +51,11 @@ public String toString() { return "Column(" + name + ")"; } + @Override + public Set references() { + return Collections.singleton(name); + } + @FunctionalInterface private interface RowRecordEvaluator { Object nullSafeEval(RowRecord record); diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index feb10b9b611..450bedf54f5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -183,7 +183,7 @@ private[standalone] object DeltaLogImpl { apply(hadoopConf, new Path(dataPath, "_delta_log"), clock) } - def apply( + private def apply( hadoopConf: Configuration, rawPath: Path, clock: Clock = new SystemClock): DeltaLogImpl = { @@ -199,7 +199,7 @@ private[standalone] object DeltaLogImpl { * information * @param partitionFilters Filters on the partition columns */ - def filterFileList( + private[internal] def filterFileList( partitionSchema: StructType, files: Seq[AddFile], partitionFilters: Seq[Expression]): Seq[AddFile] = { @@ -209,7 +209,7 @@ private[standalone] object DeltaLogImpl { files.filter { addFile => val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) val result = expr.eval(partitionRowRecord) - result == true + result.asInstanceOf[Boolean] } } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 441c0ef8309..fadf4f311e9 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import io.delta.standalone.{CommitResult, Operation, OptimisticTransaction} import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} -import io.delta.standalone.expressions.Expression +import io.delta.standalone.expressions.{And, Column, Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils, SchemaMergingUtils, SchemaUtils} @@ -130,16 +130,24 @@ private[internal] class OptimisticTransactionImpl( new CommitResult(commitVersion) } + /** Returns files matching the given predicates. */ override def markFilesAsRead( _readPredicates: java.lang.Iterable[Expression]): java.util.List[AddFileJ] = { - // TODO: PartitionFiltering::filesForScan - // TODO - // val partitionFilters = filters.filter { f => - // DeltaTableUtils.isPredicatePartitionColumnsOnly(f, metadata.partitionColumns, spark) - // } - // TODO readPredicates += ... - // TODO readFiles ++= - null + + val partitionFilters = _readPredicates.asScala.filter { f => + isPredicatePartitionColumnsOnly(f, metadata.partitionColumns) + }.toSeq + + val matchedFiles = DeltaLogImpl.filterFileList( + metadata.partitionSchema, + snapshot.allFilesScala, + partitionFilters + ) + + readPredicates += partitionFilters.reduceLeftOption(new And(_, _)).getOrElse(Literal.True) + readFiles ++= matchedFiles + + matchedFiles.map(ConversionUtils.convertAddFile).asJava } override def updateMetadata(metadataJ: MetadataJ): Unit = { @@ -403,6 +411,17 @@ private[internal] object OptimisticTransactionImpl { val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { - op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap + op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap + } + + /** + * Does the predicate only contains partition columns? + */ + def isPredicatePartitionColumnsOnly( + condition: Expression, + partitionColumns: Seq[String]): Boolean = { + // TODO: name equality resolver + + condition.references().asScala.forall(partitionColumns.contains(_)) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala index d6ed74328ea..cbd65a322b5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala @@ -202,6 +202,11 @@ private[internal] case class Metadata( new StructType(schema.getFields.filterNot(f => partitions.contains(f.getName))) } + /** Returns the partitionSchema as a [[StructType]] */ + @JsonIgnore + lazy val partitionSchema: StructType = + new StructType(partitionColumns.map(c => schema.get(c)).toArray) + override def wrap: SingleAction = SingleAction(metaData = this) } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index fa36d3c0886..14b9a2d1d2b 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -28,7 +28,7 @@ import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog, Snapshot} import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, - RemoveFile => RemoveFileJ} + RemoveFile => RemoveFileJ} import io.delta.standalone.internal.actions.{Action, Protocol} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.GoldenTableUtils._ diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala index 98bff3a6ac4..fb67be04be1 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import io.delta.standalone.data.RowRecord import io.delta.standalone.expressions._ import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.OptimisticTransactionImpl.isPredicatePartitionColumnsOnly import io.delta.standalone.types.{IntegerType, StructField, StructType} // scalastyle:off funsuite @@ -30,22 +31,31 @@ import org.scalatest.FunSuite class ExpressionSuite extends FunSuite { // scalastyle:on funsuite + private val dataSchema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true), + new StructField("col3", new IntegerType(), true))) + + private val partitionSchema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true))) + private def testPredicate( predicate: Expression, expectedResult: Boolean, - record: RowRecord = null) = { + record: RowRecord = null): Unit = { println(predicate.toString()) println(predicate.eval(record)) assert(predicate.eval(record) == expectedResult) } private def testPartitionFilter( - inputSchema: StructType, + partitionSchema: StructType, inputFiles: Seq[AddFile], filters: Seq[Expression], - expectedMatchedFiles: Seq[AddFile]) = { + expectedMatchedFiles: Seq[AddFile]): Unit = { println("filters:\n\t" + filters.map(_.toString()).mkString("\n\t")) - val matchedFiles = DeltaLogImpl.filterFileList(inputSchema, inputFiles, filters) + val matchedFiles = DeltaLogImpl.filterFileList(partitionSchema, inputFiles, filters) assert(matchedFiles.length == expectedMatchedFiles.length) assert(matchedFiles.forall(expectedMatchedFiles.contains(_))) } @@ -76,10 +86,6 @@ class ExpressionSuite extends FunSuite { } test("basic partition filter") { - val schema = new StructType(Array( - new StructField("col1", new IntegerType()), - new StructField("col2", new IntegerType()))) - val add00 = AddFile("1", Map("col1" -> "0", "col2" -> "0"), 0, 0, dataChange = true) val add01 = AddFile("2", Map("col1" -> "0", "col2" -> "1"), 0, 0, dataChange = true) val add02 = AddFile("2", Map("col1" -> "0", "col2" -> "2"), 0, 0, dataChange = true) @@ -91,43 +97,57 @@ class ExpressionSuite extends FunSuite { val add22 = AddFile("4", Map("col1" -> "2", "col2" -> "2"), 0, 0, dataChange = true) val inputFiles = Seq(add00, add01, add02, add10, add11, add12, add20, add21, add22) - val f1Expr1 = new EqualTo(schema.column("col1"), Literal.of(0)) - val f1Expr2 = new EqualTo(schema.column("col2"), Literal.of(1)) + val f1Expr1 = new EqualTo(partitionSchema.column("col1"), Literal.of(0)) + val f1Expr2 = new EqualTo(partitionSchema.column("col2"), Literal.of(1)) val f1 = new And(f1Expr1, f1Expr2) - testPartitionFilter(schema, inputFiles, f1 :: Nil, add01 :: Nil) - testPartitionFilter(schema, inputFiles, f1Expr1 :: f1Expr2 :: Nil, add01 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f1 :: Nil, add01 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f1Expr1 :: f1Expr2 :: Nil, add01 :: Nil) - val f2Expr1 = new LessThan(schema.column("col1"), Literal.of(1)) - val f2Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f2Expr1 = new LessThan(partitionSchema.column("col1"), Literal.of(1)) + val f2Expr2 = new LessThan(partitionSchema.column("col2"), Literal.of(1)) val f2 = new And(f2Expr1, f2Expr2) - testPartitionFilter(schema, inputFiles, f2 :: Nil, add00 :: Nil) - testPartitionFilter(schema, inputFiles, f2Expr1 :: f2Expr2 :: Nil, add00 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f2 :: Nil, add00 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f2Expr1 :: f2Expr2 :: Nil, add00 :: Nil) - val f3Expr1 = new EqualTo(schema.column("col1"), Literal.of(2)) - val f3Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f3Expr1 = new EqualTo(partitionSchema.column("col1"), Literal.of(2)) + val f3Expr2 = new LessThan(partitionSchema.column("col2"), Literal.of(1)) val f3 = new Or(f3Expr1, f3Expr2) - testPartitionFilter(schema, inputFiles, f3 :: Nil, Seq(add20, add21, add22, add00, add10)) + testPartitionFilter( + partitionSchema, inputFiles, f3 :: Nil, Seq(add20, add21, add22, add00, add10)) val inSet4 = (2 to 10).map(Literal.of).asJava - val f4 = new In(schema.column("col1"), inSet4) - testPartitionFilter(schema, inputFiles, f4 :: Nil, add20 :: add21 :: add22 :: Nil) + val f4 = new In(partitionSchema.column("col1"), inSet4) + testPartitionFilter(partitionSchema, inputFiles, f4 :: Nil, add20 :: add21 :: add22 :: Nil) val inSet5 = (100 to 110).map(Literal.of).asJava - val f5 = new In(schema.column("col1"), inSet5) - testPartitionFilter(schema, inputFiles, f5 :: Nil, Nil) + val f5 = new In(partitionSchema.column("col1"), inSet5) + testPartitionFilter(partitionSchema, inputFiles, f5 :: Nil, Nil) } test("not null partition filter") { - val schema = new StructType(Array( - new StructField("col1", new IntegerType(), true), - new StructField("col2", new IntegerType(), true))) - val add0Null = AddFile("1", Map("col1" -> "0", "col2" -> null), 0, 0, dataChange = true) val addNull1 = AddFile("1", Map("col1" -> null, "col2" -> "1"), 0, 0, dataChange = true) val inputFiles = Seq(add0Null, addNull1) - val f1 = new IsNotNull(schema.column("col1")) - testPartitionFilter(schema, inputFiles, f1 :: Nil, add0Null :: Nil) + val f1 = new IsNotNull(partitionSchema.column("col1")) + testPartitionFilter(partitionSchema, inputFiles, f1 :: Nil, add0Null :: Nil) + } + + test("Expr.references() and OptimisticTransaction.isPredicatePartitionColumnsOnly()") { + val dataExpr = new And( + new LessThan(dataSchema.column("col1"), Literal.of(5)), + new Or( + new EqualTo(dataSchema.column("col1"), dataSchema.column("col2")), + new EqualTo(dataSchema.column("col1"), dataSchema.column("col3")) + ) + ) + + assert(dataExpr.references().size() == 3) + + val partitionExpr = new EqualTo(dataSchema.column("col1"), dataSchema.column("col2")) + + assert(!isPredicatePartitionColumnsOnly(dataExpr, partitionSchema.getFieldNames.toSeq)) + assert(isPredicatePartitionColumnsOnly(partitionExpr, partitionSchema.getFieldNames.toSeq)) } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index daffbdd92ae..c6339f1a654 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -21,6 +21,7 @@ import scala.reflect.ClassTag import io.delta.standalone.{DeltaLog, Operation} import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.expressions.Literal import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.ConversionUtils @@ -35,7 +36,7 @@ import org.scalatest.FunSuite class OptimisticTransactionSuite extends FunSuite { // scalastyle:on funsuite - val writerId = "test-writer-id" + val engineInfo = "test-engine-info" val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) val A_P1 = "part=1/a" @@ -66,8 +67,8 @@ class OptimisticTransactionSuite extends FunSuite { val metadata = Metadata(partitionColumns = partitionCols) withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) - log.startTransaction().commit(actions, manualUpdate, writerId) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + log.startTransaction().commit(actions, manualUpdate, engineInfo) test(log) } @@ -82,7 +83,7 @@ class OptimisticTransactionSuite extends FunSuite { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val e1 = intercept[T] { - log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) } assert(e1.getMessage.contains(expectedExceptionMessageSubStr)) @@ -95,7 +96,7 @@ class OptimisticTransactionSuite extends FunSuite { test("basic commit") { withLog(addA_P1 :: addB_P1 :: Nil) { log => - log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, writerId) + log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) // [...] is what is automatically added by OptimisticTransaction // 0 -> metadata [CommitInfo, Protocol] @@ -127,7 +128,7 @@ class OptimisticTransactionSuite extends FunSuite { } else { Nil } - txn.commit(meta ++ delete ++ file, manualUpdate, writerId) + txn.commit(meta ++ delete ++ file, manualUpdate, engineInfo) } val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) @@ -144,9 +145,9 @@ class OptimisticTransactionSuite extends FunSuite { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() - txn.commit(Metadata() :: Nil, manualUpdate, writerId) + txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) val e = intercept[AssertionError] { - txn.commit(Nil, manualUpdate, writerId) + txn.commit(Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("Transaction already committed.")) } @@ -155,9 +156,9 @@ class OptimisticTransactionSuite extends FunSuite { test("user cannot commit their own CommitInfo") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) val e = intercept[AssertionError] { - log.startTransaction().commit(CommitInfo.empty() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(CommitInfo.empty() :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("Cannot commit a custom CommitInfo in a transaction.")) } @@ -168,7 +169,7 @@ class OptimisticTransactionSuite extends FunSuite { val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() val e = intercept[AssertionError] { - txn.commit(Metadata() :: Metadata() :: Nil, manualUpdate, writerId) + txn.commit(Metadata() :: Metadata() :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) } @@ -182,7 +183,7 @@ class OptimisticTransactionSuite extends FunSuite { val txn = log.startTransaction() val e = intercept[java.io.IOException] { - txn.commit(Metadata() :: Nil, manualUpdate, writerId) + txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) } val logPath = new Path(log.getPath, "_delta_log") @@ -195,7 +196,7 @@ class OptimisticTransactionSuite extends FunSuite { val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() val e = intercept[IllegalStateException] { - txn.commit(Nil, manualUpdate, writerId) + txn.commit(Nil, manualUpdate, engineInfo) } assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) } @@ -206,9 +207,9 @@ class OptimisticTransactionSuite extends FunSuite { val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) // Note that Metadata() has no partition schema specified and addA_P1 does - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) val e = intercept[IllegalStateException] { - log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, writerId) + log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + "table's partitioning schema")) @@ -221,7 +222,7 @@ class OptimisticTransactionSuite extends FunSuite { Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => val e = intercept[AssertionError] { - log.startTransaction().commit(Metadata() :: protocol :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: protocol :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("Invalid Protocol")) } @@ -231,27 +232,31 @@ class OptimisticTransactionSuite extends FunSuite { test("can't change protocol to invalid version") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Protocol() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Protocol() :: Nil, manualUpdate, engineInfo) Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => val e = intercept[AssertionError] { - log.startTransaction().commit(protocol :: Nil, manualUpdate, writerId) + log.startTransaction().commit(protocol :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("Invalid Protocol")) } } } - test("can't remove from an append-only table") { + test("Removing from an append-only table") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val metadata = Metadata(configuration = Map("appendOnly" -> "true")) - log.startTransaction().commit(metadata :: Nil, manualUpdate, writerId) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + val removeWithDataChange = addA_P1.remove.copy(dataChange = true) val e = intercept[UnsupportedOperationException] { - log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, writerId) + log.startTransaction().commit(removeWithDataChange :: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains("This table is configured to only allow appends")) + + val removeWithoutDataChange = addA_P1.remove.copy(dataChange = false) + log.startTransaction().commit(removeWithoutDataChange :: Nil, manualUpdate, engineInfo) } } @@ -275,7 +280,7 @@ class OptimisticTransactionSuite extends FunSuite { test("Protocol Action should be automatically added to transaction for new table") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) assert(log.getChanges(0, true).asScala.next().getActions.contains(new ProtocolJ(1, 2))) } } @@ -292,7 +297,7 @@ class OptimisticTransactionSuite extends FunSuite { Protocol.MIN_WRITER_VERSION_PROP -> "2" )) txn.updateMetadata(ConversionUtils.convertMetadata(metadata)) - txn.commit(Nil, manualUpdate, writerId) + txn.commit(Nil, manualUpdate, engineInfo) val writtenConfig = log.update().getMetadata.getConfiguration assert(!writtenConfig.containsKey(Protocol.MIN_READER_VERSION_PROP)) @@ -303,14 +308,14 @@ class OptimisticTransactionSuite extends FunSuite { test("commit new metadata with Protocol properties should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) val newMetadata = Metadata(configuration = Map( Protocol.MIN_READER_VERSION_PROP -> "1", Protocol.MIN_WRITER_VERSION_PROP -> "2" )) val e = intercept[AssertionError] { - log.startTransaction().commit(newMetadata:: Nil, manualUpdate, writerId) + log.startTransaction().commit(newMetadata:: Nil, manualUpdate, engineInfo) } assert(e.getMessage.contains(s"Should not have the protocol version " + s"(${Protocol.MIN_READER_VERSION_PROP}) as part of table properties")) @@ -351,10 +356,43 @@ class OptimisticTransactionSuite extends FunSuite { // commit() tests /////////////////////////////////////////////////////////////////////////// - // - TODO commitInfo is actually added to final actions (with correct engineInfo) - // - TODO isBlindAppend == true cases - // - TODO isBlindAppend == false case - // - TODO different operation names + test("CommitInfo operation and engineInfo is persisted to the delta log") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val commitInfo = log2.getCommitInfoAt(0) + assert(commitInfo.getEngineInfo.isPresent) + assert(commitInfo.getEngineInfo.get() == engineInfo) + assert(commitInfo.getOperation == manualUpdate.getName.toString) + } + } + + test("CommitInfo isBlindAppend is correctly set") { + withTempDir { dir => + def verifyIsBlindAppend(version: Int, expected: Boolean): Unit = { + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + assert(log.getCommitInfoAt(version).getIsBlindAppend.get() == expected) + } + + val add = AddFile("test", Map.empty, 1, 1, dataChange = false) + + val log0 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log0.startTransaction().commit(Metadata() :: add :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(0, expected = true) + + val log1 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log1.startTransaction().commit(add.remove :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(1, expected = false) + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn2 = log2.startTransaction() + txn2.markFilesAsRead(java.util.Arrays.asList(Literal.True)) + txn2.commit(add :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(2, expected = false) + } + } /////////////////////////////////////////////////////////////////////////// // checkForConflicts() tests From 645aa8a156ffac4e86d678b6ad4f4d91310b4c31 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 29 Sep 2021 12:30:48 -0700 Subject: [PATCH 086/291] Fix mima failure (#159) --- project/StandaloneMimaExcludes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/StandaloneMimaExcludes.scala b/project/StandaloneMimaExcludes.scala index 9100372250b..dca807ed8ac 100644 --- a/project/StandaloneMimaExcludes.scala +++ b/project/StandaloneMimaExcludes.scala @@ -26,6 +26,7 @@ object StandaloneMimaExcludes { ProblemFilters.exclude[Problem]("io.delta.standalone.internal.*"), // Public API changes in 0.2.0 -> 0.3.0 - ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.getChanges") + ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.DeltaLog.getChanges"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("io.delta.standalone.Snapshot.scan") ) } From d37869c706f3a608bff25b37c2ce37e8189b85bf Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 29 Sep 2021 16:44:16 -0700 Subject: [PATCH 087/291] [DSW] [10] Production conflict checker (#150) * Update OptimisticTransactionSuite.scala * update conflict checker detect concurrent appends; add test; add StringType to expressions * add more concurrent append and concurrent delete read exception * added concurrent protocol update conflict check; add another dataChange=false concurrent append test * add concurrent metadata update check with test * add concurrent delete delete conflict check with test * add concurrent txn conflict check with test --- .../ConcurrentDeleteDeleteException.java | 23 ++ .../ConcurrentTransactionException.java | 23 ++ .../exceptions/MetadataChangedException.java | 23 ++ .../exceptions/ProtocolChangedException.java | 23 ++ .../delta/standalone/expressions/Column.java | 2 + .../delta/standalone/expressions/Literal.java | 5 + .../io/delta/standalone/expressions/Util.java | 5 + .../standalone/internal/ConflictChecker.scala | 88 +++++++- .../standalone/internal/DeltaLogImpl.scala | 10 + .../internal/OptimisticTransactionImpl.scala | 23 +- .../standalone/internal/SnapshotImpl.scala | 27 +-- .../internal/data/PartitionRowRecord.scala | 8 +- .../internal/exception/DeltaErrors.scala | 59 ++++- .../internal/util/ConversionUtils.scala | 9 + .../internal/OptimisticTransactionSuite.scala | 201 +++++++++++++++++- 15 files changed, 479 insertions(+), 50 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java new file mode 100644 index 00000000000..0f2b529b187 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class ConcurrentDeleteDeleteException extends DeltaConcurrentModificationException { + public ConcurrentDeleteDeleteException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java new file mode 100644 index 00000000000..1ab4ec95df0 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class ConcurrentTransactionException extends DeltaConcurrentModificationException { + public ConcurrentTransactionException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java b/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java new file mode 100644 index 00000000000..e0193e49b57 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class MetadataChangedException extends DeltaConcurrentModificationException { + public MetadataChangedException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java new file mode 100644 index 00000000000..9986e09e751 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java @@ -0,0 +1,23 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class ProtocolChangedException extends DeltaConcurrentModificationException { + public ProtocolChangedException(String message) { + super(message); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Column.java b/standalone/src/main/java/io/delta/standalone/expressions/Column.java index 69a66049ba7..856a4ecefac 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Column.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Column.java @@ -27,6 +27,8 @@ public Column(String name, DataType dataType) { evaluator = (record -> record.getInt(name)); } else if (dataType instanceof BooleanType) { evaluator = (record -> record.getBoolean(name)); + } else if (dataType instanceof StringType) { + evaluator = (record -> record.getString(name)); } else { throw new RuntimeException("Couldn't find matching rowRecord DataType for column: " + name); } diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java index eb23a6f3028..e0b09c16ca8 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java @@ -4,6 +4,7 @@ import io.delta.standalone.types.BooleanType; import io.delta.standalone.types.DataType; import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.StringType; import java.util.Objects; @@ -65,4 +66,8 @@ public static Literal of(int value) { public static Literal of(boolean value) { return new Literal(value, new BooleanType()); } + + public static Literal of(String value) { + return new Literal(value, new StringType()); + } } diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Util.java b/standalone/src/main/java/io/delta/standalone/expressions/Util.java index ff173d774fc..d30f2fdaabe 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Util.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Util.java @@ -3,6 +3,7 @@ import io.delta.standalone.types.BooleanType; import io.delta.standalone.types.DataType; import io.delta.standalone.types.IntegerType; +import io.delta.standalone.types.StringType; public final class Util { @@ -15,6 +16,10 @@ public static CastingComparator createCastingComparator(DataType dataType) { return new CastingComparator(); } + if (dataType instanceof StringType) { + return new CastingComparator(); + } + throw new RuntimeException("Couldn't find matching comparator for DataType: " + dataType.toString()); } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala index ac67b32b360..04cf54f8db6 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala @@ -74,15 +74,16 @@ private[internal] class ConflictChecker( winningCommitVersion: Long, isolationLevel: IsolationLevel) { + private val deltaLog = currentTransactionInfo.deltaLog private val winningCommitSummary: WinningCommitSummary = createWinningCommitSummary() def checkConflicts(): Unit = { - // TODO checkProtocolCompatibility() - // TODO checkNoMetadataUpdates() + checkProtocolCompatibility() + checkNoMetadataUpdates() checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn() checkForDeletedFilesAgainstCurrentTxnReadFiles() - // TODO checkForDeletedFilesAgainstCurrentTxnDeletedFiles() - // TODO checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn() + checkForDeletedFilesAgainstCurrentTxnDeletedFiles() + checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn() } /** @@ -96,11 +97,39 @@ private[internal] class ConflictChecker( WinningCommitSummary(winningCommitActions, winningCommitVersion) } + /** + * Asserts that the client is up to date with the protocol and is allowed to read and write + * against the protocol set by the committed transaction. + */ + private def checkProtocolCompatibility(): Unit = { + if (winningCommitSummary.protocol.nonEmpty) { + winningCommitSummary.protocol.foreach { p => + deltaLog.assertProtocolRead(p) + deltaLog.assertProtocolWrite(p) + } + currentTransactionInfo.actions.foreach { + case Protocol(_, _) => + throw DeltaErrors.protocolChangedException(winningCommitSummary.commitInfo) + case _ => + } + } + } + + /** + * Check if the committed transaction has changed metadata. + */ + private def checkNoMetadataUpdates(): Unit = { + // Fail if the metadata is different than what the txn read. + if (winningCommitSummary.metadataUpdates.nonEmpty) { + throw DeltaErrors.metadataChangedException(winningCommitSummary.commitInfo) + } + } + /** * Check if the new files added by the already committed transactions should have been read by * the current transaction. */ - def checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn(): Unit = { + private def checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn(): Unit = { // Fail if new files have been added that the txn should have read. val addedFilesToCheckForConflicts = isolationLevel match { case Serializable => @@ -109,14 +138,27 @@ private[internal] class ConflictChecker( Seq.empty } - // TODO + val predicatesMatchingAddedFiles = currentTransactionInfo.readPredicates.flatMap { p => + val conflictingFile = DeltaLogImpl.filterFileList( + currentTransactionInfo.metadata.partitionSchema, + addedFilesToCheckForConflicts, + p :: Nil + ).headOption + + conflictingFile.map(f => getPrettyPartitionMessage(f.partitionValues)) + }.headOption + + if (predicatesMatchingAddedFiles.isDefined) { + throw DeltaErrors.concurrentAppendException( + winningCommitSummary.commitInfo, predicatesMatchingAddedFiles.get) + } } /** * Check if [[RemoveFile]] actions added by already committed transactions conflicts with files * read by the current transaction. */ - def checkForDeletedFilesAgainstCurrentTxnReadFiles(): Unit = { + private def checkForDeletedFilesAgainstCurrentTxnReadFiles(): Unit = { // Fail if files have been deleted that the txn read. val readFilePaths = currentTransactionInfo.readFiles.map( f => f.path -> f.partitionValues).toMap @@ -135,6 +177,38 @@ private[internal] class ConflictChecker( } } + /** + * Check if [[RemoveFile]] actions added by already committed transactions conflicts with + * [[RemoveFile]] actions this transaction is trying to add. + */ + private def checkForDeletedFilesAgainstCurrentTxnDeletedFiles(): Unit = { + // Fail if a file is deleted twice. + val txnDeletes = currentTransactionInfo.actions + .collect { case r: RemoveFile => r } + .map(_.path).toSet + val deleteOverlap = winningCommitSummary.removedFiles.map(_.path).toSet intersect txnDeletes + if (deleteOverlap.nonEmpty) { + throw DeltaErrors.concurrentDeleteDeleteException( + winningCommitSummary.commitInfo, deleteOverlap.head) + } + } + + /** + * Checks if the winning transaction corresponds to some AppId on which current transaction + * also depends. + */ + private def checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn(): Unit = { + // Fail if the appIds seen by the current transaction has been updated by the winning + // transaction i.e. the winning transaction have [[SetTransaction]] corresponding to + // some appId on which current transaction depends on. Example - This can happen when + // multiple instances of the same streaming query are running at the same time. + val txnOverlap = winningCommitSummary.appLevelTransactions.map(_.appId).toSet intersect + currentTransactionInfo.readAppIds + if (txnOverlap.nonEmpty) { + throw DeltaErrors.concurrentTransactionException(winningCommitSummary.commitInfo) + } + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 450bedf54f5..563e6a45fc2 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -144,6 +144,16 @@ private[internal] class DeltaLogImpl private( } } + /** + * Asserts that the client is up to date with the protocol and + * allowed to read the table that is using the given `protocol`. + */ + def assertProtocolRead(protocol: Protocol): Unit = { + if (protocol != null && Action.readerVersion < protocol.minReaderVersion) { + throw new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, protocol) + } + } + /** * Asserts that the client is up to date with the protocol and * allowed to write to the table that is using the given `protocol`. diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index fadf4f311e9..7b954aa43cc 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -33,14 +33,20 @@ private[internal] class OptimisticTransactionImpl( snapshot: SnapshotImpl) extends OptimisticTransaction { import OptimisticTransactionImpl._ + /** Tracks the appIds that have been seen by this transaction. */ + private val readTxn = new ArrayBuffer[String] + /** * Tracks the data that could have been seen by recording the partition * predicates by which files have been queried by this transaction. */ - protected val readPredicates = new ArrayBuffer[Expression] + private val readPredicates = new ArrayBuffer[Expression] /** Tracks specific files that have been seen by this transaction. */ - protected val readFiles = new scala.collection.mutable.HashSet[AddFile] + private val readFiles = new scala.collection.mutable.HashSet[AddFile] + + /** Whether the whole table was read during the transaction. */ + private var readTheWholeTable = false /** Tracks if this transaction has already committed. */ private var committed = false @@ -113,7 +119,7 @@ private[internal] class OptimisticTransactionImpl( Some(isBlindAppend), Some(op.getOperationMetrics.asScala.toMap), if (op.getUserMetadata.isPresent) Some(op.getUserMetadata.get()) else None, - Some(engineInfo) + Some(engineInfo) // TODO: engineInfo-standalone-standaloneVersion ) preparedActions = commitInfo +: preparedActions @@ -190,12 +196,13 @@ private[internal] class OptimisticTransactionImpl( } override def readWholeTable(): Unit = { - + readPredicates += Literal.True + readTheWholeTable = true } override def txnVersion(id: String): Long = { - // TODO - 0L + readTxn += id + snapshot.transactions.getOrElse(id, -1L) } /////////////////////////////////////////////////////////////////////////// @@ -351,8 +358,8 @@ private[internal] class OptimisticTransactionImpl( val currentTransactionInfo = CurrentTransactionInfo( readPredicates = readPredicates, readFiles = readFiles.toSet, - readWholeTable = false, // TODO readTheWholeTable - readAppIds = Nil.toSet, // TODO: readTxn.toSet, + readWholeTable = readTheWholeTable, + readAppIds = readTxn.toSet, metadata = metadata, actions = actions, deltaLog = deltaLog) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index a8fe9c64344..4fb2467dbf7 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -67,12 +67,12 @@ private[internal] class SnapshotImpl( // Public API Methods /////////////////////////////////////////////////////////////////////////// - override def scan(): DeltaScan = new DeltaScanImpl(activeFiles) + override def scan(): DeltaScan = new DeltaScanImpl(activeFilesJ) override def scan(predicate: Expression): DeltaScan = - new DeltaScanImpl(activeFiles, Some(predicate)) + new DeltaScanImpl(activeFilesJ, Some(predicate)) - override def getAllFiles: java.util.List[AddFileJ] = activeFiles + override def getAllFiles: java.util.List[AddFileJ] = activeFilesJ override def getMetadata: MetadataJ = ConversionUtils.convertMetadata(state.metadata) @@ -158,25 +158,14 @@ private[internal] class SnapshotImpl( ) } - private lazy val activeFiles = state.activeFiles.map(ConversionUtils.convertAddFile).toList.asJava + private lazy val activeFilesJ = + state.activeFiles.map(ConversionUtils.convertAddFile).toList.asJava - /** - * Asserts that the client is up to date with the protocol and allowed - * to read the table that is using this Snapshot's `protocol`. - */ - private def assertProtocolRead(): Unit = { - if (null != protocolScala) { - val clientReadVersion = Action.readerVersion - val tblReadVersion = protocolScala.minReaderVersion - - if (clientReadVersion < tblReadVersion) { - throw new DeltaErrors.InvalidProtocolVersionException(Action.protocolVersion, protocolScala) - } - } - } + /** A map to look up transaction version by appId. */ + lazy val transactions: Map[String, Long] = setTransactions.map(t => t.appId -> t.version).toMap /** Complete initialization by checking protocol version. */ - assertProtocolRead() + deltaLog.assertProtocolRead(protocolScala) } private[internal] object SnapshotImpl { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala index e24fe44ef0f..980bc430909 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala @@ -19,7 +19,7 @@ package io.delta.standalone.internal.data import java.sql.{Date, Timestamp} import io.delta.standalone.data.{RowRecord => RowRecordJ} -import io.delta.standalone.types.{BooleanType, IntegerType, StructType} +import io.delta.standalone.types.{BooleanType, IntegerType, StringType, StructType} /** * A RowRecord representing a Delta Lake partition of Map(partitionKey -> partitionValue) @@ -75,7 +75,11 @@ private[internal] class PartitionRowRecord( override def getDouble(fieldName: String): Double = 0 // TODO - override def getString(fieldName: String): String = null // TODO + override def getString(fieldName: String): String = { + requireFieldExists(fieldName) + require(partitionFieldToType(fieldName).isInstanceOf[StringType]) + partitionValues(fieldName) + } override def getBinary(fieldName: String): Array[Byte] = null // TODO diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index cba0f85526f..05eed4506ca 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -19,6 +19,7 @@ package io.delta.standalone.internal.exception import java.io.{FileNotFoundException, IOException} import io.delta.standalone.internal.actions.{CommitInfo, Protocol} +import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.types.StructType import io.delta.standalone.exceptions._ @@ -176,15 +177,43 @@ private[internal] object DeltaErrors { def concurrentModificationExceptionMsg( baseMessage: String, commit: Option[CommitInfo]): String = { - // TODO - "" + baseMessage + + commit.map(ci => s"\nConflicting commit: ${JsonUtils.toJson(ci)}").getOrElse("") + + s"\nRefer to https://docs.delta.io/latest/concurrency-control.html for more details." + } + + def metadataChangedException( + conflictingCommit: Option[CommitInfo]): MetadataChangedException = { + val message = DeltaErrors.concurrentModificationExceptionMsg( + "The metadata of the Delta table has been changed by a concurrent update. " + + "Please try the operation again.", + conflictingCommit) + new MetadataChangedException(message) + } + + def protocolChangedException(conflictingCommit: Option[CommitInfo]): ProtocolChangedException = { + val additionalInfo = conflictingCommit.map { v => + if (v.version.getOrElse(-1) == 0) { + "This happens when multiple writers are writing to an empty directory. " + + "Creating the table ahead of time will avoid this conflict. " + } else { + "" + } + }.getOrElse("") + + val message = DeltaErrors.concurrentModificationExceptionMsg( + "The protocol version of the Delta table has been changed by a concurrent update. " + + additionalInfo + "Please try the operation again.", + conflictingCommit) + new ProtocolChangedException(message) } def concurrentAppendException( - conflictingCommit: Option[CommitInfo]): ConcurrentAppendException = { - // TODO: include partition? + conflictingCommit: Option[CommitInfo], + partition: String): ConcurrentAppendException = { val message = DeltaErrors.concurrentModificationExceptionMsg( - s"Files were added by a concurrent update. Please try the operation again.", + s"Files were added to $partition by a concurrent update. " + + s"Please try the operation again.", conflictingCommit) new ConcurrentAppendException(message) } @@ -199,6 +228,26 @@ private[internal] object DeltaErrors { new ConcurrentDeleteReadException(message) } + def concurrentDeleteDeleteException( + conflictingCommit: Option[CommitInfo], + file: String): ConcurrentDeleteDeleteException = { + val message = DeltaErrors.concurrentModificationExceptionMsg( + "This transaction attempted to delete one or more files that were deleted " + + s"(for example $file) by a concurrent update. Please try the operation again.", + conflictingCommit) + new ConcurrentDeleteDeleteException(message) + } + + def concurrentTransactionException( + conflictingCommit: Option[CommitInfo]): ConcurrentTransactionException = { + val message = DeltaErrors.concurrentModificationExceptionMsg( + s"This error occurs when multiple streaming queries are using the same checkpoint to write " + + "into this table. Did you run multiple instances of the same streaming query" + + " at the same time?", + conflictingCommit) + new ConcurrentTransactionException(message) + } + def maxCommitRetriesExceededException( attemptNumber: Int, attemptVersion: Long, diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index d836310b083..6c68126fa9f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -217,6 +217,7 @@ private[internal] object ConversionUtils { case x: CommitInfoJ => convertCommitInfoJ(x) case x: MetadataJ => convertMetadataJ(x) case x: ProtocolJ => convertProtocolJ(x) + case x: SetTransactionJ => convertSetTransactionJ(x) // TODO others case _ => throw new UnsupportedOperationException("cannot convert this Java Action") } @@ -298,4 +299,12 @@ private[internal] object ConversionUtils { ) } + def convertSetTransactionJ(external: SetTransactionJ): SetTransaction = { + SetTransaction( + external.getAppId, + external.getVerion, + external.getLastUpdated // implicit check this! + ) + } + } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index c6339f1a654..612a3a3d2b0 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -20,8 +20,9 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import io.delta.standalone.{DeltaLog, Operation} -import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} -import io.delta.standalone.expressions.Literal +import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDeleteDeleteException, ConcurrentDeleteReadException, ConcurrentTransactionException, MetadataChangedException, ProtocolChangedException} +import io.delta.standalone.expressions.{EqualTo, Expression, Literal} import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.ConversionUtils @@ -36,6 +37,10 @@ import org.scalatest.FunSuite class OptimisticTransactionSuite extends FunSuite { // scalastyle:on funsuite + + implicit def exprSeqToList[T <: Expression](seq: Seq[T]): java.util.List[Expression] = + seq.asInstanceOf[Seq[Expression]].asJava + val engineInfo = "test-engine-info" val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) @@ -63,8 +68,7 @@ class OptimisticTransactionSuite extends FunSuite { test: DeltaLog => Unit): Unit = { val schemaFields = partitionCols.map { p => new StructField(p, new StringType()) }.toArray val schema = new StructType(schemaFields) - // TODO val metadata = Metadata(partitionColumns = partitionCols, schemaString = schema.json) - val metadata = Metadata(partitionColumns = partitionCols) + val metadata = Metadata(partitionColumns = partitionCols, schemaString = schema.toJson) withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) @@ -398,14 +402,193 @@ class OptimisticTransactionSuite extends FunSuite { // checkForConflicts() tests /////////////////////////////////////////////////////////////////////////// - // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) + private def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { + fileActions.map { + case a: AddFile => a.copy(dataChange = false) + case r: RemoveFile => r.copy(dataChange = false) + case cdc: AddCDCFile => cdc // change files are always dataChange = false + } + } + + ////////////////////////////////// + // protocolChangedException tests + ////////////////////////////////// + + test("concurrent protocol update should fail") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + val tx2 = log.startTransaction() + tx2.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) + + assertThrows[ProtocolChangedException] { + tx1.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // metadataChangedException tests + ////////////////////////////////// + + test("concurrent metadata update should fail") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + val tx2 = log.startTransaction() + tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "foo"))) + tx2.commit(Nil, manualUpdate, engineInfo) + + assertThrows[MetadataChangedException] { + tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "bar"))) + tx1.commit(Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentAppend tests + ////////////////////////////////// + + test("block concurrent commit when read partition was appended to by concurrent write") { + withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // TX1 reads only P1 + val tx1Read = tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) + assert(tx1Read.asScala.map(_.getPath) == A_P1 :: Nil) + + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True :: Nil) + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentAppendException] { + // P1 was modified + tx1.commit(addC_P2 :: addE_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("block concurrent commit on full table scan") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.markFilesAsRead(Literal.True :: Nil) + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) + + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True :: Nil) + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentAppendException] { + tx1.commit(addE_P3 :: addF_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("no data change: allow data rearrange when new files concurrently added") { + // This tests the case when isolationLevel == SnapshotIsolation + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.markFilesAsRead(Literal.True :: Nil) + + val tx2 = log.startTransaction() + tx1.markFilesAsRead(Literal.True :: Nil) + tx2.commit(addE_P3 :: Nil, manualUpdate, engineInfo) - // TODO: test more ConcurrentAppendException + // tx1 rearranges files (dataChange = false) + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + manualUpdate, engineInfo) - // TODO: test more ConcurrentDeleteReadException (including readWholeTable) + assert(log.update().getAllFiles.asScala.map(_.getPath) == C_P1 :: E_P3 :: Nil) + } + } + + ////////////////////////////////// + // concurrentDeleteRead tests + ////////////////////////////////// + + test("block concurrent commit on read & delete conflicting partitions") { + // txn.readFiles will be non-empty, so this covers the first ConcurrentDeleteReadException + // case in checkForDeletedFilesAgainstCurrentTxnReadFiles + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // read P1 + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True :: Nil) + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteReadException] { + // P1 read by TX1 was removed by TX2 + tx1.commit(addE_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("readWholeTable should block concurrent delete") { + // txn.readFiles will be empty, so this covers the second ConcurrentDeleteReadException + // case in checkForDeletedFilesAgainstCurrentTxnReadFiles + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.readWholeTable() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteReadException] { + // tx1 reads the whole table but tx2 removes files before tx1 commits + tx1.commit(addB_P1 :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentDeleteDelete tests + ////////////////////////////////// + + test("block commit with concurrent removes on same file") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteDeleteException] { + // tx1 tries to remove the same file + tx1.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentTransaction tests + ////////////////////////////////// + + test("block concurrent set-txns with the same app id") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + tx1.txnVersion("t1") + + val winningTxn = log.startTransaction() + winningTxn.txnVersion("t1") + winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentTransactionException] { + tx1.commit(Nil, manualUpdate, engineInfo) + } + } + } + + // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) - // TODO: test checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn with SnapshotIsolation - // i.e. datachange = false + // TODO: readWholeTable tests // TODO: test Checkpoint > partialWriteVisible (==> useRename) From ffa0c90ffde4016a2c258e9496ec1d005f22c64f Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Thu, 30 Sep 2021 14:20:59 -0700 Subject: [PATCH 088/291] [DSW] [14] OptimisticTransaction.markFilesAsRead returns a DeltaScan (#160) * first pass working; refactored DeltaScanImpl into Base and Filtered versions; fixed usages and tests * update scaladoc * added test suite; added equals method to some expressions; changed FilteredDeltaScanImpl constructor * introduce snapshot.scanScala interfaces; add scaladoc explaining difference between PartitionUtils.filterFileList and FilteredDeltaScanImpl * add test for expression equality * improve expression equality test case --- .../standalone/OptimisticTransaction.java | 7 +- .../expressions/BinaryExpression.java | 23 +++++ .../delta/standalone/expressions/Column.java | 15 +++ .../expressions/LeafExpression.java | 10 ++ .../expressions/UnaryExpression.java | 18 ++++ .../util/ParquetToSparkSchemaConverter.java | 28 ------ .../standalone/internal/ConflictChecker.scala | 6 +- .../standalone/internal/DeltaLogImpl.scala | 25 +---- .../standalone/internal/DeltaScanImpl.scala | 52 ---------- .../internal/OptimisticTransactionImpl.scala | 39 +++----- .../standalone/internal/SnapshotImpl.scala | 26 ++++- .../internal/scan/DeltaScanImpl.scala | 89 +++++++++++++++++ .../internal/scan/FilteredDeltaScanImpl.scala | 57 +++++++++++ .../internal/util/PartitionUtils.scala | 95 +++++++++++++++++++ .../standalone/internal/DeltaScanSuite.scala | 81 ++++++++++++++++ .../standalone/internal/ExpressionSuite.scala | 74 ++++++++++----- .../internal/OptimisticTransactionSuite.scala | 25 +++-- 17 files changed, 489 insertions(+), 181 deletions(-) delete mode 100644 standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java delete mode 100644 standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/DeltaScanSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index eddc14a3cf7..67791f0be3a 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -39,10 +39,11 @@ public interface OptimisticTransaction { * readFiles were changed by TXN2. Thus there are no logical conflicts and TXN1 can commit at * table version N+1. * - * @param readPredicates Predicates used to determine which files were read. - * @return Files matching the given predicates. + * @param readPredicate Predicates used to determine which files were read. + * @return a {@link DeltaScan} containing the list of files matching the push portion of the + * readPredicate. */ - List markFilesAsRead(Iterable readPredicates); + DeltaScan markFilesAsRead(Expression readPredicate); /** * Records an update to the metadata that should be committed with this transaction. diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java index 2d02d6328f2..bc258662eed 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java @@ -4,6 +4,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Objects; /** * An expression with two inputs and one output. The output is by default evaluated to null @@ -18,6 +19,14 @@ public BinaryExpression(Expression left, Expression right) { this.right = right; } + public Expression getLeft() { + return left; + } + + public Expression getRight() { + return right; + } + @Override public final Object eval(RowRecord record) { Object leftResult = left.eval(record); @@ -35,4 +44,18 @@ public final Object eval(RowRecord record) { public List children() { return Arrays.asList(left, right); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + BinaryExpression that = (BinaryExpression) o; + return Objects.equals(left, that.left) && + Objects.equals(right, that.right); + } + + @Override + public int hashCode() { + return Objects.hash(left, right); + } } diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Column.java b/standalone/src/main/java/io/delta/standalone/expressions/Column.java index 856a4ecefac..69c956b59c2 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/Column.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/Column.java @@ -1,6 +1,7 @@ package io.delta.standalone.expressions; import java.util.Collections; +import java.util.Objects; import java.util.Set; import io.delta.standalone.data.RowRecord; @@ -58,6 +59,20 @@ public Set references() { return Collections.singleton(name); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Column column = (Column) o; + return Objects.equals(name, column.name) && + Objects.equals(dataType, column.dataType); + } + + @Override + public int hashCode() { + return Objects.hash(name, dataType); + } + @FunctionalInterface private interface RowRecordEvaluator { Object nullSafeEval(RowRecord record); diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java index 0df332e0e04..aafa1b0d5cc 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java @@ -2,10 +2,20 @@ import java.util.Collections; import java.util.List; +import java.util.Set; public abstract class LeafExpression implements Expression { @Override public List children() { return Collections.emptyList(); } + + @Override + public Set references() { + return Collections.emptySet(); + } + + public abstract boolean equals(Object o); + + public abstract int hashCode(); } diff --git a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java index febf9223ce8..96d33a3e7a5 100644 --- a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java +++ b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java @@ -4,6 +4,7 @@ import java.util.Collections; import java.util.List; +import java.util.Objects; /** * An expression with one input and one output. The output is by default evaluated to null @@ -16,6 +17,10 @@ public UnaryExpression(Expression child) { this.child = child; } + public Expression getChild() { + return child; + } + @Override public Object eval(RowRecord record) { Object childResult = child.eval(record); @@ -33,4 +38,17 @@ protected Object nullSafeEval(Object childResult) { public List children() { return Collections.singletonList(child); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UnaryExpression that = (UnaryExpression) o; + return Objects.equals(child, that.child); + } + + @Override + public int hashCode() { + return Objects.hash(child); + } } diff --git a/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java b/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java deleted file mode 100644 index 2dbfe86ee01..00000000000 --- a/standalone/src/main/java/io/delta/standalone/util/ParquetToSparkSchemaConverter.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.util; - -import io.delta.standalone.types.StructType; -import org.apache.parquet.schema.MessageType; - -public final class ParquetToSparkSchemaConverter { - - public static MessageType convert(StructType catalystSchema) { - // TODO - return null; - } -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala index 04cf54f8db6..a1ae058a552 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala @@ -19,7 +19,7 @@ package io.delta.standalone.internal import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.util.FileNames +import io.delta.standalone.internal.util.{FileNames, PartitionUtils} /** * A class representing different attributes of current transaction needed for conflict detection. @@ -139,10 +139,10 @@ private[internal] class ConflictChecker( } val predicatesMatchingAddedFiles = currentTransactionInfo.readPredicates.flatMap { p => - val conflictingFile = DeltaLogImpl.filterFileList( + val conflictingFile = PartitionUtils.filterFileList( currentTransactionInfo.metadata.partitionSchema, addedFilesToCheckForConflicts, - p :: Nil + p ).headOption conflictingFile.map(f => getPrettyPartitionMessage(f.partitionValues)) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 563e6a45fc2..a88225b72a1 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -25,13 +25,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import io.delta.standalone.{DeltaLog, OptimisticTransaction, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} -import io.delta.standalone.expressions.{And, Expression, Literal} -import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol} -import io.delta.standalone.internal.data.PartitionRowRecord +import io.delta.standalone.internal.actions.{Action, Metadata, Protocol} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.storage.LogStoreProvider import io.delta.standalone.internal.util.{Clock, ConversionUtils, FileNames, SystemClock} -import io.delta.standalone.types.StructType /** * Scala implementation of Java interface [[DeltaLog]]. @@ -202,24 +199,4 @@ private[standalone] object DeltaLogImpl { new DeltaLogImpl(hadoopConf, path, path.getParent, clock) } - - /** - * Filters the given [[AddFile]]s by the given `partitionFilters`, returning those that match. - * @param files The active files in the DeltaLog state, which contains the partition value - * information - * @param partitionFilters Filters on the partition columns - */ - private[internal] def filterFileList( - partitionSchema: StructType, - files: Seq[AddFile], - partitionFilters: Seq[Expression]): Seq[AddFile] = { - val expr = partitionFilters.reduceLeftOption(new And(_, _)).getOrElse(Literal.True) - // TODO: compressedExpr = ... - - files.filter { addFile => - val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) - val result = expr.eval(partitionRowRecord) - result.asInstanceOf[Boolean] - } - } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala deleted file mode 100644 index 4622090b69f..00000000000 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaScanImpl.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal - -import java.util.Optional - -import io.delta.standalone.DeltaScan -import io.delta.standalone.actions.{AddFile => AddFileJ} -import io.delta.standalone.data.CloseableIterator -import io.delta.standalone.expressions.Expression - -/** - * Scala implementation of Java interface [[DeltaScan]]. - * - * TODO this is currently a naive implementation, since - * a) it takes in the in-memory AddFiles. - * b) it uses the metadata.partitionColumns, but the metadata won't be known until the log files - * are scanned - */ -final class DeltaScanImpl( - files: java.util.List[AddFileJ], - expr: Option[Expression] = None) extends DeltaScan { - - override def getFiles: CloseableIterator[AddFileJ] = new CloseableIterator[AddFileJ] { - private val iter = files.iterator - - override def hasNext: Boolean = iter.hasNext - - override def next(): AddFileJ = iter.next() - - override def close(): Unit = { } - } - - override def getPushedPredicate: Optional[Expression] = Optional.empty() - - override def getResidualPredicate: Optional[Expression] = Optional.ofNullable(expr.orNull) - -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 7b954aa43cc..c45f622a1d3 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -21,11 +21,12 @@ import java.nio.file.FileAlreadyExistsException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import io.delta.standalone.{CommitResult, Operation, OptimisticTransaction} -import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} -import io.delta.standalone.expressions.{And, Column, Expression, Literal} +import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction} +import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ} +import io.delta.standalone.expressions.{Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.scan.FilteredDeltaScanImpl import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils, SchemaMergingUtils, SchemaUtils} private[internal] class OptimisticTransactionImpl( @@ -137,23 +138,16 @@ private[internal] class OptimisticTransactionImpl( } /** Returns files matching the given predicates. */ - override def markFilesAsRead( - _readPredicates: java.lang.Iterable[Expression]): java.util.List[AddFileJ] = { + override def markFilesAsRead(readPredicate: Expression): DeltaScan = { + val scan = snapshot.scanScala(readPredicate) + val matchedFiles = scan.getFilesScala - val partitionFilters = _readPredicates.asScala.filter { f => - isPredicatePartitionColumnsOnly(f, metadata.partitionColumns) - }.toSeq - - val matchedFiles = DeltaLogImpl.filterFileList( - metadata.partitionSchema, - snapshot.allFilesScala, - partitionFilters - ) - - readPredicates += partitionFilters.reduceLeftOption(new And(_, _)).getOrElse(Literal.True) + if (scan.getPushedPredicate.isPresent) { + readPredicates += scan.getPushedPredicate.get() + } readFiles ++= matchedFiles - matchedFiles.map(ConversionUtils.convertAddFile).asJava + scan } override def updateMetadata(metadataJ: MetadataJ): Unit = { @@ -420,15 +414,4 @@ private[internal] object OptimisticTransactionImpl { def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap } - - /** - * Does the predicate only contains partition columns? - */ - def isPredicatePartitionColumnsOnly( - condition: Expression, - partitionColumns: Seq[String]): Boolean = { - // TODO: name equality resolver - - condition.references().asScala.forall(partitionColumns.contains(_)) - } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 4fb2467dbf7..a4f36e5c64b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -31,6 +31,7 @@ import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.scan.{DeltaScanImpl, FilteredDeltaScanImpl} import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils} @@ -67,10 +68,13 @@ private[internal] class SnapshotImpl( // Public API Methods /////////////////////////////////////////////////////////////////////////// - override def scan(): DeltaScan = new DeltaScanImpl(activeFilesJ) + override def scan(): DeltaScan = new DeltaScanImpl(allFilesScala) override def scan(predicate: Expression): DeltaScan = - new DeltaScanImpl(activeFilesJ, Some(predicate)) + new FilteredDeltaScanImpl( + allFilesScala, + predicate, + metadataScala.partitionSchema) override def getAllFiles: java.util.List[AddFileJ] = activeFilesJ @@ -92,6 +96,20 @@ private[internal] class SnapshotImpl( // Internal-Only Methods /////////////////////////////////////////////////////////////////////////// + /** + * Returns an implementation that provides an accessor to the files as internal Scala + * [[AddFile]]s. This prevents us from having to replay the log internally, generate Scala + * actions, convert them to Java actions (as per the [[DeltaScan]] interface), and then + * convert them back to Scala actions. + */ + def scanScala(): DeltaScanImpl = new DeltaScanImpl(allFilesScala) + + def scanScala(predicate: Expression): DeltaScanImpl = + new FilteredDeltaScanImpl( + allFilesScala, + predicate, + metadataScala.partitionSchema) + def allFilesScala: Seq[AddFile] = state.activeFiles.toSeq def tombstonesScala: Seq[RemoveFile] = state.tombstones.toSeq def setTransactions: Seq[SetTransaction] = state.setTransactions @@ -227,8 +245,8 @@ private class InitialSnapshotImpl( SnapshotImpl.State(Protocol(), Metadata(), Nil, Nil, Nil, 0L, 0L, 1L, 1L, 0L, 0L) } - override def scan(): DeltaScan = new DeltaScanImpl(Nil.asJava) + override def scan(): DeltaScan = new DeltaScanImpl(Nil) override def scan(predicate: Expression): DeltaScan = - new DeltaScanImpl(Nil.asJava, Some(predicate)) + new FilteredDeltaScanImpl(Nil, predicate, metadataScala.partitionSchema) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala new file mode 100644 index 00000000000..955163617a1 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala @@ -0,0 +1,89 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.scan + +import java.util.{NoSuchElementException, Optional} + +import io.delta.standalone.DeltaScan +import io.delta.standalone.actions.{AddFile => AddFileJ} +import io.delta.standalone.data.CloseableIterator +import io.delta.standalone.expressions.Expression +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.util.ConversionUtils + +/** + * Scala implementation of Java interface [[DeltaScan]]. + * + * TODO this is currently a naive implementation, since + * a) it takes in the in-memory AddFiles. + * b) it uses the metadata.partitionColumns, but the metadata won't be known until the log files + * are scanned + */ +private[internal] class DeltaScanImpl(files: Seq[AddFile]) extends DeltaScan { + + /** + * Whether or not the given [[addFile]] should be returned during iteration. + */ + protected def accept(addFile: AddFile): Boolean = true + + /** + * This is a utility method for internal use cases where we need the filtered files + * as their Scala instances, instead of Java. + * + * Since this is for internal use, we can keep this as a [[Seq]]. + */ + def getFilesScala: Seq[AddFile] = files.filter(accept) + + // TODO: memory-optimized implementation + override def getFiles: CloseableIterator[AddFileJ] = new CloseableIterator[AddFileJ] { + private var nextValid: Option[AddFile] = None + private val iter = files.iterator + + // Initialize next valid element so that the first hasNext() and next() calls succeed + findNextValid() + + private def findNextValid(): Unit = { + while (iter.hasNext) { + val next = iter.next() + if (accept(next)) { + nextValid = Some(next) + return + } + } + + // No next valid found + nextValid = None + } + + override def hasNext: Boolean = { + nextValid.isDefined + } + + override def next(): AddFileJ = { + if (!hasNext) throw new NoSuchElementException() + val ret = ConversionUtils.convertAddFile(nextValid.get) + findNextValid() + ret + } + + override def close(): Unit = { } + } + + override def getPushedPredicate: Optional[Expression] = Optional.empty() + + override def getResidualPredicate: Optional[Expression] = Optional.empty() +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala new file mode 100644 index 00000000000..317427205d4 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala @@ -0,0 +1,57 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.scan + +import java.util.Optional + +import io.delta.standalone.expressions.Expression +import io.delta.standalone.types.StructType +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.data.PartitionRowRecord +import io.delta.standalone.internal.util.PartitionUtils + +/** + * An implementation of [[io.delta.standalone.DeltaScan]] that filters files and only returns + * those that match the [[getPushedPredicate]]. + * + * If the pushed predicate is empty, then all files are returned. + */ +final private[internal] class FilteredDeltaScanImpl( + files: Seq[AddFile], + expr: Expression, + partitionSchema: StructType) extends DeltaScanImpl(files) { + + private val partitionColumns = partitionSchema.getFieldNames.toSeq + + private val (metadataConjunction, dataConjunction) = + PartitionUtils.splitMetadataAndDataPredicates(expr, partitionColumns) + + override protected def accept(addFile: AddFile): Boolean = { + if (metadataConjunction.isEmpty) return true + + val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) + val result = metadataConjunction.get.eval(partitionRowRecord) + result.asInstanceOf[Boolean] + } + + override def getPushedPredicate: Optional[Expression] = + Optional.ofNullable(metadataConjunction.orNull) + + override def getResidualPredicate: Optional[Expression] = + Optional.ofNullable(dataConjunction.orNull) + +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala new file mode 100644 index 00000000000..045f46196a9 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala @@ -0,0 +1,95 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import scala.collection.JavaConverters._ + +import io.delta.standalone.expressions.{And, Expression, Literal} +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.data.PartitionRowRecord +import io.delta.standalone.types.StructType + +private[internal] object PartitionUtils { + + /** + * Filters the given [[AddFile]]s by the given `partitionFilters`, returning those that match. + * + * This is different from + * [[io.delta.standalone.internal.scan.FilteredDeltaScanImpl.getFilesScala]] in that this method + * already has the [[AddFile]]s in memory, whereas the `FilteredDeltaScanImpl` performs a + * memory-optimized replay to collect and filter the files. + * + * @param files The active files in the DeltaLog state, which contains the partition value + * information + * @param partitionFilter Filter on the partition columns + */ + def filterFileList( + partitionSchema: StructType, + files: Seq[AddFile], + partitionFilter: Expression): Seq[AddFile] = { + // TODO: compressedExpr = ... + + files.filter { addFile => + val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) + val result = partitionFilter.eval(partitionRowRecord) + result.asInstanceOf[Boolean] + } + } + + /** + * Partition the given condition into two optional conjunctive predicates M, D such that + * condition = M AND D, where we define: + * - M: conjunction of predicates that can be evaluated using metadata only. + * - D: conjunction of other predicates. + */ + def splitMetadataAndDataPredicates( + condition: Expression, + partitionColumns: Seq[String]): (Option[Expression], Option[Expression]) = { + val (metadataPredicates, dataPredicates) = splitConjunctivePredicates(condition) + .partition(isPredicateMetadataOnly(_, partitionColumns)) + + val metadataConjunction = if (metadataPredicates.isEmpty) { + None + } else { + Some(metadataPredicates.reduceLeftOption(new And(_, _)).getOrElse(Literal.True)) + } + + val dataConjunction = if (dataPredicates.isEmpty) { + None + } else { + Some(dataPredicates.reduceLeftOption(new And(_, _)).getOrElse(Literal.True)) + } + + (metadataConjunction, dataConjunction) + } + + /** + * Check if condition can be evaluated using only metadata (i.e. partition columns) + */ + def isPredicateMetadataOnly(condition: Expression, partitionColumns: Seq[String]): Boolean = { + // TODO: name equality resolver ? + condition.references().asScala.forall(partitionColumns.contains(_)) + } + + private def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case a: And => splitConjunctivePredicates(a.getLeft) ++ splitConjunctivePredicates(a.getRight) + case other => other :: Nil + } + } + +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaScanSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaScanSuite.scala new file mode 100644 index 00000000000..77a51c4b458 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaScanSuite.scala @@ -0,0 +1,81 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import scala.collection.JavaConverters._ + +import io.delta.standalone.expressions.{And, EqualTo, LessThan, Literal} +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.scan.FilteredDeltaScanImpl +import io.delta.standalone.internal.util.ConversionUtils +import io.delta.standalone.types.{IntegerType, StructField, StructType} +import org.scalatest.FunSuite + +class DeltaScanSuite extends FunSuite { + + private val schema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true), + new StructField("col3", new IntegerType(), true), + new StructField("col4", new IntegerType(), true) + )) + + private val partitionSchema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true) + )) + + private val files = (1 to 10).map { i => + val partitionValues = Map("col1" -> (i % 3).toString, "col2" -> (i % 2).toString) + AddFile(i.toString, partitionValues, 1L, 1L, dataChange = true) + } + + private val metadataConjunct = new EqualTo(schema.column("col1"), Literal.of(0)) + private val dataConjunct = new EqualTo(schema.column("col3"), Literal.of(5)) + + test("properly splits metadata (pushed) and data (residual) predicates") { + val mixedConjunct = new LessThan(schema.column("col2"), schema.column("col4")) + val filter = new And(new And(metadataConjunct, dataConjunct), mixedConjunct) + val scan = new FilteredDeltaScanImpl(files, filter, partitionSchema) + + assert(scan.getPushedPredicate.get == metadataConjunct) + assert(scan.getResidualPredicate.get == new And(dataConjunct, mixedConjunct)) + } + + test("filtered scan with a metadata (pushed) conjunct should return matched files") { + val filter = new And(metadataConjunct, dataConjunct) + val scan = new FilteredDeltaScanImpl(files, filter, partitionSchema) + + assert(scan.getFiles.asScala.toSeq.map(ConversionUtils.convertAddFileJ) == + files.filter(_.partitionValues("col1").toInt == 0)) + assert(scan.getFilesScala == files.filter(_.partitionValues("col1").toInt == 0)) + + assert(scan.getPushedPredicate.get == metadataConjunct) + assert(scan.getResidualPredicate.get == dataConjunct) + } + + test("filtered scan with only data (residual) predicate should return all files") { + val filter = dataConjunct + val scan = new FilteredDeltaScanImpl(files, filter, partitionSchema) + + assert(scan.getFiles.asScala.toSeq.map(ConversionUtils.convertAddFileJ) == files) + assert(scan.getFilesScala == files) + + assert(!scan.getPushedPredicate.isPresent) + assert(scan.getResidualPredicate.get == filter) + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala index fb67be04be1..8454339f5f8 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala @@ -20,9 +20,9 @@ import scala.collection.JavaConverters._ import io.delta.standalone.data.RowRecord import io.delta.standalone.expressions._ -import io.delta.standalone.internal.actions.AddFile -import io.delta.standalone.internal.OptimisticTransactionImpl.isPredicatePartitionColumnsOnly import io.delta.standalone.types.{IntegerType, StructField, StructType} +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.util.PartitionUtils // scalastyle:off funsuite import org.scalatest.FunSuite @@ -31,15 +31,15 @@ import org.scalatest.FunSuite class ExpressionSuite extends FunSuite { // scalastyle:on funsuite - private val dataSchema = new StructType(Array( - new StructField("col1", new IntegerType(), true), - new StructField("col2", new IntegerType(), true), - new StructField("col3", new IntegerType(), true))) - private val partitionSchema = new StructType(Array( new StructField("col1", new IntegerType(), true), new StructField("col2", new IntegerType(), true))) + private val dataSchema = new StructType(Array( + new StructField("col3", new IntegerType(), true), + new StructField("col4", new IntegerType(), true), + new StructField("col5", new IntegerType(), true))) + private def testPredicate( predicate: Expression, expectedResult: Boolean, @@ -52,10 +52,10 @@ class ExpressionSuite extends FunSuite { private def testPartitionFilter( partitionSchema: StructType, inputFiles: Seq[AddFile], - filters: Seq[Expression], + filter: Expression, expectedMatchedFiles: Seq[AddFile]): Unit = { - println("filters:\n\t" + filters.map(_.toString()).mkString("\n\t")) - val matchedFiles = DeltaLogImpl.filterFileList(partitionSchema, inputFiles, filters) + println("filter: " + filter.toString) + val matchedFiles = PartitionUtils.filterFileList(partitionSchema, inputFiles, filter) assert(matchedFiles.length == expectedMatchedFiles.length) assert(matchedFiles.forall(expectedMatchedFiles.contains(_))) } @@ -101,28 +101,26 @@ class ExpressionSuite extends FunSuite { val f1Expr2 = new EqualTo(partitionSchema.column("col2"), Literal.of(1)) val f1 = new And(f1Expr1, f1Expr2) - testPartitionFilter(partitionSchema, inputFiles, f1 :: Nil, add01 :: Nil) - testPartitionFilter(partitionSchema, inputFiles, f1Expr1 :: f1Expr2 :: Nil, add01 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f1, add01 :: Nil) val f2Expr1 = new LessThan(partitionSchema.column("col1"), Literal.of(1)) val f2Expr2 = new LessThan(partitionSchema.column("col2"), Literal.of(1)) val f2 = new And(f2Expr1, f2Expr2) - testPartitionFilter(partitionSchema, inputFiles, f2 :: Nil, add00 :: Nil) - testPartitionFilter(partitionSchema, inputFiles, f2Expr1 :: f2Expr2 :: Nil, add00 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f2, add00 :: Nil) val f3Expr1 = new EqualTo(partitionSchema.column("col1"), Literal.of(2)) val f3Expr2 = new LessThan(partitionSchema.column("col2"), Literal.of(1)) val f3 = new Or(f3Expr1, f3Expr2) testPartitionFilter( - partitionSchema, inputFiles, f3 :: Nil, Seq(add20, add21, add22, add00, add10)) + partitionSchema, inputFiles, f3, Seq(add20, add21, add22, add00, add10)) val inSet4 = (2 to 10).map(Literal.of).asJava val f4 = new In(partitionSchema.column("col1"), inSet4) - testPartitionFilter(partitionSchema, inputFiles, f4 :: Nil, add20 :: add21 :: add22 :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f4, add20 :: add21 :: add22 :: Nil) val inSet5 = (100 to 110).map(Literal.of).asJava val f5 = new In(partitionSchema.column("col1"), inSet5) - testPartitionFilter(partitionSchema, inputFiles, f5 :: Nil, Nil) + testPartitionFilter(partitionSchema, inputFiles, f5, Nil) } test("not null partition filter") { @@ -131,23 +129,49 @@ class ExpressionSuite extends FunSuite { val inputFiles = Seq(add0Null, addNull1) val f1 = new IsNotNull(partitionSchema.column("col1")) - testPartitionFilter(partitionSchema, inputFiles, f1 :: Nil, add0Null :: Nil) + testPartitionFilter(partitionSchema, inputFiles, f1, add0Null :: Nil) } - test("Expr.references() and OptimisticTransaction.isPredicatePartitionColumnsOnly()") { + test("Expr.references() and PredicateUtils.isPredicateMetadataOnly()") { val dataExpr = new And( - new LessThan(dataSchema.column("col1"), Literal.of(5)), + new LessThan(dataSchema.column("col3"), Literal.of(5)), new Or( - new EqualTo(dataSchema.column("col1"), dataSchema.column("col2")), - new EqualTo(dataSchema.column("col1"), dataSchema.column("col3")) + new EqualTo(dataSchema.column("col3"), dataSchema.column("col4")), + new EqualTo(dataSchema.column("col3"), dataSchema.column("col5")) ) ) assert(dataExpr.references().size() == 3) - val partitionExpr = new EqualTo(dataSchema.column("col1"), dataSchema.column("col2")) + val partitionExpr = new EqualTo(partitionSchema.column("col1"), partitionSchema.column("col2")) + + assert( + !PartitionUtils.isPredicateMetadataOnly(dataExpr, partitionSchema.getFieldNames.toSeq)) + + assert( + PartitionUtils.isPredicateMetadataOnly(partitionExpr, partitionSchema.getFieldNames.toSeq)) + } - assert(!isPredicatePartitionColumnsOnly(dataExpr, partitionSchema.getFieldNames.toSeq)) - assert(isPredicatePartitionColumnsOnly(partitionExpr, partitionSchema.getFieldNames.toSeq)) + test("expression equality") { + // BinaryExpression + val and = new And(partitionSchema.column("col1"), partitionSchema.column("col2")) + val andCopy = new And(partitionSchema.column("col1"), partitionSchema.column("col2")) + val and2 = new And(dataSchema.column("col3"), Literal.of(44)) + assert(and == andCopy) + assert(and != and2) + + // UnaryExpression + val not = new Not(new EqualTo(Literal.of(1), Literal.of(1))) + val notCopy = new Not(new EqualTo(Literal.of(1), Literal.of(1))) + val not2 = new Not(new EqualTo(Literal.of(45), dataSchema.column("col4"))) + assert(not == notCopy) + assert(not != not2) + + // LeafExpression + val col1 = partitionSchema.column("col1") + val col1Copy = partitionSchema.column("col1") + val col2 = partitionSchema.column("col2") + assert(col1 == col1Copy) + assert(col1 != col2) } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 612a3a3d2b0..e9d469211ca 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -38,9 +38,6 @@ import org.scalatest.FunSuite class OptimisticTransactionSuite extends FunSuite { // scalastyle:on funsuite - implicit def exprSeqToList[T <: Expression](seq: Seq[T]): java.util.List[Expression] = - seq.asInstanceOf[Seq[Expression]].asJava - val engineInfo = "test-engine-info" val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) @@ -392,7 +389,7 @@ class OptimisticTransactionSuite extends FunSuite { val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn2 = log2.startTransaction() - txn2.markFilesAsRead(java.util.Arrays.asList(Literal.True)) + txn2.markFilesAsRead(Literal.True) txn2.commit(add :: Nil, manualUpdate, engineInfo) verifyIsBlindAppend(2, expected = false) } @@ -453,11 +450,11 @@ class OptimisticTransactionSuite extends FunSuite { val schema = log.update().getMetadata.getSchema val tx1 = log.startTransaction() // TX1 reads only P1 - val tx1Read = tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) - assert(tx1Read.asScala.map(_.getPath) == A_P1 :: Nil) + val tx1Read = tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) + assert(tx1Read.getFiles.asScala.toSeq.map(_.getPath) == A_P1 :: Nil) val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True :: Nil) + tx2.markFilesAsRead(Literal.True) // TX2 modifies only P1 tx2.commit(addB_P1 :: Nil, manualUpdate, engineInfo) @@ -473,11 +470,11 @@ class OptimisticTransactionSuite extends FunSuite { val schema = log.update().getMetadata.getSchema val tx1 = log.startTransaction() // TX1 full table scan - tx1.markFilesAsRead(Literal.True :: Nil) - tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) + tx1.markFilesAsRead(Literal.True) + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True :: Nil) + tx2.markFilesAsRead(Literal.True) tx2.commit(addC_P2 :: addD_P2.remove :: Nil, manualUpdate, engineInfo) intercept[ConcurrentAppendException] { @@ -490,10 +487,10 @@ class OptimisticTransactionSuite extends FunSuite { // This tests the case when isolationLevel == SnapshotIsolation withLog(addA_P1 :: addB_P1 :: Nil) { log => val tx1 = log.startTransaction() - tx1.markFilesAsRead(Literal.True :: Nil) + tx1.markFilesAsRead(Literal.True) val tx2 = log.startTransaction() - tx1.markFilesAsRead(Literal.True :: Nil) + tx1.markFilesAsRead(Literal.True) tx2.commit(addE_P3 :: Nil, manualUpdate, engineInfo) // tx1 rearranges files (dataChange = false) @@ -516,11 +513,11 @@ class OptimisticTransactionSuite extends FunSuite { val schema = log.update().getMetadata.getSchema val tx1 = log.startTransaction() // read P1 - tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1")) :: Nil) + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) // tx2 commits before tx1 val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True :: Nil) + tx2.markFilesAsRead(Literal.True) tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) intercept[ConcurrentDeleteReadException] { From af9fd0e28a1500fd3c8047ca38855618d2523bfe Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 4 Oct 2021 12:19:51 -0700 Subject: [PATCH 089/291] [DSW] [11] Production updateMetadata (#152) --- .../io/delta/standalone/types/DataType.java | 5 + .../internal/OptimisticTransactionImpl.scala | 45 ++++-- .../internal/exception/DeltaErrors.scala | 9 +- .../internal/util/DataTypeParser.scala | 6 +- .../internal/util/SchemaMergingUtils.scala | 3 + .../internal/util/SchemaUtils.scala | 82 ++++++++-- .../internal/DeltaRetentionSuite.scala | 42 +++-- .../internal/DeltaRetentionSuiteBase.scala | 36 ++++- .../internal/OptimisticTransactionSuite.scala | 153 +++++++++++++++++- 9 files changed, 313 insertions(+), 68 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/types/DataType.java b/standalone/src/main/java/io/delta/standalone/types/DataType.java index 6e0cd566a9a..4649e55da8f 100644 --- a/standalone/src/main/java/io/delta/standalone/types/DataType.java +++ b/standalone/src/main/java/io/delta/standalone/types/DataType.java @@ -81,6 +81,11 @@ public String getCatalogString() { */ public String toJson() { return DataTypeParser.toJson(this); } + /** + * @return a pretty (i.e. indented) JSON (@code String} representation of the type + */ + public String toPrettyJson() { return DataTypeParser.toPrettyJson(this); } + /** * Builds a readable {@code String} representation of the {@code ArrayType} */ diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index c45f622a1d3..be1a72a724d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -162,20 +162,9 @@ private[internal] class OptimisticTransactionImpl( newProtocol = Some(Protocol()) } - latestMetadata = if (snapshot.metadataScala.schemaString == latestMetadata.schemaString) { - // Shortcut when the schema hasn't changed to avoid generating spurious schema change logs. - // It's fine if two different but semantically equivalent schema strings skip this special - // case - that indicates that something upstream attempted to do a no-op schema change, and - // we'll just end up doing a bit of redundant work in the else block. - latestMetadata - } else { - // TODO getJson() - // val fixedSchema = - // SchemaUtils.removeUnenforceableNotNullConstraints(metadata.schema).getJson() - // metadata.copy(schemaString = fixedSchema) - - latestMetadata - } + if (snapshot.metadataScala.schemaString != latestMetadata.schemaString) { + SchemaUtils.checkUnenforceableNotNullConstraints(latestMetadata.schema) + } // Remove the protocol version properties val noProtocolVersionConfig = latestMetadata.configuration.filter { @@ -403,8 +392,32 @@ private[internal] class OptimisticTransactionImpl( /** Creates new metadata with global Delta configuration defaults. */ private def withGlobalConfigDefaults(metadata: Metadata): Metadata = { - // TODO - metadata + // TODO DeltaConfigs.mergeGlobalConfigs + + val defaultConfigs = Map( + "deletedFileRetentionDuration" -> "604800000", // 1 week + "checkpointInterval" -> "10", + "enableExpiredLogCleanup" -> "true", + "logRetentionDuration" -> "2592000000", // 30 days + "appendOnly" -> "false" + ) + + // Priority is: + // 1. user-provided configs (via metadata.configuration) + // 2. global hadoop configs + // 3. default configs + val newMetadataConfig = defaultConfigs.keySet.map { key => + val value = if (metadata.configuration.contains(key)) { + metadata.configuration(key) + } else { + deltaLog.hadoopConf.get(key, defaultConfigs(key)) + } + + key -> value + }.toMap + + // User provided configs take precedence. + metadata.copy(configuration = newMetadataConfig) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 05eed4506ca..fd9732f027f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -20,7 +20,7 @@ import java.io.{FileNotFoundException, IOException} import io.delta.standalone.internal.actions.{CommitInfo, Protocol} import io.delta.standalone.internal.util.JsonUtils -import io.delta.standalone.types.StructType +import io.delta.standalone.types.{DataType, StructType} import io.delta.standalone.exceptions._ import org.apache.hadoop.fs.Path @@ -266,6 +266,13 @@ private[internal] object DeltaErrors { """.stripMargin) } + def nestedNotNullConstraint( + parent: String, nested: DataType, nestType: String): RuntimeException = { + new RuntimeException(s"The $nestType type of the field $parent contains a NOT NULL " + + s"constraint. Delta does not support NOT NULL constraints nested within arrays or maps. " + + s"Parsed $nestType type:\n${nested.toPrettyJson}") + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala index 7f96f708725..8fb3058db0d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala @@ -84,9 +84,9 @@ private[standalone] object DataTypeParser { s"Failed to convert the JSON string '${compact(render(other))}' to a data type.") } - def toJson(value : DataType): String = { - compact(render(dataTypeToJValue(value))) - } + def toJson(value: DataType): String = compact(render(dataTypeToJValue(value))) + + def toPrettyJson(value: DataType): String = pretty(render(dataTypeToJValue(value))) private def dataTypeToJValue(dataType: DataType): JValue = dataType match { case array: ArrayType => diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala index 2a76c3eb40d..8831af9cff3 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala @@ -18,6 +18,9 @@ package io.delta.standalone.internal.util import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType} +/** + * Utils to merge table schema with data schema. + */ private[internal] object SchemaMergingUtils { /** diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala index 32a3b06f485..56e3409574b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala @@ -17,7 +17,7 @@ package io.delta.standalone.internal.util import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.types.StructType +import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType} private[internal] object SchemaUtils { @@ -35,19 +35,79 @@ private[internal] object SchemaUtils { } /** - * Go through the schema to look for unenforceable NOT NULL constraints. By default we'll throw - * when they're encountered, but if this is suppressed through SQLConf they'll just be silently - * removed. - * - * Note that this should only be applied to schemas created from explicit user DDL - in other - * scenarios, the nullability information may be inaccurate and Delta should always coerce the - * nullability flag to true. + * This is a simpler version of Delta OSS SchemaUtils::typeAsNullable. Instead of returning the + * nullable DataType, returns true if the input `dt` matches the nullable DataType. */ - def removeUnenforceableNotNullConstraints(schema: StructType): StructType = { - // TODO - schema + private def matchesNullableType(dt: DataType): Boolean = dt match { + case s: StructType => s.getFields.forall { field => + field.isNullable && matchesNullableType(field.getDataType) + } + + case a: ArrayType => a.getElementType match { + case s: StructType => + a.containsNull() && matchesNullableType(s) + case _ => + a.containsNull() + } + + case m: MapType => (m.getKeyType, m.getValueType) match { + case (s1: StructType, s2: StructType) => + m.valueContainsNull() && matchesNullableType(s1) && matchesNullableType(s2) + case (s1: StructType, _) => + m.valueContainsNull() && matchesNullableType(s1) + case (_, s2: StructType) => + m.valueContainsNull() && matchesNullableType(s2) + case _ => true + } + + case _ => true } + /** + * Go through the schema to look for unenforceable NOT NULL constraints and throw when they're + * encountered. + */ + def checkUnenforceableNotNullConstraints(schema: StructType): Unit = { + def checkField(path: Seq[String], f: StructField): Unit = f.getDataType match { + case a: ArrayType => if (!matchesNullableType(a.getElementType)) { + throw DeltaErrors.nestedNotNullConstraint( + prettyFieldName(path :+ f.getName), a.getElementType, nestType = "element") + } + case m: MapType => + val keyTypeNullable = matchesNullableType(m.getKeyType) + val valueTypeNullable = matchesNullableType(m.getValueType) + + if (!keyTypeNullable) { + throw DeltaErrors.nestedNotNullConstraint( + prettyFieldName(path :+ f.getName), m.getKeyType, nestType = "key") + } + if (!valueTypeNullable) { + throw DeltaErrors.nestedNotNullConstraint( + prettyFieldName(path :+ f.getName), m.getValueType, nestType = "value") + } + case _ => // nothing + } + + def traverseColumns[E <: DataType](path: Seq[String], dt: E): Unit = dt match { + case s: StructType => + s.getFields.foreach { field => + checkField(path, field) + traverseColumns(path :+ field.getName, field.getDataType) + } + case a: ArrayType => + traverseColumns(path :+ "element", a.getElementType) + case m: MapType => + traverseColumns(path :+ "key", m.getKeyType) + traverseColumns(path :+ "value", m.getValueType) + case _ => // nothing + } + + traverseColumns(Seq.empty, schema) + } + + private def prettyFieldName(columnPath: Seq[String]): String = + columnPath.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + private object ParquetSchemaConverter { def checkFieldName(name: String): Unit = { // ,;{}()\n\t= and space are special characters in Parquet schema diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala index 4fd53e85e29..4159668dedb 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala @@ -22,19 +22,16 @@ import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, RemoveFi import io.delta.standalone.internal.util.ManualClock import io.delta.standalone.internal.util.TestUtils._ import io.delta.standalone.Operation -import org.apache.hadoop.conf.Configuration - // scalastyle:off funsuite import org.scalatest.FunSuite // scalastyle:off removeFile -class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { +class DeltaRetentionSuite extends DeltaRetentionSuiteBase { // scalastyle:on funsuite val writerId = "test-writer-id" val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) - val manualLogCleanupMetadata = Metadata(configuration = Map("enableExpiredLogCleanup" -> "false")) protected def getLogFiles(dir: File): Seq[File] = getDeltaFiles(dir) ++ getCheckpointFiles(dir) @@ -42,18 +39,17 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { test("delete expired logs") { withTempDir { dir => val clock = new ManualClock(System.currentTimeMillis()) - val log = DeltaLogImpl.forTable(new Configuration(), dir.getCanonicalPath, clock) + val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath, clock) val logPath = new File(log.logPath.toUri) (1 to 5).foreach { i => - val txn = log.startTransaction() - val metadata = if (i == 1) manualLogCleanupMetadata :: Nil else Nil + val txn = if (i == 1) startTxnWithManualLogCleanup(log) else log.startTransaction() val file = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil val delete: Seq[Action] = if (i > 1) { RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil } else { Nil } - txn.commit(metadata ++ delete ++ file, manualUpdate, writerId) + txn.commit(delete ++ file, manualUpdate, writerId) } val initialFiles = getLogFiles(logPath) @@ -85,15 +81,13 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { test("delete expired logs 2") { withTempDir { dir => val clock = new ManualClock(System.currentTimeMillis()) - val conf = new Configuration() - val log = DeltaLogImpl.forTable(conf, dir.getCanonicalPath, clock) + val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath, clock) val logPath = new File(log.logPath.toUri) // write 000.json to 009.json (0 to 9).foreach { i => - val txn = log.startTransaction() - val metadata = if (i == 0) manualLogCleanupMetadata :: Nil else Nil - txn.commit(metadata :+ AddFile(i.toString, Map.empty, 1, 1, true), manualUpdate, writerId) + val txn = if (i == 0) startTxnWithManualLogCleanup(log) else log.startTransaction() + txn.commit(AddFile(i.toString, Map.empty, 1, 1, true) :: Nil, manualUpdate, writerId) } assert(log.update().version == 9) @@ -139,13 +133,15 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { test("Can set enableExpiredLogCleanup") { withTempDir { tempDir => - val log = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath) + val log = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath) log.startTransaction().commit( Metadata(configuration = Map("enableExpiredLogCleanup" -> "true")) :: Nil, manualUpdate, writerId) assert(log.enableExpiredLogCleanup) - log.startTransaction().commit(manualLogCleanupMetadata :: Nil, manualUpdate, writerId) + log.startTransaction().commit( + Metadata(configuration = Map("enableExpiredLogCleanup" -> "false")) :: Nil, + manualUpdate, writerId) assert(!log.enableExpiredLogCleanup) log.startTransaction().commit(Metadata() :: Nil, manualUpdate, writerId) @@ -157,17 +153,17 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { "RemoveFiles persist across checkpoints as tombstones if retention time hasn't expired") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log1 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + val log1 = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath, clock) - val txn1 = log1.startTransaction() + val txn1 = startTxnWithManualLogCleanup(log1) val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) - txn1.commit(files1 :+ manualLogCleanupMetadata, manualUpdate, writerId) + txn1.commit(files1, manualUpdate, writerId) val txn2 = log1.startTransaction() val files2 = (1 to 4).map(f => RemoveFile(f.toString, Some(clock.getTimeMillis()))) txn2.commit(files2, manualUpdate, writerId) log1.checkpoint() - val log2 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + val log2 = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath, clock) assert(log2.snapshot.tombstonesScala.size === 4) assert(log2.snapshot.allFilesScala.size === 6) } @@ -176,11 +172,11 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { test("RemoveFiles get deleted during checkpoint if retention time has passed") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log1 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + val log1 = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath, clock) - val txn = log1.startTransaction() + val txn1 = startTxnWithManualLogCleanup(log1) val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) - txn.commit(files1 :+ manualLogCleanupMetadata, manualUpdate, writerId) + txn1.commit(files1, manualUpdate, writerId) val txn2 = log1.startTransaction() val files2 = (1 to 4).map(f => RemoveFile(f.toString, Some(clock.getTimeMillis()))) txn2.commit(files2, manualUpdate, writerId) @@ -190,7 +186,7 @@ class DeltaRetentionSuite extends FunSuite with DeltaRetentionSuiteBase { log1.checkpoint() - val log2 = DeltaLogImpl.forTable(new Configuration(), tempDir.getCanonicalPath, clock) + val log2 = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath, clock) assert(log2.snapshot.tombstonesScala.size === 0) assert(log2.snapshot.allFilesScala.size === 6) } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala index bf867e66541..bd4ed41e573 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala @@ -18,12 +18,24 @@ package io.delta.standalone.internal import java.io.File -import io.delta.standalone.{DeltaLog, OptimisticTransaction} +import io.delta.standalone.{DeltaLog, Operation, OptimisticTransaction} import io.delta.standalone.internal.actions.Metadata +import io.delta.standalone.internal.util.TestUtils._ import io.delta.standalone.internal.util.{ConversionUtils, FileNames} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -trait DeltaRetentionSuiteBase { +// scalastyle:off funsuite +import org.scalatest.FunSuite + +trait DeltaRetentionSuiteBase extends FunSuite { + // scalastyle:on removeFile + + protected def hadoopConf: Configuration = { + val conf = new Configuration() + conf.set("enableExpiredLogCleanup", "false") + conf + } protected def getDeltaFiles(dir: File): Seq[File] = dir.listFiles().filter(_.getName.endsWith(".json")) @@ -35,10 +47,18 @@ trait DeltaRetentionSuiteBase { * Start a txn that disables automatic log cleanup. Some tests may need to manually clean up logs * to get deterministic behaviors. */ - // TODO: this is dependent on withGlobalConfigDefaults - // protected def startTxnWithManualLogCleanup(log: DeltaLog): OptimisticTransaction = { - // val txn = log.startTransaction() - // txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) - // txn - // } + protected def startTxnWithManualLogCleanup(log: DeltaLog): OptimisticTransaction = { + val txn = log.startTransaction() + txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + txn + } + + test("startTxnWithManualLogCleanup") { + withTempDir { dir => + val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath) + startTxnWithManualLogCleanup(log) + .commit(Nil, new Operation(Operation.Name.MANUAL_UPDATE), "test-writer-id") + assert(!log.enableExpiredLogCleanup) + } + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index e9d469211ca..6032c210ee2 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -25,8 +25,8 @@ import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDele import io.delta.standalone.expressions.{EqualTo, Expression, Literal} import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.util.ConversionUtils -import io.delta.standalone.types.{StringType, StructField, StructType} +import io.delta.standalone.internal.util.{ConversionUtils, SchemaUtils} +import io.delta.standalone.types._ import io.delta.standalone.internal.util.TestUtils._ import org.apache.hadoop.conf.Configuration @@ -323,6 +323,151 @@ class OptimisticTransactionSuite extends FunSuite { } } + test("unenforceable not null constraints") { + val validSchema = new StructType(Array( + new StructField( + "col1", + new MapType(new ArrayType(new StringType(), true), new IntegerType(), true), + true + ), + new StructField( + "col2", + new MapType(new IntegerType(), new ArrayType(new StringType(), true), true), + true + ), + new StructField( + "col3", + new ArrayType(new MapType(new StringType(), new IntegerType(), true), + true) + ) + )) + + SchemaUtils.checkUnenforceableNotNullConstraints(validSchema) // should not throw + + // case 1: not null within array + val inValidSchema1 = new StructType( + Array( + new StructField( + "arr", + new ArrayType( + new StructType( + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + false // arr (ArrayType) containsNull + ) + ) + ) + ) + + val e1 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema1) + }.getMessage + + assert(e1.contains("The element type of the field arr contains a NOT NULL constraint.")) + + // case 2: null within map key + val inValidSchema2 = new StructType( + Array( + new StructField( + "m", + new MapType( + new StructType( // m.key + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + new IntegerType(), // m.value + false // m (MapType) valueContainsNull + ) + ) + ) + ) + + val e2 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema2) + }.getMessage + + assert(e2.contains("The key type of the field m contains a NOT NULL constraint.")) + + // case 3: null within map key + val inValidSchema3 = new StructType( + Array( + new StructField( + "m", + new MapType( + new IntegerType(), // m.key + new StructType( // m.value + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + false // m (MapType) valueContainsNull + ) + ) + ) + ) + + val e3 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema3) + }.getMessage + + assert(e3.contains("The value type of the field m contains a NOT NULL constraint.")) + + // case 4: not null within nested array + val inValidSchema4 = new StructType( + Array( + new StructField( + "s", + new StructType( + Array( + new StructField("n", new IntegerType, false), + new StructField( + "arr", + new ArrayType( + new StructType( + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + true // arr (ArrayType) containsNull + ), + false // arr (StructField) nullable + ) + ) + ), + true // s (StructField) nullable + ) + ) + ) + + val e4 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema4) + }.getMessage + + assert(e4.contains("The element type of the field s.arr contains a NOT NULL constraint.")) + } + + test("updateMetadata withGlobalConfigDefaults") { + // TODO: use DeltaConfigs... + withTempDir { dir => + // note that the default for logRetentionDuration is 2592000000 + val hadoopConf = new Configuration() + hadoopConf.set("logRetentionDuration", "1000") + val metadata = Metadata(configuration = Map("logRetentionDuration" -> "2000")) + + val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + + assert(log.deltaRetentionMillis == 2000) + } + } + /////////////////////////////////////////////////////////////////////////// // verifyNewMetadata() tests /////////////////////////////////////////////////////////////////////////// @@ -349,10 +494,6 @@ class OptimisticTransactionSuite extends FunSuite { "Found partition columns having invalid character(s)") } - // TODO: test updateMetadata > unenforceable not null constraints removed from metadata schemaStr - - // TODO: test updateMetadata > withGlobalConfigDefaults - /////////////////////////////////////////////////////////////////////////// // commit() tests /////////////////////////////////////////////////////////////////////////// From 92b986e638dcafadb6c5cdff85802650dd8be9e9 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 5 Oct 2021 08:05:24 -0700 Subject: [PATCH 090/291] [DSW] [15] DSW-OSS Compatibility Unit Tests framework prototype (#161) --- .circleci/config.yml | 1 + build.sbt | 25 +++- .../internal/util/StandaloneUtil.scala | 78 +++++++++++ .../compatibility/OSSCompatibilitySuite.scala | 121 ++++++++++++++++++ .../standalone/util/ComparisonUtil.scala | 93 ++++++++++++++ .../io/delta/standalone/util/OSSUtil.scala | 74 +++++++++++ .../standalone/OptimisticTransaction.java | 2 +- .../internal/OptimisticTransactionImpl.scala | 6 +- .../internal/OptimisticTransactionSuite.scala | 14 +- 9 files changed, 399 insertions(+), 15 deletions(-) create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala diff --git a/.circleci/config.yml b/.circleci/config.yml index 51689c86462..8795cf53f3e 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -13,6 +13,7 @@ jobs: build/sbt "++ 2.12.8 hiveMR/test" build/sbt "++ 2.12.8 hiveTez/test" build/sbt "++ 2.12.8 sqlDeltaImport/test" + build/sbt "++ 2.12.8 compatibility/test" build/sbt "++ 2.11.12 standalone/test" build/sbt "++ 2.11.12 hive/test" build/sbt "++ 2.11.12 hiveMR/test" diff --git a/build.sbt b/build.sbt index d24b6d1c638..0b7a5c1b962 100644 --- a/build.sbt +++ b/build.sbt @@ -23,11 +23,9 @@ crossScalaVersions in ThisBuild := Seq("2.12.8", "2.11.12") lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") lazy val testScalastyle = taskKey[Unit]("testScalastyle") -val sparkVersion = "2.4.3" val hadoopVersion = "3.1.0" val hiveVersion = "3.1.2" val tezVersion = "0.9.2" -val hiveDeltaVersion = "0.5.0" lazy val commonSettings = Seq( organization := "io.delta", @@ -260,8 +258,8 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("org.slf4j", "slf4j-api"), ExclusionRule("org.apache.parquet", "parquet-hadoop") ), - "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.9.8", - "org.json4s" %% "json4s-jackson" % "3.5.3" excludeAll ( + "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.10.0", + "org.json4s" %% "json4s-jackson" % "3.7.0-M5" excludeAll ( ExclusionRule("com.fasterxml.jackson.core"), ExclusionRule("com.fasterxml.jackson.module") ), @@ -336,6 +334,23 @@ lazy val mimaSettings = Seq( mimaBinaryIssueFilters ++= StandaloneMimaExcludes.ignoredABIProblems ) +lazy val compatibility = (project in file("oss-compatibility-tests")) + .dependsOn(standalone) + .settings( + name := "compatibility", + commonSettings, + skipReleaseSettings, + libraryDependencies ++= Seq( + // Test Dependencies + "org.scalatest" %% "scalatest" % "3.1.0" % "test", + "org.apache.spark" % "spark-sql_2.12" % "3.1.1" % "test", + "io.delta" % "delta-core_2.12" % "1.0.0" % "test", + "commons-io" % "commons-io" % "2.8.0" % "test", + "org.apache.spark" % "spark-catalyst_2.12" % "3.1.1" % "test" classifier "tests", + "org.apache.spark" % "spark-core_2.12" % "3.1.1" % "test" classifier "tests", + "org.apache.spark" % "spark-sql_2.12" % "3.1.1" % "test" classifier "tests" + ) + ) lazy val goldenTables = (project in file("golden-tables")) settings ( name := "golden-tables", @@ -360,7 +375,7 @@ lazy val sqlDeltaImport = (project in file("sql-delta-import")) publishArtifact := scalaBinaryVersion.value == "2.12", publishArtifact in Test := false, libraryDependencies ++= Seq( - "org.apache.spark" %% "spark-sql" % sparkVersion % "provided", + "org.apache.spark" %% "spark-sql" % "2.4.3" % "provided", "io.delta" % "delta-core_2.12" % "0.7.0" % "provided", "org.rogach" %% "scallop" % "3.5.1", "org.scalatest" %% "scalatest" % "3.1.1" % "test", diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala new file mode 100644 index 00000000000..5fc839a696e --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala @@ -0,0 +1,78 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package io.delta.standalone.internal.util // need this to access internal.util.JsonUtils + +import java.util.Collections + +import scala.collection.JavaConverters._ + +import io.delta.standalone.Operation +import io.delta.standalone.actions.{AddFile, Format, Metadata, RemoveFile} +import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} + +object StandaloneUtil { + + val engineInfo = "standaloneEngineInfo" + + val schema = new StructType(Array( + new StructField("col1_part", new IntegerType(), true), + new StructField("col2_part", new StringType(), true) + )) + + val partitionColumns: Seq[String] = + schema.getFieldNames.filter(_.contains("part")).toSeq + + val op = new Operation(Operation.Name.MANUAL_UPDATE, Map[String, Object]( + "mode" -> "Append", + "partitionBy" -> JsonUtils.toJson(partitionColumns), + "predicate" -> "predicate_str" + ).asJava) + + val metadata: Metadata = Metadata.builder() + .id("id") + .name("name") + .description("description") + .format(new Format("parquet", Collections.singletonMap("format_key", "format_value"))) + .partitionColumns(partitionColumns.asJava) + .schema(schema) + .createdTime(1000L) + .build() + + val addFiles: Seq[AddFile] = (0 until 50).map { i => + new AddFile( + i.toString, // path + partitionColumns.map { col => col -> i.toString }.toMap.asJava, // partition values + 100L, // size + 1000, // modification time + true, // data change + null, // stats + Map("tag_key" -> "tag_val").asJava // tags + ) + } + + val removeFiles: Seq[RemoveFile] = addFiles.map { a => + RemoveFile.builder(a.getPath) + .deletionTimestamp(2000L) + .dataChange(true) + .extendedFileMetadata(true) + .partitionValues(a.getPartitionValues) + .size(a.getSize) + .tags(a.getTags) + .build() + } +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala new file mode 100644 index 00000000000..748ce90abd1 --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala @@ -0,0 +1,121 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.tests.compatibility + +import java.io.File +import java.nio.file.Files +import java.util.UUID + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{DeltaLog => StandaloneDeltaLog} +import io.delta.standalone.internal.util.StandaloneUtil +import io.delta.standalone.util.{ComparisonUtil, OSSUtil} + +import org.apache.spark.sql.delta.{DeltaLog => OSSDeltaLog} +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession + +class OSSCompatibilitySuite extends QueryTest with SharedSparkSession with ComparisonUtil { + + private val ss = StandaloneUtil + private val oo = OSSUtil + + /** + * Creates a temporary directory, a Standalone DeltaLog, and a DeltaOSS DeltaLog, which are all + * then passed to `f`. The temporary directory will be deleted after `f` returns. + */ + private def withTempDirAndLogs(f: (File, StandaloneDeltaLog, OSSDeltaLog) => Unit): Unit = { + val dir = Files.createTempDirectory(UUID.randomUUID().toString).toFile + + val standaloneLog = StandaloneDeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val ossLog = OSSDeltaLog.forTable(spark, dir.getCanonicalPath) + + try f(dir, standaloneLog, ossLog) finally { + FileUtils.deleteDirectory(dir) + } + } + + test("assert static actions are the same (without any writes/reads)") { + compareMetadata(ss.metadata, oo.metadata) + } + + /** + * For each (logType1, logType2, action) below, we will test the case of: + * logType1 write action (A1), logType2 read action (A2), assert A1 == A2 + * + * case 1a: standalone, oss, Metadata + * case 1b: oss, standalone, Metadata + * + * case 2a: standalone, oss, CommitInfo + * case 2b: oss, standalone, CommitInfo + * + * case 3a: standalone, oss, AddFile + * case 3b: oss, standalone, AddFile + * + * case 4a: standalone, oss, RemoveFile + * case 4b: oss, standalone, RemoveFile + */ + test("read/write actions") { + withTempDirAndLogs { (_, standaloneLog, ossLog) => + val standaloneTxn0 = standaloneLog.startTransaction() + standaloneTxn0.commit(Iterable(ss.metadata).asJava, ss.op, ss.engineInfo) + + // case 1a + compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) + + // case 2a + compareCommitInfo(standaloneLog.getCommitInfoAt(0), oo.getCommitInfoAt(ossLog, 0)) + + val ossTxn1 = ossLog.startTransaction() + ossTxn1.commit(Seq(oo.metadata), oo.op) + + // case 1b + compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) + + // case 2b + compareCommitInfo(standaloneLog.getCommitInfoAt(1), oo.getCommitInfoAt(ossLog, 1)) + + val standaloneTxn2 = standaloneLog.startTransaction() + standaloneTxn2.commit(ss.addFiles.asJava, ss.op, ss.engineInfo) + + // case 3a + compareAddFiles(standaloneLog.update(), ossLog.update()) + + val ossTxn3 = ossLog.startTransaction() + ossTxn3.commit(oo.addFiles, oo.op) + + // case 3b + compareAddFiles(standaloneLog.update(), ossLog.update()) + + val standaloneTxn4 = standaloneLog.startTransaction() + standaloneTxn4.commit(ss.removeFiles.asJava, ss.op, ss.engineInfo) + + // case 4a TODO + + } + } + + test("concurrency conflicts") { + withTempDirAndLogs { (dir, standaloneLog, ossLog) => + // TODO + } + } +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala new file mode 100644 index 00000000000..f53da11f90b --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala @@ -0,0 +1,93 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package io.delta.standalone.util + +import scala.collection.JavaConverters._ + +trait ComparisonUtil { + + private def compareOptions[J, S](a: java.util.Optional[J], b: Option[S]): Unit = { + assert(a.isPresent == b.isDefined) + if (a.isPresent) { + assert(a.get() == b.get) + } + } + + def compareMetadata( + standalone: io.delta.standalone.actions.Metadata, + oss: org.apache.spark.sql.delta.actions.Metadata): Unit = { + + assert(standalone.getId == oss.id) + assert(standalone.getName == oss.name) + assert(standalone.getDescription == oss.description) + compareFormat(standalone.getFormat, oss.format) + assert(standalone.getSchema.toJson == oss.schemaString) + assert(standalone.getPartitionColumns.asScala == oss.partitionColumns) + assert(standalone.getConfiguration.asScala == oss.configuration) + compareOptions(standalone.getCreatedTime, oss.createdTime) + } + + def compareFormat( + standalone: io.delta.standalone.actions.Format, + oss: org.apache.spark.sql.delta.actions.Format): Unit = { + + assert(standalone.getProvider == oss.provider) + assert(standalone.getOptions.asScala == oss.options) + } + + def compareCommitInfo( + standalone: io.delta.standalone.actions.CommitInfo, + oss: org.apache.spark.sql.delta.actions.CommitInfo): Unit = { + + // Do not compare `version`s. Standalone will inject the commitVersion using + // DeltaHistoryManager. To get the OSS commitInfo, we are just reading using the store, so + // the version is not injected. + + assert(standalone.getTimestamp == oss.timestamp) + compareOptions(standalone.getUserId, oss.userId) + compareOptions(standalone.getUserName, oss.userName) + assert(standalone.getOperation == oss.operation) + assert(standalone.getOperationParameters.asScala == oss.operationParameters) + // TODO: job + // TODO: notebook + compareOptions(standalone.getClusterId, oss.clusterId) + compareOptions(standalone.getReadVersion, oss.readVersion) + compareOptions(standalone.getIsolationLevel, oss.isolationLevel) + } + + def compareAddFiles( + standaloneSnapshot: io.delta.standalone.Snapshot, + ossSnapshot: org.apache.spark.sql.delta.Snapshot): Unit = { + val standaloneAddFilesMap2 = standaloneSnapshot.getAllFiles.asScala + .map { f => f.getPath -> f }.toMap + val ossAddFilesMap2 = ossSnapshot.allFiles.collect().map { f => f.path -> f }.toMap + + assert(standaloneAddFilesMap2.size == ossAddFilesMap2.size) + assert(standaloneAddFilesMap2.keySet == ossAddFilesMap2.keySet) + + standaloneAddFilesMap2.keySet.foreach { path => + compareAddFile(standaloneAddFilesMap2(path), ossAddFilesMap2(path)) + } + } + + def compareAddFile( + standalone: io.delta.standalone.actions.AddFile, + oss: org.apache.spark.sql.delta.actions.AddFile): Unit = { + // TODO + } +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala new file mode 100644 index 00000000000..33f56367e7d --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala @@ -0,0 +1,74 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package io.delta.standalone.util + +import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations} +import org.apache.spark.sql.delta.actions._ +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.SaveMode + +object OSSUtil { + + val schema: StructType = StructType(Array( + StructField("col1_part", IntegerType, nullable = true), + StructField("col2_part", StringType, nullable = true) + )) + + private val partitionColumns = schema.fieldNames.filter(_.contains("part")).toSeq + + val op: DeltaOperations.Write = + DeltaOperations.Write(SaveMode.Append, Some(partitionColumns), Some("predicate_str")) + + val metadata: Metadata = Metadata( + id = "id", + name = "name", + description = "description", + format = Format(provider = "parquet", options = Map("format_key" -> "format_value")), + partitionColumns = partitionColumns, + schemaString = schema.json, + createdTime = Some(1000L) + ) + + val addFiles: Seq[AddFile] = (0 until 50).map { i => + AddFile( + path = i.toString, + partitionValues = partitionColumns.map { col => col -> i.toString }.toMap, + size = 100L, + modificationTime = 1000L, + dataChange = true, + stats = null, + tags = Map("tag_key" -> "tag_val") + ) + } + + val removeFiles: Seq[RemoveFile] = addFiles.map(_.removeWithTimestamp(2000L, dataChange = true)) + + def getCommitInfoAt(log: DeltaLog, version: Long): CommitInfo = { + log.update() + + val firstChange = log.getChanges(version).next() + assert(firstChange._1 == version, s"getOssCommitInfoAt: expected first version to be $version" + + s"but got ${firstChange._1} instead.") + + val commitInfoOpt = firstChange._2.collectFirst { case c: CommitInfo => c } + assert(commitInfoOpt.isDefined, s"getOssCommitInfoAt: expected to find a CommitInfo action at" + + s"version $version, but none was found.") + + commitInfoOpt.get + } +} diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 67791f0be3a..4f8e983a43b 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -20,7 +20,7 @@ public interface OptimisticTransaction { * "{engineName}-{engineVersion}". * @return a {@link CommitResult}, wrapping the table version that was committed. */ - CommitResult commit(Iterable actions, Operation op, String engineInfo); + CommitResult commit(Iterable actions, Operation op, String engineInfo); /** * Mark files matched by the `readPredicates` as read by this transaction. diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index be1a72a724d..4783ef50f3f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction} -import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ} +import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.expressions.{Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors @@ -83,8 +83,8 @@ private[internal] class OptimisticTransactionImpl( // Public Java API Methods /////////////////////////////////////////////////////////////////////////// - override def commit( - actionsJ: java.lang.Iterable[ActionJ], + override def commit[T <: ActionJ]( + actionsJ: java.lang.Iterable[T], op: Operation, engineInfo: String): CommitResult = { val actions = actionsJ.asScala.map(ConversionUtils.convertActionJ).toSeq diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 6032c210ee2..bbad3b3f178 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -148,7 +148,7 @@ class OptimisticTransactionSuite extends FunSuite { val txn = log.startTransaction() txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) val e = intercept[AssertionError] { - txn.commit(Nil, manualUpdate, engineInfo) + txn.commit(Iterable().asJava, manualUpdate, engineInfo) } assert(e.getMessage.contains("Transaction already committed.")) } @@ -197,7 +197,7 @@ class OptimisticTransactionSuite extends FunSuite { val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() val e = intercept[IllegalStateException] { - txn.commit(Nil, manualUpdate, engineInfo) + txn.commit(Iterable().asJava, manualUpdate, engineInfo) } assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) } @@ -298,7 +298,7 @@ class OptimisticTransactionSuite extends FunSuite { Protocol.MIN_WRITER_VERSION_PROP -> "2" )) txn.updateMetadata(ConversionUtils.convertMetadata(metadata)) - txn.commit(Nil, manualUpdate, engineInfo) + txn.commit(Iterable().asJava, manualUpdate, engineInfo) val writtenConfig = log.update().getMetadata.getConfiguration assert(!writtenConfig.containsKey(Protocol.MIN_READER_VERSION_PROP)) @@ -508,6 +508,8 @@ class OptimisticTransactionSuite extends FunSuite { assert(commitInfo.getEngineInfo.isPresent) assert(commitInfo.getEngineInfo.get() == engineInfo) assert(commitInfo.getOperation == manualUpdate.getName.toString) + + // TODO: test commitInfo.operationParameters } } @@ -573,11 +575,11 @@ class OptimisticTransactionSuite extends FunSuite { val tx1 = log.startTransaction() val tx2 = log.startTransaction() tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "foo"))) - tx2.commit(Nil, manualUpdate, engineInfo) + tx2.commit(Iterable().asJava, manualUpdate, engineInfo) assertThrows[MetadataChangedException] { tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "bar"))) - tx1.commit(Nil, manualUpdate, engineInfo) + tx1.commit(Iterable().asJava, manualUpdate, engineInfo) } } } @@ -719,7 +721,7 @@ class OptimisticTransactionSuite extends FunSuite { winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, manualUpdate, engineInfo) intercept[ConcurrentTransactionException] { - tx1.commit(Nil, manualUpdate, engineInfo) + tx1.commit(Iterable().asJava, manualUpdate, engineInfo) } } } From 7090c51f04ccb3c1bd342953f296173759003d3c Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Tue, 5 Oct 2021 10:25:28 -0700 Subject: [PATCH 091/291] [DSW] [12] Production Checkpoint API (#154) --- .../standalone/internal/Checkpoints.scala | 2 +- .../standalone/internal/DeltaLogImpl.scala | 11 + .../standalone/internal/SnapshotImpl.scala | 15 +- .../BufferLogDeletionIteratorSuite.scala | 21 -- .../BufferingLogDeletionIteratorSuite.scala | 242 ++++++++++++++++++ .../internal/DeltaRetentionSuite.scala | 42 +++ 6 files changed, 298 insertions(+), 35 deletions(-) delete mode 100644 standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/BufferingLogDeletionIteratorSuite.scala diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index d9a4424bdcd..17bfbccc88e 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -238,7 +238,7 @@ private[internal] object Checkpoints { val writerOptions = ParquetWriter.Options( compressionCodecName = CompressionCodecName.SNAPPY, - timeZone = snapshot.readTimeZone // TODO: this should just be timeZone + timeZone = deltaLog.timezone ) val writer = ParquetWriter.writer[SingleAction](writtenPath, writerOptions) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index a88225b72a1..9c4ab9e1198 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -18,6 +18,7 @@ package io.delta.standalone.internal import java.io.IOException import java.util.concurrent.locks.ReentrantLock +import java.util.TimeZone import scala.collection.JavaConverters._ @@ -27,6 +28,7 @@ import io.delta.standalone.{DeltaLog, OptimisticTransaction, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} import io.delta.standalone.internal.actions.{Action, Metadata, Protocol} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.storage.LogStoreProvider import io.delta.standalone.internal.util.{Clock, ConversionUtils, FileNames, SystemClock} @@ -76,6 +78,15 @@ private[internal] class DeltaLogImpl private( // TODO: DeltaConfigs.CHECKPOINT_INTERVAL def checkpointInterval: Int = metadata.configuration.getOrElse("checkpointInterval", "10").toInt + /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ + def timezone: TimeZone = { + if (hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID) == null) { + TimeZone.getDefault + } else { + TimeZone.getTimeZone(hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID)) + } + } + /////////////////////////////////////////////////////////////////////////// // Public Java API Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index a4f36e5c64b..97ef1fe9204 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -53,17 +53,6 @@ private[internal] class SnapshotImpl( import SnapshotImpl._ - /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ - // TODO: this should be at the log level - // TODO: rename to timeZone - val readTimeZone = { - if (hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID) == null) { - TimeZone.getDefault - } else { - TimeZone.getTimeZone(hadoopConf.get(StandaloneHadoopConf.PARQUET_DATA_TIME_ZONE_ID)) - } - } - /////////////////////////////////////////////////////////////////////////// // Public API Methods /////////////////////////////////////////////////////////////////////////// @@ -89,7 +78,7 @@ private[internal] class SnapshotImpl( .map(FileNames.absolutePath(deltaLog.dataPath, _).toString), getMetadata.getSchema, // the time zone ID if it exists, else null - readTimeZone, + deltaLog.timezone, hadoopConf) /////////////////////////////////////////////////////////////////////////// @@ -126,7 +115,7 @@ private[internal] class SnapshotImpl( } else if (path.endsWith("parquet")) { ParquetReader.read[Parquet4sSingleActionWrapper]( path, ParquetReader.Options( - timeZone = readTimeZone, hadoopConf = hadoopConf) + timeZone = deltaLog.timezone, hadoopConf = hadoopConf) ).toSeq.map(_.unwrap) } else Seq.empty[SingleAction] }.toList diff --git a/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala deleted file mode 100644 index 1a66598928b..00000000000 --- a/standalone/src/test/scala/io/delta/standalone/internal/BufferLogDeletionIteratorSuite.scala +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal - -class BufferingLogDeletionIteratorSuite { - // TODO -} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/BufferingLogDeletionIteratorSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/BufferingLogDeletionIteratorSuite.scala new file mode 100644 index 00000000000..55f5a13573e --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/BufferingLogDeletionIteratorSuite.scala @@ -0,0 +1,242 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import org.apache.hadoop.fs.{FileStatus, Path} + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class BufferingLogDeletionIteratorSuite extends FunSuite { + // scalastyle:on funsuite + /** + * Creates FileStatus objects, where the name is the version of a commit, and the modification + * timestamps come from the input. + */ + private def createFileStatuses(modTimes: Long*): Iterator[FileStatus] = { + modTimes.zipWithIndex.map { case (time, version) => + new FileStatus(10L, false, 1, 10L, time, new Path(version.toString)) + }.iterator + } + + /** + * Creates a log deletion iterator with a retention `maxTimestamp` and `maxVersion` (both + * inclusive). The input iterator takes the original file timestamps, and the deleted output will + * return the adjusted timestamps of files that would actually be consumed by the iterator. + */ + private def testBufferingLogDeletionIterator( + maxTimestamp: Long, + maxVersion: Long)(inputTimestamps: Seq[Long], deleted: Seq[Long]): Unit = { + val i = new BufferingLogDeletionIterator( + createFileStatuses(inputTimestamps: _*), maxTimestamp, maxVersion, _.getName.toLong) + deleted.foreach { ts => + assert(i.hasNext, s"Was supposed to delete $ts, but iterator returned hasNext: false") + assert(i.next().getModificationTime === ts, "Returned files out of order!") + } + assert(!i.hasNext, "Iterator should be consumed") + } + + test("BufferingLogDeletionIterator: iterator behavior") { + val i1 = new BufferingLogDeletionIterator(Iterator.empty, 100, 100, _ => 1) + intercept[NoSuchElementException](i1.next()) + assert(!i1.hasNext) + + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 100)( + inputTimestamps = Seq(10), + deleted = Seq(10) + ) + + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 100)( + inputTimestamps = Seq(10, 15, 25), + deleted = Seq(10, 15, 25) + ) + } + + test("BufferingLogDeletionIterator: " + + "early exit while handling adjusted timestamps due to timestamp") { + // only should return 5 because 5 < 7 + testBufferingLogDeletionIterator(maxTimestamp = 7, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5) + ) + + // Should only return 5, because 10 is used to adjust the following 8 to 11 + testBufferingLogDeletionIterator(maxTimestamp = 10, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5) + ) + + // When it is 11, we can delete both 10 and 8 + testBufferingLogDeletionIterator(maxTimestamp = 11, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5, 10, 11) + ) + + // When it is 12, we can return all + testBufferingLogDeletionIterator(maxTimestamp = 12, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5, 10, 11, 12) + ) + + // Should only return 5, because 10 is used to adjust the following 8 to 11 + testBufferingLogDeletionIterator(maxTimestamp = 10, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8), + deleted = Seq(5) + ) + + // When it is 11, we can delete both 10 and 8 + testBufferingLogDeletionIterator(maxTimestamp = 11, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8), + deleted = Seq(5, 10, 11) + ) + } + + test("BufferingLogDeletionIterator: " + + "early exit while handling adjusted timestamps due to version") { + // only should return 5 because we can delete only up to version 0 + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 0)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5) + ) + + // Should only return 5, because 10 is used to adjust the following 8 to 11 + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 1)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5) + ) + + // When we can delete up to version 2, we can return up to version 2 + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 2)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5, 10, 11) + ) + + // When it is version 3, we can return all + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 3)( + inputTimestamps = Seq(5, 10, 8, 12), + deleted = Seq(5, 10, 11, 12) + ) + + // Should only return 5, because 10 is used to adjust the following 8 to 11 + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 1)( + inputTimestamps = Seq(5, 10, 8), + deleted = Seq(5) + ) + + // When we can delete up to version 2, we can return up to version 2 + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 2)( + inputTimestamps = Seq(5, 10, 8), + deleted = Seq(5, 10, 11) + ) + } + + test("BufferingLogDeletionIterator: multiple adjusted timestamps") { + Seq(9, 10, 11).foreach { retentionTimestamp => + // Files should be buffered but not deleted, because of the file 11, which has adjusted ts 12 + testBufferingLogDeletionIterator(maxTimestamp = retentionTimestamp, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 11, 14), + deleted = Seq(5) + ) + } + + // Safe to delete everything before (including) file: 11 which has adjusted timestamp 12 + testBufferingLogDeletionIterator(maxTimestamp = 12, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 11, 14), + deleted = Seq(5, 10, 11, 12) + ) + + Seq(0, 1, 2).foreach { retentionVersion => + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = retentionVersion)( + inputTimestamps = Seq(5, 10, 8, 11, 14), + deleted = Seq(5) + ) + } + + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 3)( + inputTimestamps = Seq(5, 10, 8, 11, 14), + deleted = Seq(5, 10, 11, 12) + ) + + // Test when the last element is adjusted with both timestamp and version + Seq(9, 10, 11).foreach { retentionTimestamp => + testBufferingLogDeletionIterator(maxTimestamp = retentionTimestamp, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 9), + deleted = Seq(5) + ) + } + + testBufferingLogDeletionIterator(maxTimestamp = 12, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 9), + deleted = Seq(5, 10, 11, 12) + ) + + Seq(0, 1, 2).foreach { retentionVersion => + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = retentionVersion)( + inputTimestamps = Seq(5, 10, 8, 9), + deleted = Seq(5) + ) + } + + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 3)( + inputTimestamps = Seq(5, 10, 8, 9), + deleted = Seq(5, 10, 11, 12) + ) + + Seq(9, 10, 11).foreach { retentionTimestamp => + testBufferingLogDeletionIterator(maxTimestamp = retentionTimestamp, maxVersion = 100)( + inputTimestamps = Seq(10, 8, 9), + deleted = Nil + ) + } + + // Test the first element causing cascading adjustments + testBufferingLogDeletionIterator(maxTimestamp = 12, maxVersion = 100)( + inputTimestamps = Seq(10, 8, 9), + deleted = Seq(10, 11, 12) + ) + + Seq(0, 1).foreach { retentionVersion => + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = retentionVersion)( + inputTimestamps = Seq(10, 8, 9), + deleted = Nil + ) + } + + testBufferingLogDeletionIterator(maxTimestamp = 100, maxVersion = 2)( + inputTimestamps = Seq(10, 8, 9), + deleted = Seq(10, 11, 12) + ) + + // Test multiple batches of time adjustments + testBufferingLogDeletionIterator(maxTimestamp = 12, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 9, 12, 15, 14, 14), // 5, 10, 11, 12, 13, 15, 16, 17 + deleted = Seq(5) + ) + + Seq(13, 14, 15, 16).foreach { retentionTimestamp => + testBufferingLogDeletionIterator(maxTimestamp = retentionTimestamp, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 9, 12, 15, 14, 14), // 5, 10, 11, 12, 13, 15, 16, 17 + deleted = Seq(5, 10, 11, 12, 13) + ) + } + + testBufferingLogDeletionIterator(maxTimestamp = 17, maxVersion = 100)( + inputTimestamps = Seq(5, 10, 8, 9, 12, 15, 14, 14), // 5, 10, 11, 12, 13, 15, 16, 17 + deleted = Seq(5, 10, 11, 12, 13, 15, 16, 17) + ) + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala index 4159668dedb..e0381e47667 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala @@ -191,4 +191,46 @@ class DeltaRetentionSuite extends DeltaRetentionSuiteBase { assert(log2.snapshot.allFilesScala.size === 6) } } + + test("the checkpoint file for version 0 should be cleaned") { + withTempDir { tempDir => + val now = System.currentTimeMillis() + val clock = new ManualClock(now) + val log = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath, clock) + val logPath = new File(log.logPath.toUri) + startTxnWithManualLogCleanup(log) + .commit(AddFile("0", Map.empty, 1, 1, true) :: Nil, manualUpdate, writerId) + log.checkpoint() + + val initialFiles = getLogFiles(logPath) + clock.advance(log.deltaRetentionMillis + 1000*60*60*24) // 1 day + + // Create a new checkpoint so that the previous version can be deleted + log.startTransaction() + .commit(AddFile("1", Map.empty, 1, 1, true) :: Nil, manualUpdate, writerId) + log.checkpoint() + + // We need to manually set the last modified timestamp to match that expected by the manual + // clock. If we don't, then sometimes the version 00 and version 01 log files will have the + // exact same lastModified time, since the local filesystem truncates the lastModified time + // to seconds instead of milliseconds. Here's what that looks like: + // + // _delta_log/00000000000000000000.checkpoint.parquet 1632267876000 + // _delta_log/00000000000000000000.json 1632267876000 + // _delta_log/00000000000000000001.checkpoint.parquet 1632267876000 + // _delta_log/00000000000000000001.json 1632267876000 + // + // By modifying the lastModified time, this better resembles the real-world lastModified + // times that the latest log files should have. + getLogFiles(logPath) + .filter(_.getName.contains("001.")) + .foreach(_.setLastModified(now + log.deltaRetentionMillis + 1000*60*60*24)) + + log.cleanUpExpiredLogs() + val afterCleanup = getLogFiles(logPath) + initialFiles.foreach { file => + assert(!afterCleanup.contains(file)) + } + } + } } From bca014e0386bad3c1115228d72521239eb70494a Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Oct 2021 10:26:55 -0700 Subject: [PATCH 092/291] Fix jackson incompatibility issue for Delta Hive 3 connector (#165) Found some jackson incompatibility issue when testing on an Amazon EMR 6.4.0. This PR fixes it by - Upgrade jackson-module-scala version - Upgrade json4s version - Shade jackson-module-scala to fix the incompatibility issue --- build.sbt | 21 ++++++++++++++----- .../standalone/internal/util/JsonUtils.scala | 3 +-- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/build.sbt b/build.sbt index d24b6d1c638..62db714c201 100644 --- a/build.sbt +++ b/build.sbt @@ -159,12 +159,23 @@ lazy val hive = (project in file("hive")) dependsOn(standalone) settings ( // default merge strategy assemblyShadeRules in assembly := Seq( /** - * Hive 2.3.7 uses an old paranamer version that doesn't support Scala 2.12 + * Hive uses an old paranamer version that doesn't support Scala 2.12 * (https://issues.apache.org/jira/browse/SPARK-22128), so we need to shade our own paranamer * version to avoid conflicts. */ - ShadeRule.rename("com.thoughtworks.paranamer.**" -> "shadedelta.@0").inAll - ) + ShadeRule.rename("com.thoughtworks.paranamer.**" -> "shadedelta.@0").inAll, + // Hive 3 now has jackson-module-scala on the classpath. We need to shade it otherwise we may + // pick up Hive's jackson-module-scala and use the above old paranamer jar on Hive's classpath. + ShadeRule.rename("com.fasterxml.jackson.module.scala.**" -> "shadedelta.@0").inAll + ), + assemblyMergeStrategy in assembly := { + // Discard `module-info.class` to fix the `different file contents found` error. + // TODO Upgrade SBT to 1.5 which will do this automatically + case "module-info.class" => MergeStrategy.discard + case x => + val oldStrategy = (assemblyMergeStrategy in assembly).value + oldStrategy(x) + } ) lazy val hiveMR = (project in file("hive-mr")) dependsOn(hive % "test->test") settings ( @@ -260,8 +271,8 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("org.slf4j", "slf4j-api"), ExclusionRule("org.apache.parquet", "parquet-hadoop") ), - "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.9.8", - "org.json4s" %% "json4s-jackson" % "3.5.3" excludeAll ( + "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.10.0", + "org.json4s" %% "json4s-jackson" % "3.7.0-M5" excludeAll ( ExclusionRule("com.fasterxml.jackson.core"), ExclusionRule("com.fasterxml.jackson.module") ), diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/JsonUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/JsonUtils.scala index 5b4769c3671..a1c6d8e37e3 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/JsonUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/JsonUtils.scala @@ -18,8 +18,7 @@ package io.delta.standalone.internal.util import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper +import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} /** Useful json functions used around the Delta codebase. */ private[internal] object JsonUtils { From a9e56819b241fa629f01fc5ed39b402d9011ff2d Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Thu, 7 Oct 2021 17:05:04 -0700 Subject: [PATCH 093/291] [DSW] [16] OSS Compatibility Tests v2 (#166) --- .../tests/OSSCompatibilitySuite.scala | 209 ++++++++++++++++++ .../internal/util/ComparisonUtil.scala | 153 +++++++++++++ .../{ => internal}/util/OSSUtil.scala | 14 +- .../internal/util/StandaloneUtil.scala | 17 +- .../compatibility/OSSCompatibilitySuite.scala | 121 ---------- .../standalone/util/ComparisonUtil.scala | 93 -------- .../standalone/internal/Checkpoints.scala | 2 +- .../standalone/internal/SnapshotImpl.scala | 16 +- 8 files changed, 391 insertions(+), 234 deletions(-) create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala rename oss-compatibility-tests/src/test/scala/io/delta/standalone/{ => internal}/util/OSSUtil.scala (87%) delete mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala delete mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala new file mode 100644 index 00000000000..f74bf269a5d --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala @@ -0,0 +1,209 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.compatibility.tests + +import java.io.File +import java.nio.file.Files +import java.util.UUID + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{DeltaLog => StandaloneDeltaLog} +import io.delta.standalone.internal.{DeltaLogImpl => InternalStandaloneDeltaLog} +import io.delta.standalone.internal.util.{ComparisonUtil, OSSUtil, StandaloneUtil} + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.delta.{DeltaLog => OSSDeltaLog} +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession + +class OSSCompatibilitySuite extends QueryTest with SharedSparkSession with ComparisonUtil { + + private val now = System.currentTimeMillis() + private val ss = new StandaloneUtil(now) + private val oo = new OSSUtil(now) + + /** + * Creates a temporary directory, a public Standalone DeltaLog, an internal Standalone DeltaLog, + * and a DeltaOSS DeltaLog, which are all then passed to `f`. + * + * The internal Standalone DeltaLog is used to gain access to internal, non-public Java APIs + * to verify internal state. + * + * The temporary directory will be deleted after `f` returns. + */ + private def withTempDirAndLogs( + f: (File, StandaloneDeltaLog, InternalStandaloneDeltaLog, OSSDeltaLog) => Unit): Unit = { + val dir = Files.createTempDirectory(UUID.randomUUID().toString).toFile + + val standaloneLog = StandaloneDeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val standaloneInternalLog = + InternalStandaloneDeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val ossLog = OSSDeltaLog.forTable(spark, dir.getCanonicalPath) + + try f(dir, standaloneLog, standaloneInternalLog, ossLog) finally { + FileUtils.deleteDirectory(dir) + } + } + + test("assert static actions are the same (without any writes/reads)") { + compareMetadata(ss.metadata, oo.metadata) + compareAddFiles(ss.addFiles, oo.addFiles) + compareRemoveFiles(ss.removeFiles, oo.removeFiles) + compareSetTransaction(ss.setTransaction, oo.setTransaction) + } + + /** + * For each (logType1, logType2, action) below, we will test the case of: + * logType1 write action (A1), logType2 read action (A2), assert A1 == A2 + * + * case 1a: standalone, oss, Metadata + * case 1b: oss, standalone, Metadata + * + * case 2a: standalone, oss, CommitInfo + * case 2b: oss, standalone, CommitInfo + * + * case 3a: standalone, oss, Protocol + * case 3b: oss, standalone, Protocol + * + * case 4a: standalone, oss, AddFile + * case 4b: oss, standalone, AddFile + * + * case 5a: standalone, oss, RemoveFile + * case 5b: oss, standalone, RemoveFile + * + * case 6a: standalone, oss, SetTransaction + * case 6b: oss, standalone, SetTransaction + */ + test("read/write actions") { + withTempDirAndLogs { (_, standaloneLog, standaloneInternalLog, ossLog) => + // === Standalone commit Metadata & CommitInfo === + val standaloneTxn0 = standaloneLog.startTransaction() + standaloneTxn0.commit(Iterable(ss.metadata).asJava, ss.op, ss.engineInfo) + + // case 1a + compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) + + // case 2a + compareCommitInfo(standaloneLog.getCommitInfoAt(0), oo.getCommitInfoAt(ossLog, 0)) + + // case 3a + compareProtocol(standaloneInternalLog.update().protocol, ossLog.snapshot.protocol) + + // === OSS commit Metadata & CommitInfo === + val ossTxn1 = ossLog.startTransaction() + ossTxn1.commit(oo.metadata :: Nil, oo.op) + + // case 1b + compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) + + // case 2b + compareCommitInfo(standaloneLog.getCommitInfoAt(1), oo.getCommitInfoAt(ossLog, 1)) + + // case 3b + compareProtocol(standaloneInternalLog.update().protocol, ossLog.snapshot.protocol) + + // === Standalone commit AddFiles === + val standaloneTxn2 = standaloneLog.startTransaction() + standaloneTxn2.commit(ss.addFiles.asJava, ss.op, ss.engineInfo) + + def assertAddFiles(): Unit = { + standaloneLog.update() + ossLog.update() + + val scanFiles = standaloneLog.snapshot().scan().getFiles.asScala.toSeq + assert(standaloneLog.snapshot().getAllFiles.size() == ss.addFiles.size) + assert(scanFiles.size == ss.addFiles.size) + assert(ossLog.snapshot.allFiles.count() == ss.addFiles.size) + + compareAddFiles( + standaloneLog.update().getAllFiles.asScala, ossLog.update().allFiles.collect()) + compareAddFiles(scanFiles, ossLog.update().allFiles.collect()) + } + + // case 4a + assertAddFiles() + + // === OSS commit AddFiles === + val ossTxn3 = ossLog.startTransaction() + ossTxn3.commit(oo.addFiles, oo.op) + + // case 4b + assertAddFiles() + + // === Standalone commit RemoveFiles === + val standaloneTxn4 = standaloneLog.startTransaction() + standaloneTxn4.commit(ss.removeFiles.asJava, ss.op, ss.engineInfo) + + def assertRemoveFiles(): Unit = { + standaloneLog.update() + standaloneInternalLog.update() + ossLog.update() + + assert(standaloneLog.snapshot().getAllFiles.isEmpty) + assert(ossLog.snapshot.allFiles.isEmpty) + assert(standaloneInternalLog.snapshot.tombstones.size == ss.removeFiles.size) + assert(ossLog.snapshot.tombstones.count() == ss.removeFiles.size) + compareRemoveFiles( + standaloneInternalLog.snapshot.tombstones, ossLog.snapshot.tombstones.collect()) + } + + // case 5a + assertRemoveFiles() + + // === OSS commit RemoveFiles === + val ossTxn5 = ossLog.startTransaction() + ossTxn5.commit(oo.removeFiles, oo.op) + + // case 5b + assertRemoveFiles() + + // === Standalone commit SetTransaction === + val standaloneTxn6 = standaloneLog.startTransaction() + standaloneTxn6.commit(Iterable(ss.setTransaction).asJava, ss.op, ss.engineInfo) + + def assertSetTransactions(): Unit = { + standaloneInternalLog.update() + ossLog.update() + assert(standaloneInternalLog.snapshot.setTransactionsScala.length == 1) + assert(ossLog.snapshot.setTransactions.length == 1) + compareSetTransaction( + standaloneInternalLog.snapshot.setTransactions.head, + ossLog.snapshot.setTransactions.head) + } + + // case 6a + assertSetTransactions() + + // === OSS commit SetTransaction === + val ossTxn7 = ossLog.startTransaction() + ossTxn7.commit(oo.setTransaction :: Nil, oo.op) + + // case 6b + assertSetTransactions() + } + } + + test("Standalone (with fixed Protocol(1, 2)) read from higher protocol OSS table") { + // TODO + } + + test("concurrency conflicts") { + // TODO + } +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala new file mode 100644 index 00000000000..ef6ed10502b --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala @@ -0,0 +1,153 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package io.delta.standalone.internal.util + +import scala.collection.JavaConverters._ + +trait ComparisonUtil { + + private def compareOptions(a: java.util.Optional[_], b: Option[_]): Unit = { + assert(a.isPresent == b.isDefined) + if (a.isPresent) { + assert(a.get() == b.get) + } + } + + private def compareNullableMaps(a: java.util.Map[_, _], b: Map[_, _]): Unit = { + if (null == a) { + assert(null == b) + } else { + assert(a.asScala == b) + } + } + + def compareMetadata( + standalone: io.delta.standalone.actions.Metadata, + oss: org.apache.spark.sql.delta.actions.Metadata): Unit = { + + assert(standalone.getId == oss.id) + assert(standalone.getName == oss.name) + assert(standalone.getDescription == oss.description) + compareFormat(standalone.getFormat, oss.format) + assert(standalone.getSchema.toJson == oss.schemaString) + assert(standalone.getPartitionColumns.asScala == oss.partitionColumns) + assert(standalone.getConfiguration.asScala == oss.configuration) + compareOptions(standalone.getCreatedTime, oss.createdTime) + } + + def compareFormat( + standalone: io.delta.standalone.actions.Format, + oss: org.apache.spark.sql.delta.actions.Format): Unit = { + + assert(standalone.getProvider == oss.provider) + assert(standalone.getOptions.asScala == oss.options) + } + + def compareCommitInfo( + standalone: io.delta.standalone.actions.CommitInfo, + oss: org.apache.spark.sql.delta.actions.CommitInfo): Unit = { + + // Do not compare `version`s. Standalone will inject the commitVersion using + // DeltaHistoryManager. To get the OSS commitInfo, we are just reading using the store, so + // the version is not injected. + + assert(standalone.getTimestamp == oss.timestamp) + compareOptions(standalone.getUserId, oss.userId) + compareOptions(standalone.getUserName, oss.userName) + assert(standalone.getOperation == oss.operation) + compareNullableMaps(standalone.getOperationParameters, oss.operationParameters) + // TODO: job + // TODO: notebook + compareOptions(standalone.getClusterId, oss.clusterId) + compareOptions(standalone.getReadVersion, oss.readVersion) + compareOptions(standalone.getIsolationLevel, oss.isolationLevel) + compareOptions(standalone.getIsBlindAppend, oss.isBlindAppend) + assert(standalone.getOperationMetrics.isPresent == oss.operationMetrics.isDefined) + if (standalone.getOperationMetrics.isPresent) { + compareNullableMaps(standalone.getOperationMetrics.get(), oss.operationMetrics.get) + } + compareOptions(standalone.getUserMetadata, oss.userMetadata) + } + + def compareProtocol( + standalone: io.delta.standalone.actions.Protocol, + oss: org.apache.spark.sql.delta.actions.Protocol): Unit = { + assert(standalone.getMinReaderVersion == oss.minReaderVersion) + assert(standalone.getMinWriterVersion == oss.minWriterVersion) + } + + def compareAddFiles( + standaloneFiles: Seq[io.delta.standalone.actions.AddFile], + ossFiles: Seq[org.apache.spark.sql.delta.actions.AddFile]): Unit = { + val standaloneAddFilesMap = standaloneFiles.map { f => f.getPath -> f }.toMap + val ossAddFilesMap = ossFiles.map { f => f.path -> f }.toMap + + assert(standaloneAddFilesMap.size == ossAddFilesMap.size) + assert(standaloneAddFilesMap.keySet == ossAddFilesMap.keySet) + + standaloneAddFilesMap.keySet.foreach { path => + compareAddFile(standaloneAddFilesMap(path), ossAddFilesMap(path)) + } + } + + private def compareAddFile( + standalone: io.delta.standalone.actions.AddFile, + oss: org.apache.spark.sql.delta.actions.AddFile): Unit = { + assert(standalone.getPath == oss.path) + compareNullableMaps(standalone.getPartitionValues, oss.partitionValues) + assert(standalone.getSize == oss.size) + assert(standalone.getModificationTime == oss.modificationTime) + assert(standalone.isDataChange == oss.dataChange) + assert(standalone.getStats == oss.stats) + compareNullableMaps(standalone.getTags, oss.tags) + } + + def compareRemoveFiles( + standaloneFiles: Seq[io.delta.standalone.actions.RemoveFile], + ossFiles: Seq[org.apache.spark.sql.delta.actions.RemoveFile]): Unit = { + val standaloneAddFilesMap2 = standaloneFiles.map { f => f.getPath -> f }.toMap + val ossAddFilesMap2 = ossFiles.map { f => f.path -> f }.toMap + + assert(standaloneAddFilesMap2.size == ossAddFilesMap2.size) + assert(standaloneAddFilesMap2.keySet == ossAddFilesMap2.keySet) + + standaloneAddFilesMap2.keySet.foreach { path => + compareRemoveFile(standaloneAddFilesMap2(path), ossAddFilesMap2(path)) + } + } + + def compareRemoveFile( + standalone: io.delta.standalone.actions.RemoveFile, + oss: org.apache.spark.sql.delta.actions.RemoveFile): Unit = { + assert(standalone.getPath == oss.path) + compareOptions(standalone.getDeletionTimestamp, oss.deletionTimestamp) + assert(standalone.isDataChange == oss.dataChange) + assert(standalone.isExtendedFileMetadata == oss.extendedFileMetadata) + compareNullableMaps(standalone.getPartitionValues, oss.partitionValues) + assert(standalone.getSize == oss.size) + compareNullableMaps(standalone.getTags, oss.tags) + } + + def compareSetTransaction( + standalone: io.delta.standalone.actions.SetTransaction, + oss: org.apache.spark.sql.delta.actions.SetTransaction): Unit = { + assert(standalone.getAppId == oss.appId) + assert(standalone.getVerion == oss.version) + compareOptions(standalone.getLastUpdated, oss.lastUpdated) + } +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala similarity index 87% rename from oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala rename to oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala index 33f56367e7d..51d87ba31b0 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/OSSUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala @@ -14,15 +14,14 @@ * limitations under the License. */ - -package io.delta.standalone.util +package io.delta.standalone.internal.util import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations} import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.SaveMode -object OSSUtil { +class OSSUtil(now: Long) { val schema: StructType = StructType(Array( StructField("col1_part", IntegerType, nullable = true), @@ -41,7 +40,7 @@ object OSSUtil { format = Format(provider = "parquet", options = Map("format_key" -> "format_value")), partitionColumns = partitionColumns, schemaString = schema.json, - createdTime = Some(1000L) + createdTime = Some(now) ) val addFiles: Seq[AddFile] = (0 until 50).map { i => @@ -49,14 +48,17 @@ object OSSUtil { path = i.toString, partitionValues = partitionColumns.map { col => col -> i.toString }.toMap, size = 100L, - modificationTime = 1000L, + modificationTime = now, dataChange = true, stats = null, tags = Map("tag_key" -> "tag_val") ) } - val removeFiles: Seq[RemoveFile] = addFiles.map(_.removeWithTimestamp(2000L, dataChange = true)) + val removeFiles: Seq[RemoveFile] = + addFiles.map(_.removeWithTimestamp(now + 100, dataChange = true)) + + val setTransaction: SetTransaction = SetTransaction("appId", 123, Some(now + 200)) def getCommitInfoAt(log: DeltaLog, version: Long): CommitInfo = { log.update() diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala index 5fc839a696e..a899e469bed 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala @@ -15,17 +15,17 @@ */ -package io.delta.standalone.internal.util // need this to access internal.util.JsonUtils +package io.delta.standalone.internal.util import java.util.Collections import scala.collection.JavaConverters._ -import io.delta.standalone.Operation -import io.delta.standalone.actions.{AddFile, Format, Metadata, RemoveFile} +import io.delta.standalone.actions.{AddFile, Format, Metadata, RemoveFile, SetTransaction} import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} +import io.delta.standalone.Operation -object StandaloneUtil { +class StandaloneUtil(now: Long) { val engineInfo = "standaloneEngineInfo" @@ -50,7 +50,7 @@ object StandaloneUtil { .format(new Format("parquet", Collections.singletonMap("format_key", "format_value"))) .partitionColumns(partitionColumns.asJava) .schema(schema) - .createdTime(1000L) + .createdTime(now) .build() val addFiles: Seq[AddFile] = (0 until 50).map { i => @@ -58,7 +58,7 @@ object StandaloneUtil { i.toString, // path partitionColumns.map { col => col -> i.toString }.toMap.asJava, // partition values 100L, // size - 1000, // modification time + now, // modification time true, // data change null, // stats Map("tag_key" -> "tag_val").asJava // tags @@ -67,7 +67,7 @@ object StandaloneUtil { val removeFiles: Seq[RemoveFile] = addFiles.map { a => RemoveFile.builder(a.getPath) - .deletionTimestamp(2000L) + .deletionTimestamp(now + 100) .dataChange(true) .extendedFileMetadata(true) .partitionValues(a.getPartitionValues) @@ -75,4 +75,7 @@ object StandaloneUtil { .tags(a.getTags) .build() } + + val setTransaction: SetTransaction = + new SetTransaction("appId", 123, java.util.Optional.of(now + 200)) } diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala deleted file mode 100644 index 748ce90abd1..00000000000 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/tests/compatibility/OSSCompatibilitySuite.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.tests.compatibility - -import java.io.File -import java.nio.file.Files -import java.util.UUID - -import scala.collection.JavaConverters._ - -import io.delta.standalone.{DeltaLog => StandaloneDeltaLog} -import io.delta.standalone.internal.util.StandaloneUtil -import io.delta.standalone.util.{ComparisonUtil, OSSUtil} - -import org.apache.spark.sql.delta.{DeltaLog => OSSDeltaLog} -import org.apache.commons.io.FileUtils -import org.apache.hadoop.conf.Configuration - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.SharedSparkSession - -class OSSCompatibilitySuite extends QueryTest with SharedSparkSession with ComparisonUtil { - - private val ss = StandaloneUtil - private val oo = OSSUtil - - /** - * Creates a temporary directory, a Standalone DeltaLog, and a DeltaOSS DeltaLog, which are all - * then passed to `f`. The temporary directory will be deleted after `f` returns. - */ - private def withTempDirAndLogs(f: (File, StandaloneDeltaLog, OSSDeltaLog) => Unit): Unit = { - val dir = Files.createTempDirectory(UUID.randomUUID().toString).toFile - - val standaloneLog = StandaloneDeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val ossLog = OSSDeltaLog.forTable(spark, dir.getCanonicalPath) - - try f(dir, standaloneLog, ossLog) finally { - FileUtils.deleteDirectory(dir) - } - } - - test("assert static actions are the same (without any writes/reads)") { - compareMetadata(ss.metadata, oo.metadata) - } - - /** - * For each (logType1, logType2, action) below, we will test the case of: - * logType1 write action (A1), logType2 read action (A2), assert A1 == A2 - * - * case 1a: standalone, oss, Metadata - * case 1b: oss, standalone, Metadata - * - * case 2a: standalone, oss, CommitInfo - * case 2b: oss, standalone, CommitInfo - * - * case 3a: standalone, oss, AddFile - * case 3b: oss, standalone, AddFile - * - * case 4a: standalone, oss, RemoveFile - * case 4b: oss, standalone, RemoveFile - */ - test("read/write actions") { - withTempDirAndLogs { (_, standaloneLog, ossLog) => - val standaloneTxn0 = standaloneLog.startTransaction() - standaloneTxn0.commit(Iterable(ss.metadata).asJava, ss.op, ss.engineInfo) - - // case 1a - compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) - - // case 2a - compareCommitInfo(standaloneLog.getCommitInfoAt(0), oo.getCommitInfoAt(ossLog, 0)) - - val ossTxn1 = ossLog.startTransaction() - ossTxn1.commit(Seq(oo.metadata), oo.op) - - // case 1b - compareMetadata(standaloneLog.update().getMetadata, ossLog.update().metadata) - - // case 2b - compareCommitInfo(standaloneLog.getCommitInfoAt(1), oo.getCommitInfoAt(ossLog, 1)) - - val standaloneTxn2 = standaloneLog.startTransaction() - standaloneTxn2.commit(ss.addFiles.asJava, ss.op, ss.engineInfo) - - // case 3a - compareAddFiles(standaloneLog.update(), ossLog.update()) - - val ossTxn3 = ossLog.startTransaction() - ossTxn3.commit(oo.addFiles, oo.op) - - // case 3b - compareAddFiles(standaloneLog.update(), ossLog.update()) - - val standaloneTxn4 = standaloneLog.startTransaction() - standaloneTxn4.commit(ss.removeFiles.asJava, ss.op, ss.engineInfo) - - // case 4a TODO - - } - } - - test("concurrency conflicts") { - withTempDirAndLogs { (dir, standaloneLog, ossLog) => - // TODO - } - } -} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala deleted file mode 100644 index f53da11f90b..00000000000 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/util/ComparisonUtil.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package io.delta.standalone.util - -import scala.collection.JavaConverters._ - -trait ComparisonUtil { - - private def compareOptions[J, S](a: java.util.Optional[J], b: Option[S]): Unit = { - assert(a.isPresent == b.isDefined) - if (a.isPresent) { - assert(a.get() == b.get) - } - } - - def compareMetadata( - standalone: io.delta.standalone.actions.Metadata, - oss: org.apache.spark.sql.delta.actions.Metadata): Unit = { - - assert(standalone.getId == oss.id) - assert(standalone.getName == oss.name) - assert(standalone.getDescription == oss.description) - compareFormat(standalone.getFormat, oss.format) - assert(standalone.getSchema.toJson == oss.schemaString) - assert(standalone.getPartitionColumns.asScala == oss.partitionColumns) - assert(standalone.getConfiguration.asScala == oss.configuration) - compareOptions(standalone.getCreatedTime, oss.createdTime) - } - - def compareFormat( - standalone: io.delta.standalone.actions.Format, - oss: org.apache.spark.sql.delta.actions.Format): Unit = { - - assert(standalone.getProvider == oss.provider) - assert(standalone.getOptions.asScala == oss.options) - } - - def compareCommitInfo( - standalone: io.delta.standalone.actions.CommitInfo, - oss: org.apache.spark.sql.delta.actions.CommitInfo): Unit = { - - // Do not compare `version`s. Standalone will inject the commitVersion using - // DeltaHistoryManager. To get the OSS commitInfo, we are just reading using the store, so - // the version is not injected. - - assert(standalone.getTimestamp == oss.timestamp) - compareOptions(standalone.getUserId, oss.userId) - compareOptions(standalone.getUserName, oss.userName) - assert(standalone.getOperation == oss.operation) - assert(standalone.getOperationParameters.asScala == oss.operationParameters) - // TODO: job - // TODO: notebook - compareOptions(standalone.getClusterId, oss.clusterId) - compareOptions(standalone.getReadVersion, oss.readVersion) - compareOptions(standalone.getIsolationLevel, oss.isolationLevel) - } - - def compareAddFiles( - standaloneSnapshot: io.delta.standalone.Snapshot, - ossSnapshot: org.apache.spark.sql.delta.Snapshot): Unit = { - val standaloneAddFilesMap2 = standaloneSnapshot.getAllFiles.asScala - .map { f => f.getPath -> f }.toMap - val ossAddFilesMap2 = ossSnapshot.allFiles.collect().map { f => f.path -> f }.toMap - - assert(standaloneAddFilesMap2.size == ossAddFilesMap2.size) - assert(standaloneAddFilesMap2.keySet == ossAddFilesMap2.keySet) - - standaloneAddFilesMap2.keySet.foreach { path => - compareAddFile(standaloneAddFilesMap2(path), ossAddFilesMap2(path)) - } - } - - def compareAddFile( - standalone: io.delta.standalone.actions.AddFile, - oss: org.apache.spark.sql.delta.actions.AddFile): Unit = { - // TODO - } -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 17bfbccc88e..b4b35967bfc 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -220,7 +220,7 @@ private[internal] object Checkpoints { val actions: Seq[SingleAction] = ( Seq(snapshot.metadataScala, snapshot.protocolScala) ++ - snapshot.setTransactions ++ + snapshot.setTransactionsScala ++ snapshot.allFilesScala ++ snapshot.tombstonesScala).map(_.wrap) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 97ef1fe9204..f67fd0e819d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -17,7 +17,6 @@ package io.delta.standalone.internal import java.net.URI -import java.util.TimeZone import scala.collection.JavaConverters._ @@ -25,14 +24,13 @@ import com.github.mjakubowski84.parquet4s.ParquetReader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import io.delta.standalone.{DeltaScan, Snapshot} -import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} +import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ, Protocol => ProtocolJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} import io.delta.standalone.expressions.Expression -import io.delta.standalone.internal.actions.{Action, AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction} +import io.delta.standalone.internal.actions.{AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.scan.{DeltaScanImpl, FilteredDeltaScanImpl} -import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils} /** @@ -99,9 +97,14 @@ private[internal] class SnapshotImpl( predicate, metadataScala.partitionSchema) + def tombstones: Seq[RemoveFileJ] = state.tombstones.toSeq.map(ConversionUtils.convertRemoveFile) + def setTransactions: Seq[SetTransactionJ] = + state.setTransactions.map(ConversionUtils.convertSetTransaction) + def protocol: ProtocolJ = ConversionUtils.convertProtocol(state.protocol) + def allFilesScala: Seq[AddFile] = state.activeFiles.toSeq def tombstonesScala: Seq[RemoveFile] = state.tombstones.toSeq - def setTransactions: Seq[SetTransaction] = state.setTransactions + def setTransactionsScala: Seq[SetTransaction] = state.setTransactions def protocolScala: Protocol = state.protocol def metadataScala: Metadata = state.metadata def numOfFiles: Long = state.numOfFiles @@ -169,7 +172,8 @@ private[internal] class SnapshotImpl( state.activeFiles.map(ConversionUtils.convertAddFile).toList.asJava /** A map to look up transaction version by appId. */ - lazy val transactions: Map[String, Long] = setTransactions.map(t => t.appId -> t.version).toMap + lazy val transactions: Map[String, Long] = + setTransactionsScala.map(t => t.appId -> t.version).toMap /** Complete initialization by checking protocol version. */ deltaLog.assertProtocolRead(protocolScala) From 2228c8442e23261ad03954da1b4967755034a3c4 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 8 Oct 2021 12:18:54 -0700 Subject: [PATCH 094/291] [DSW] [13] Production Log Store (#157) --- .../standalone/data/CloseableIterator.java | 1 + .../standalone/internal/Checkpoints.scala | 25 ++- .../standalone/internal/ConflictChecker.scala | 13 +- .../internal/DeltaHistoryManager.scala | 21 ++- .../standalone/internal/DeltaLogImpl.scala | 14 +- .../standalone/internal/MetadataCleanup.scala | 5 +- .../internal/OptimisticTransactionImpl.scala | 4 +- .../standalone/internal/SnapshotImpl.scala | 21 ++- .../internal/SnapshotManagement.scala | 8 +- .../internal/data/RowParquetRecordImpl.scala | 7 +- .../internal/storage/HDFSLogStore.scala | 58 +++---- .../storage/HadoopFileSystemLogStore.scala | 121 +++++++++++++ .../storage/LineCloseableIterator.scala | 74 ++++++++ .../internal/storage/LogStore.scala | 159 ------------------ .../internal/storage/LogStoreProvider.scala | 50 ++++++ .../standalone/internal/util/Implicits.scala | 42 +++++ .../standalone/internal/LogStoreSuite.scala | 49 +++--- 17 files changed, 421 insertions(+), 251 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala delete mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/Implicits.scala diff --git a/standalone/src/main/java/io/delta/standalone/data/CloseableIterator.java b/standalone/src/main/java/io/delta/standalone/data/CloseableIterator.java index 7a459b09a8d..73f27ab1799 100644 --- a/standalone/src/main/java/io/delta/standalone/data/CloseableIterator.java +++ b/standalone/src/main/java/io/delta/standalone/data/CloseableIterator.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.data; import java.io.Closeable; diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index b4b35967bfc..cbf8be9dec6 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -19,14 +19,16 @@ package io.delta.standalone.internal import java.io.FileNotFoundException import java.util.UUID -import com.github.mjakubowski84.parquet4s.ParquetWriter +import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, SingleAction} -import org.apache.hadoop.fs.Path +import io.delta.standalone.internal.actions.SingleAction import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.internal.util.FileNames._ +import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.metadata.CompressionCodecName +import com.github.mjakubowski84.parquet4s.ParquetWriter +import io.delta.standalone.data.CloseableIterator /** * Records information about a checkpoint. @@ -119,17 +121,18 @@ private[internal] trait Checkpoints { } val checkpointMetaData = Checkpoints.writeCheckpoint(this, snapshotToCheckpoint) val json = JsonUtils.toJson(checkpointMetaData) - store.write(LAST_CHECKPOINT, Iterator(json), overwrite = true) + store.write(LAST_CHECKPOINT, Iterator(json).asJava, true, hadoopConf) // overwrite = true doLogCleanup() } /** Loads the checkpoint metadata from the _last_checkpoint file. */ private def loadMetadataFromFile(tries: Int): Option[CheckpointMetaData] = { + var checkpointMetadataJson: CloseableIterator[String] = null try { - val checkpointMetadataJson = store.read(LAST_CHECKPOINT) + checkpointMetadataJson = store.read(LAST_CHECKPOINT, hadoopConf) val checkpointMetadata = - JsonUtils.mapper.readValue[CheckpointMetaData](checkpointMetadataJson.head) + JsonUtils.mapper.readValue[CheckpointMetaData](checkpointMetadataJson.next()) Some(checkpointMetadata) } catch { case _: FileNotFoundException => @@ -151,6 +154,10 @@ private[internal] trait Checkpoints { // CheckpointMetaData from it. val verifiedCheckpoint = findLastCompleteCheckpoint(CheckpointInstance(-1L, None)) verifiedCheckpoint.map(manuallyLoadCheckpoint) + } finally { + if (null != checkpointMetadataJson) { + checkpointMetadataJson.close() + } } } @@ -167,7 +174,9 @@ private[internal] trait Checkpoints { protected def findLastCompleteCheckpoint(cv: CheckpointInstance): Option[CheckpointInstance] = { var cur = math.max(cv.version, 0L) while (cur >= 0) { - val checkpoints = store.listFrom(checkpointPrefix(logPath, math.max(0, cur - 1000))) + val checkpoints = store + .listFrom(checkpointPrefix(logPath, math.max(0, cur - 1000)), hadoopConf) + .asScala .map(_.getPath) .filter(isCheckpointFile) .map(CheckpointInstance(_)) @@ -210,7 +219,7 @@ private[internal] object Checkpoints { // The writing of checkpoints doesn't go through log store, so we need to check with the // log store and decide whether to use rename. - val useRename = deltaLog.store.isPartialWriteVisible(deltaLog.logPath) + val useRename = deltaLog.store.isPartialWriteVisible(deltaLog.logPath, deltaLog.hadoopConf) var checkpointSize = 0L var numOfFiles = 0L diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala index a1ae058a552..b9aa675a60e 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala @@ -16,6 +16,8 @@ package io.delta.standalone.internal +import scala.collection.JavaConverters._ + import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors @@ -91,9 +93,14 @@ private[internal] class ConflictChecker( * transaction (winning transaction). */ private def createWinningCommitSummary(): WinningCommitSummary = { + import io.delta.standalone.internal.util.Implicits._ + val deltaLog = currentTransactionInfo.deltaLog - val winningCommitActions = deltaLog.store.read( - FileNames.deltaFile(deltaLog.logPath, winningCommitVersion)).map(Action.fromJson) + val winningCommitActions = deltaLog.store + .read(FileNames.deltaFile(deltaLog.logPath, winningCommitVersion), deltaLog.hadoopConf) + .toArray + .map(Action.fromJson) + WinningCommitSummary(winningCommitActions, winningCommitVersion) } @@ -222,7 +229,7 @@ private[internal] class ConflictChecker( val partition = partitionColumns.map { name => s"$name=${partitionValues(name)}" }.mkString("[", ", ", "]") - s"partition ${partition}" + s"partition $partition" } } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala index 680e70babd8..05212dd9153 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala @@ -18,11 +18,14 @@ package io.delta.standalone.internal import java.sql.Timestamp -import org.apache.hadoop.fs.Path +import scala.collection.JavaConverters._ + import io.delta.standalone.internal.actions.{Action, CommitInfo, CommitMarker} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.FileNames -import io.delta.standalone.internal.storage.LogStore +import io.delta.standalone.storage.LogStore + +import org.apache.hadoop.fs.Path /** * This class keeps tracks of the version of commits and their timestamps for a Delta table to @@ -34,8 +37,11 @@ private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { /** Get the persisted commit info for the given delta file. */ def getCommitInfo(version: Long): CommitInfo = { - val info = deltaLog.store.read(FileNames.deltaFile(deltaLog.logPath, version)) - .iterator + import io.delta.standalone.internal.util.Implicits._ + + val info = deltaLog.store + .read(FileNames.deltaFile(deltaLog.logPath, version), deltaLog.hadoopConf) + .toArray .map(Action.fromJson) .collectFirst { case c: CommitInfo => c } if (info.isEmpty) { @@ -98,7 +104,9 @@ private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { * commits are contiguous. */ private def getEarliestReproducibleCommitVersion: Long = { - val files = deltaLog.store.listFrom(FileNames.deltaFile(deltaLog.logPath, 0)) + val files = deltaLog.store + .listFrom(FileNames.deltaFile(deltaLog.logPath, 0), deltaLog.hadoopConf) + .asScala .filter(f => FileNames.isDeltaFile(f.getPath) || FileNames.isCheckpointFile(f.getPath)) // A map of checkpoint version and number of parts, to number of parts observed @@ -158,7 +166,8 @@ private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { logPath: Path, start: Long, end: Long): Array[Commit] = { - val commits = logStore.listFrom(FileNames.deltaFile(logPath, start)) + val commits = logStore.listFrom(FileNames.deltaFile(logPath, start), deltaLog.hadoopConf) + .asScala .filter(f => FileNames.isDeltaFile(f.getPath)) .map { fileStatus => Commit(FileNames.deltaVersion(fileStatus.getPath), fileStatus.getModificationTime) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 9c4ab9e1198..b98136408f2 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -101,9 +101,12 @@ private[internal] class DeltaLogImpl private( override def getChanges( startVersion: Long, failOnDataLoss: Boolean): java.util.Iterator[VersionLog] = { + import io.delta.standalone.internal.util.Implicits._ + if (startVersion < 0) throw new IllegalArgumentException(s"Invalid startVersion: $startVersion") - val deltaPaths = store.listFrom(FileNames.deltaFile(logPath, startVersion)) + val deltaPaths = store.listFrom(FileNames.deltaFile(logPath, startVersion), hadoopConf) + .asScala .filter(f => FileNames.isDeltaFile(f.getPath)) // Subtract 1 to ensure that we have the same check for the inclusive startVersion @@ -116,8 +119,13 @@ private[internal] class DeltaLogImpl private( } lastSeenVersion = version - new VersionLog(version, - store.read(p).map(x => ConversionUtils.convertAction(Action.fromJson(x))).toList.asJava) + new VersionLog( + version, + store.read(p, hadoopConf) + .toArray + .map(x => ConversionUtils.convertAction(Action.fromJson(x))) + .toList + .asJava) }.asJava } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala index 5ae830bd169..7fe45bda5e9 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala @@ -18,6 +18,8 @@ package io.delta.standalone.internal import java.util.{Calendar, TimeZone} +import scala.collection.JavaConverters._ + import io.delta.standalone.internal.util.FileNames.{checkpointPrefix, isCheckpointFile, isDeltaFile, checkpointVersion, deltaVersion} import org.apache.commons.lang.time.DateUtils @@ -66,7 +68,8 @@ private[internal] trait MetadataCleanup { val latestCheckpoint = lastCheckpoint if (latestCheckpoint.isEmpty) return Iterator.empty val threshold = latestCheckpoint.get.version - 1L - val files = store.listFrom(checkpointPrefix(logPath, 0)) + val files = store.listFrom(checkpointPrefix(logPath, 0), hadoopConf) + .asScala .filter(f => isCheckpointFile(f.getPath) || isDeltaFile(f.getPath)) def getVersion(filePath: Path): Long = { if (isCheckpointFile(filePath)) { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 4783ef50f3f..8b82a521e62 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -301,7 +301,9 @@ private[internal] class OptimisticTransactionImpl( deltaLog.lockInterruptibly { deltaLog.store.write( FileNames.deltaFile(deltaLog.logPath, attemptVersion), - actions.map(_.json).toIterator + actions.map(_.json).toIterator.asJava, + false, // overwrite = false + deltaLog.hadoopConf ) val postCommitSnapshot = deltaLog.update() diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index f67fd0e819d..8e4faa09b6b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -20,9 +20,6 @@ import java.net.URI import scala.collection.JavaConverters._ -import com.github.mjakubowski84.parquet4s.ParquetReader -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} import io.delta.standalone.{DeltaScan, Snapshot} import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ, Protocol => ProtocolJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} @@ -33,6 +30,10 @@ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.scan.{DeltaScanImpl, FilteredDeltaScanImpl} import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils} +import com.github.mjakubowski84.parquet4s.ParquetReader +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} + /** * Scala implementation of Java interface [[Snapshot]]. * @@ -112,13 +113,17 @@ private[internal] class SnapshotImpl( private def load(paths: Seq[Path]): Seq[SingleAction] = { paths.map(_.toString).sortWith(_ < _).par.flatMap { path => if (path.endsWith("json")) { - deltaLog.store.read(path).map { line => - JsonUtils.mapper.readValue[SingleAction](line) - } + import io.delta.standalone.internal.util.Implicits._ + deltaLog.store + .read(new Path(path), hadoopConf) + .toArray + .map { line => JsonUtils.mapper.readValue[SingleAction](line) } } else if (path.endsWith("parquet")) { ParquetReader.read[Parquet4sSingleActionWrapper]( - path, ParquetReader.Options( - timeZone = deltaLog.timezone, hadoopConf = hadoopConf) + path, + ParquetReader.Options( + timeZone = deltaLog.timezone, + hadoopConf = hadoopConf) ).toSeq.map(_.unwrap) } else Seq.empty[SingleAction] }.toList diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala index 5e2204fef70..05d09616a6b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala @@ -19,11 +19,13 @@ package io.delta.standalone.internal import java.io.FileNotFoundException import java.sql.Timestamp -import org.apache.hadoop.fs.{FileStatus, Path} +import scala.collection.JavaConverters._ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.FileNames._ +import org.apache.hadoop.fs.{FileStatus, Path} + /** * Manages the creation, computation, and access of Snapshot's for Delta tables. Responsibilities * include: @@ -99,7 +101,9 @@ private[internal] trait SnapshotManagement { self: DeltaLogImpl => // List from the starting checkpoint. If a checkpoint doesn't exist, this will still return // deltaVersion=0. - val newFiles = store.listFrom(checkpointPrefix(logPath, startCheckpoint.getOrElse(0L))) + val newFiles = store + .listFrom(checkpointPrefix(logPath, startCheckpoint.getOrElse(0L)), hadoopConf) + .asScala // Pick up all checkpoint and delta files .filter { file => isCheckpointFile(file.getPath) || isDeltaFile(file.getPath) } // filter out files that aren't atomically visible. Checkpoint files of 0 size are invalid diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala index 1847a76be9f..24b09238638 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala @@ -46,7 +46,6 @@ package io.delta.standalone.internal.data import java.sql.{Date, Timestamp} -import java.util import java.util.TimeZone import scala.collection.JavaConverters._ @@ -113,9 +112,11 @@ private[internal] case class RowParquetRecordImpl( override def getRecord(fieldName: String): RowParquetRecordJ = getAs[RowParquetRecordJ](fieldName) - override def getList[T](fieldName: String): util.List[T] = getAs[util.List[T]](fieldName) + override def getList[T](fieldName: String): java.util.List[T] = + getAs[java.util.List[T]](fieldName) - override def getMap[K, V](fieldName: String): util.Map[K, V] = getAs[util.Map[K, V]](fieldName) + override def getMap[K, V](fieldName: String): java.util.Map[K, V] = + getAs[java.util.Map[K, V]](fieldName) /////////////////////////////////////////////////////////////////////////// // Decoding Helper Methods diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala index c99baae37ee..efd204ef070 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala @@ -16,18 +16,18 @@ package io.delta.standalone.internal.storage -import java.io.{BufferedReader, FileNotFoundException, InputStreamReader, IOException} +import java.io.IOException import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.FileAlreadyExistsException -import java.util.{EnumSet, UUID} +import java.util.EnumSet import scala.collection.JavaConverters._ import scala.util.control.NonFatal import io.delta.standalone.internal.exception.DeltaErrors -import org.apache.commons.io.IOUtils + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileContext, FileStatus, Options, Path, RawLocalFileSystem} +import org.apache.hadoop.fs.{FileContext, Options, Path, RawLocalFileSystem} import org.apache.hadoop.fs.CreateFlag.CREATE import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} @@ -39,49 +39,41 @@ import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} * * 2. Consistent file listing: HDFS file listing is consistent. */ -private[internal] class HDFSLogStore(hadoopConf: Configuration) extends LogStore { +private[internal] class HDFSLogStore(override val initHadoopConf: Configuration) + extends HadoopFileSystemLogStore(initHadoopConf) { val noAbstractFileSystemExceptionMessage = "No AbstractFileSystem" - override def read(path: Path): Seq[String] = { - val fs = path.getFileSystem(hadoopConf) - val stream = fs.open(path) - try { - val reader = new BufferedReader(new InputStreamReader(stream, UTF_8)) - IOUtils.readLines(reader).asScala.map(_.trim) - } finally { - stream.close() - } - } - - override def listFrom(path: Path): Iterator[FileStatus] = { - val fs = path.getFileSystem(hadoopConf) - if (!fs.exists(path.getParent)) { - throw new FileNotFoundException(s"No such file or directory: ${path.getParent}") - } - val files = fs.listStatus(path.getParent) - files.filter(_.getPath.getName >= path.getName).sortBy(_.getPath.getName).iterator - } - - override def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit = { + override def write( + path: Path, + actions: java.util.Iterator[String], + overwrite: java.lang.Boolean, + hadoopConf: Configuration): Unit = { val isLocalFs = path.getFileSystem(hadoopConf).isInstanceOf[RawLocalFileSystem] if (isLocalFs) { // We need to add `synchronized` for RawLocalFileSystem as its rename will not throw an // exception when the target file exists. Hence we must make sure `exists + rename` in // `writeInternal` for RawLocalFileSystem is atomic in our tests. synchronized { - writeInternal(path, actions, overwrite) + writeInternal(path, actions.asScala, overwrite, hadoopConf) } } else { // rename is atomic and also will fail when the target file exists. Not need to add the extra // `synchronized`. - writeInternal(path, actions, overwrite) + writeInternal(path, actions.asScala, overwrite, hadoopConf) } } - private def writeInternal(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { + override def isPartialWriteVisible( + path: Path, hadoopConf: Configuration): java.lang.Boolean = false + + private def writeInternal( + path: Path, + actions: Iterator[String], + overwrite: Boolean, + hadoopConf: Configuration): Unit = { val fc: FileContext = try { - getFileContext(path) + getFileContext(path, hadoopConf) } catch { case e: IOException if e.getMessage.contains(noAbstractFileSystemExceptionMessage) => val newException = DeltaErrors.incorrectLogStoreImplementationException(e) @@ -122,11 +114,7 @@ private[internal] class HDFSLogStore(hadoopConf: Configuration) extends LogStore } } - private def createTempPath(path: Path): Path = { - new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}.tmp") - } - - private def getFileContext(path: Path): FileContext = { + private def getFileContext(path: Path, hadoopConf: Configuration): FileContext = { FileContext.getFileContext(path.toUri, hadoopConf) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala new file mode 100644 index 00000000000..413333b3d22 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala @@ -0,0 +1,121 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.storage + +import java.io.{BufferedReader, FileNotFoundException, InputStreamReader} +import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.FileAlreadyExistsException +import java.util.UUID + +import scala.collection.JavaConverters._ + +import io.delta.standalone.data.CloseableIterator +import io.delta.standalone.storage.LogStore + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} + +/** + * Default implementation of [[LogStore]] for Hadoop [[FileSystem]] implementations. + */ +abstract class HadoopFileSystemLogStore(override val initHadoopConf: Configuration) + extends LogStore(initHadoopConf) { + + override def read(path: Path, hadoopConf: Configuration): CloseableIterator[String] = { + val fs = path.getFileSystem(hadoopConf) + val stream = fs.open(path) + val reader = new BufferedReader(new InputStreamReader(stream, UTF_8)) + new LineCloseableIterator(reader) + } + + override def listFrom(path: Path, hadoopConf: Configuration): java.util.Iterator[FileStatus] = { + val fs = path.getFileSystem(hadoopConf) + if (!fs.exists(path.getParent)) { + throw new FileNotFoundException(s"No such file or directory: ${path.getParent}") + } + val files = fs.listStatus(path.getParent) + files.filter(_.getPath.getName >= path.getName).sortBy(_.getPath.getName).iterator.asJava + } + + override def resolvePathOnPhysicalStorage(path: Path, hadoopConf: Configuration): Path = { + path.getFileSystem(hadoopConf).makeQualified(path) + } + + /** + * An internal write implementation that uses FileSystem.rename(). + * + * This implementation should only be used for the underlying file systems that support atomic + * renames, e.g., Azure is OK but HDFS is not. + */ + protected def writeWithRename( + path: Path, + actions: Iterator[String], + overwrite: Boolean, + hadoopConf: Configuration): Unit = { + val fs = path.getFileSystem(hadoopConf) + + if (!fs.exists(path.getParent)) { + throw new FileNotFoundException(s"No such file or directory: ${path.getParent}") + } + if (overwrite) { + val stream = fs.create(path, true) + try { + actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) + } finally { + stream.close() + } + } else { + if (fs.exists(path)) { + throw new FileAlreadyExistsException(path.toString) + } + val tempPath = createTempPath(path) + var streamClosed = false // This flag is to avoid double close + var renameDone = false // This flag is to save the delete operation in most of cases. + val stream = fs.create(tempPath) + try { + actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) + stream.close() + streamClosed = true + try { + if (fs.rename(tempPath, path)) { + renameDone = true + } else { + if (fs.exists(path)) { + throw new FileAlreadyExistsException(path.toString) + } else { + throw new IllegalStateException(s"Cannot rename $tempPath to $path") + } + } + } catch { + case _: org.apache.hadoop.fs.FileAlreadyExistsException => + throw new FileAlreadyExistsException(path.toString) + } + } finally { + if (!streamClosed) { + stream.close() + } + if (!renameDone) { + fs.delete(tempPath, false) + } + } + } + } + + protected def createTempPath(path: Path): Path = { + new Path(path.getParent, s".${path.getName}.${UUID.randomUUID}.tmp") + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala new file mode 100644 index 00000000000..f691c46c111 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala @@ -0,0 +1,74 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.storage + +import java.io.Reader + +import io.delta.standalone.data.CloseableIterator +import org.apache.commons.io.IOUtils + +/** + * Turn a `Reader` to `ClosableIterator` which can be read on demand. Each element is + * a trimmed line. + */ +class LineCloseableIterator(_reader: Reader) extends CloseableIterator[String] { + private val reader = IOUtils.toBufferedReader(_reader) + // Whether `nextValue` is valid. If it's invalid, we should try to read the next line. + private var gotNext = false + // The next value to return when `next` is called. This is valid only if `getNext` is true. + private var nextValue: String = _ + // Whether the reader is closed. + private var closed = false + // Whether we have consumed all data in the reader. + private var finished = false + + override def hasNext(): Boolean = { + if (!finished) { + // Check whether we have closed the reader before reading. Even if `nextValue` is valid, we + // still don't return `nextValue` after a reader is closed. Otherwise, it would be confusing. + if (closed) { + throw new IllegalStateException("Iterator is closed") + } + if (!gotNext) { + val nextLine = reader.readLine() + if (nextLine == null) { + finished = true + close() + } else { + nextValue = nextLine.trim + } + gotNext = true + } + } + !finished + } + + override def next(): String = { + if (!hasNext) { + throw new NoSuchElementException("End of stream") + } + gotNext = false + nextValue + } + + override def close(): Unit = { + if (!closed) { + closed = true + reader.close() + } + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala deleted file mode 100644 index 887ec6ab7c5..00000000000 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStore.scala +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Copyright (2020) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.standalone.internal.storage - -import io.delta.standalone.data.{CloseableIterator => ClosebleIteratorJ} -import io.delta.standalone.storage.{LogStore => LogStoreJ} -import io.delta.standalone.internal.sources.StandaloneHadoopConf - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} - -/** - * General interface for all critical file system operations required to read and write the - * [[io.delta.standalone.DeltaLog]]. The correctness of the [[io.delta.standalone.DeltaLog]] is - * predicated on the atomicity and durability guarantees of the implementation of this interface. - * Specifically, - * - * 1. Atomic visibility of files: Any file written through this store must - * be made visible atomically. In other words, this should not generate partial files. - * - * 2. Mutual exclusion: Only one writer must be able to create (or rename) a file at the final - * destination. - * - * 3. Consistent listing: Once a file has been written in a directory, all future listings for - * that directory must return that file. - */ -private[internal] trait LogStore { // TODO: rename and refactor - - /** Read the given `path` */ - def read(path: String): Seq[String] = read(new Path(path)) - - /** Read the given `path` */ - def read(path: Path): Seq[String] - - /** - * List the paths in the same directory that are lexicographically greater or equal to - * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. - */ - def listFrom(path: String): Iterator[FileStatus] = listFrom(new Path(path)) - - /** - * List the paths in the same directory that are lexicographically greater or equal to - * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name. - */ - def listFrom(path: Path): Iterator[FileStatus] - - /** - * Write the given `actions` to the given `path` without overwriting any existing file. - * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file - * already exists. Furthermore, implementation must ensure that the entire file is made - * visible atomically, that is, it should not generate partial files. - */ - final def write(path: String, actions: Iterator[String]): Unit = write(new Path(path), actions) - - /** - * Write the given `actions` to the given `path` with or without overwrite as indicated. - * Implementation must throw [[java.nio.file.FileAlreadyExistsException]] exception if the file - * already exists and overwrite = false. Furthermore, implementation must ensure that the - * entire file is made visible atomically, that is, it should not generate partial files. - */ - def write(path: Path, actions: Iterator[String], overwrite: Boolean = false): Unit - - /** Resolve the fully qualified path for the given `path`. */ - def resolvePathOnPhysicalStorage(path: Path): Path = { - throw new UnsupportedOperationException() - } - - /** - * Whether a partial write is visible when writing to `path`. - * - * As this depends on the underlying file system implementations, we require the input of `path` - * here in order to identify the underlying file system, even though in most cases a log store - * only deals with one file system. - * - * The default value is only provided here for legacy reasons, which will be removed. - * Any LogStore implementation should override this instead of relying on the default. - */ - def isPartialWriteVisible(path: Path): Boolean = true -} - -private[internal] object LogStore extends LogStoreProvider - -private[internal] trait LogStoreProvider { - - val defaultLogStoreClassName = classOf[HDFSLogStore].getName - - def createLogStore(hadoopConf: Configuration): LogStore = { - val logStoreClassName = - hadoopConf.get(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, defaultLogStoreClassName) - - // scalastyle:off classforname - val logStoreClass = - Class.forName(logStoreClassName, true, Thread.currentThread().getContextClassLoader) - // scalastyle:on classforname - - if (classOf[LogStoreJ].isAssignableFrom(logStoreClass)) { - val logStoreImpl = logStoreClass.getConstructor(classOf[Configuration]) - .newInstance(hadoopConf).asInstanceOf[LogStoreJ] - new LogStoreAdaptor(logStoreImpl, hadoopConf) - } else { - logStoreClass.getConstructor(classOf[Configuration]).newInstance(hadoopConf) - .asInstanceOf[LogStore] - } - } -} - -/** - * An adapter from external Java instances of [[LogStoreJ]] to internal Scala instances of - * [[LogStore]]. - */ -private[internal] class LogStoreAdaptor( - logStoreImpl: LogStoreJ, - hadoopConf: Configuration) extends LogStore { - - override def read(path: Path): Seq[String] = { - var iter: ClosebleIteratorJ[String] = null - try { - iter = logStoreImpl.read(path, hadoopConf) - val contents = iter.asScala.toArray - contents - } finally { - if (iter != null) { - iter.close() - } - } - } - - override def write(path: Path, actions: Iterator[String], overwrite: Boolean): Unit = { - logStoreImpl.write(path, actions.asJava, overwrite, hadoopConf) - } - - override def listFrom(path: Path): Iterator[FileStatus] = { - logStoreImpl.listFrom(path, hadoopConf).asScala - } - - override def resolvePathOnPhysicalStorage(path: Path): Path = { - logStoreImpl.resolvePathOnPhysicalStorage(path, hadoopConf) - } - - override def isPartialWriteVisible(path: Path): Boolean = { - logStoreImpl.isPartialWriteVisible(path, hadoopConf) - } -} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala new file mode 100644 index 00000000000..e2658d65d66 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala @@ -0,0 +1,50 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.storage + +import io.delta.standalone.storage.LogStore +import io.delta.standalone.internal.sources.StandaloneHadoopConf + +import org.apache.hadoop.conf.Configuration + +private[internal] object LogStoreProvider extends LogStoreProvider + +private[internal] trait LogStoreProvider { + + val defaultLogStoreClassName: String = classOf[HDFSLogStore].getName + + def createLogStore(hadoopConf: Configuration): LogStore = { + val logStoreClassName = + hadoopConf.get(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, defaultLogStoreClassName) + + // scalastyle:off classforname + val logStoreClass = + Class.forName(logStoreClassName, true, Thread.currentThread().getContextClassLoader) + // scalastyle:on classforname + + if (classOf[LogStore].isAssignableFrom(logStoreClass)) { + logStoreClass + .getConstructor(classOf[Configuration]) + .newInstance(hadoopConf) + .asInstanceOf[LogStore] + } else { + // TODO proper error? + throw new IllegalArgumentException(s"Can't instantiate a LogStore with classname " + + s"$logStoreClassName.") + } + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/Implicits.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/Implicits.scala new file mode 100644 index 00000000000..8eee9de5d4c --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/Implicits.scala @@ -0,0 +1,42 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import scala.reflect.ClassTag + +import io.delta.standalone.data.CloseableIterator + +private[internal] object Implicits { + implicit class CloseableIteratorOps[T: ClassTag](private val iter: CloseableIterator[T]) { + import scala.collection.JavaConverters._ + + /** + * Convert the [[CloseableIterator]] (Java) to an in-memory [[Array]] (Scala). + * + * [[scala.collection.Iterator.toArray]] is used over [[scala.collection.Iterable.toSeq]] + * because `toSeq` is lazy, meaning `iter.close()` would be called before the Seq was actually + * generated. + */ + def toArray: Array[T] = { + try { + iter.asScala.toArray + } finally { + iter.close() + } + } + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala index 9683d5fe73c..86c40563dc7 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala @@ -18,12 +18,13 @@ package io.delta.standalone.internal import java.io.File +import scala.collection.JavaConverters._ + import io.delta.standalone.data.{CloseableIterator => CloseableIteratorJ} -import io.delta.standalone.storage.{LogStore => LogStoreJ} +import io.delta.standalone.storage.LogStore import io.delta.standalone.internal.sources.StandaloneHadoopConf -import io.delta.standalone.internal.storage.{HDFSLogStore, LogStore, LogStoreAdaptor, LogStoreProvider} +import io.delta.standalone.internal.storage.{HDFSLogStore, LogStoreProvider} import io.delta.standalone.internal.util.GoldenTableUtils._ -import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -54,17 +55,20 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { } test("instantiation") { - val expectedClassName = logStoreClassName.getOrElse(LogStore.defaultLogStoreClassName) + val expectedClassName = logStoreClassName.getOrElse(LogStoreProvider.defaultLogStoreClassName) assert(createLogStore(hadoopConf).getClass.getName == expectedClassName) } test("read") { withGoldenTable("log-store-read") { tablePath => - val logStore = createLogStore(hadoopConf) + import io.delta.standalone.internal.util.Implicits._ + val logStore = createLogStore(hadoopConf) val deltas = Seq(0, 1).map(i => new File(tablePath, i.toString)).map(_.getCanonicalPath) - assert(logStore.read(deltas.head) == Seq("zero", "none")) - assert(logStore.read(deltas(1)) == Seq("one")) + + assert(logStore.read(new Path(deltas.head), hadoopConf).toArray sameElements + Array("zero", "none")) + assert(logStore.read(new Path(deltas(1)), hadoopConf).toArray sameElements Array("one")) } } @@ -77,18 +81,20 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { .map(_.toURI) .map(new Path(_)) - assert(logStore.listFrom(deltas.head).map(_.getPath.getName) + assert(logStore.listFrom(deltas.head, hadoopConf).asScala.map(_.getPath.getName) .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) - assert(logStore.listFrom(deltas(1)).map(_.getPath.getName) + assert(logStore.listFrom(deltas(1), hadoopConf).asScala.map(_.getPath.getName) .filterNot(_ == "_delta_log").toArray === Seq(1, 2, 3).map(_.toString)) - assert(logStore.listFrom(deltas(2)).map(_.getPath.getName) + assert(logStore.listFrom(deltas(2), hadoopConf).asScala.map(_.getPath.getName) .filterNot(_ == "_delta_log").toArray === Seq(2, 3).map(_.toString)) - assert(logStore.listFrom(deltas(3)).map(_.getPath.getName) + assert(logStore.listFrom(deltas(3), hadoopConf).asScala.map(_.getPath.getName) .filterNot(_ == "_delta_log").toArray === Seq(3).map(_.toString)) - assert(logStore.listFrom(deltas(4)).map(_.getPath.getName) + assert(logStore.listFrom(deltas(4), hadoopConf).asScala.map(_.getPath.getName) .filterNot(_ == "_delta_log").toArray === Nil) } } + + // TODO: log store write tests } /** @@ -110,8 +116,7 @@ class DefaultLogStoreSuite extends LogStoreSuiteBase { * Test having the user provide their own LogStore. */ class UserDefinedLogStoreSuite extends LogStoreSuiteBase { - // The actual type of LogStore created will be LogStoreAdaptor. - override def logStoreClassName: Option[String] = Some(classOf[LogStoreAdaptor].getName) + override def logStoreClassName: Option[String] = Some(classOf[UserDefinedLogStore].getName) override def hadoopConf: Configuration = { val conf = new Configuration() @@ -121,17 +126,17 @@ class UserDefinedLogStoreSuite extends LogStoreSuiteBase { } /** - * Sample user-defined log store implementing [[LogStoreJ]] + * Sample user-defined log store implementing [[LogStore]]. */ class UserDefinedLogStore(override val initHadoopConf: Configuration) - extends LogStoreJ(initHadoopConf) { + extends LogStore(initHadoopConf) { private val mockImpl = new HDFSLogStore(initHadoopConf) override def read(path: Path, hadoopConf: Configuration): CloseableIteratorJ[String] = { - val iter = mockImpl.read(path).iterator + val iter = mockImpl.read(path, hadoopConf) new CloseableIteratorJ[String] { - override def close(): Unit = {} + override def close(): Unit = iter.close() override def hasNext: Boolean = iter.hasNext override def next(): String = iter.next } @@ -142,18 +147,18 @@ class UserDefinedLogStore(override val initHadoopConf: Configuration) actions: java.util.Iterator[String], overwrite: java.lang.Boolean, hadoopConf: Configuration): Unit = { - mockImpl.write(path, actions.asScala, overwrite) + mockImpl.write(path, actions, overwrite, hadoopConf) } override def listFrom(path: Path, hadoopConf: Configuration): java.util.Iterator[FileStatus] = { - mockImpl.listFrom(path).asJava + mockImpl.listFrom(path, hadoopConf) } override def resolvePathOnPhysicalStorage(path: Path, hadoopConf: Configuration): Path = { - mockImpl.resolvePathOnPhysicalStorage(path) + mockImpl.resolvePathOnPhysicalStorage(path, hadoopConf) } override def isPartialWriteVisible(path: Path, hadoopConf: Configuration): java.lang.Boolean = { - mockImpl.isPartialWriteVisible(path) + mockImpl.isPartialWriteVisible(path, hadoopConf) } } From 9692cabf1eb8615981555f8af805428077debd13 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 8 Oct 2021 12:20:15 -0700 Subject: [PATCH 095/291] [DSW] [19] Delta OSS Opt Txn unit tests added to DSW (v1) (#168) * rename OptTxnSuite to OptTxnLegacySuite; add new OptTxnSuiteBase and OptTxnSuite; add OptTxn.metadata API; add no-arg Protocol constructor * add disjoint txn; disjoint delete/read tests * add disjoint add/read test * Update OptimisticTransactionSuite.scala --- .../standalone/OptimisticTransaction.java | 6 + .../io/delta/standalone/actions/Metadata.java | 1 + .../io/delta/standalone/actions/Protocol.java | 22 + .../internal/OptimisticTransactionImpl.scala | 8 +- .../internal/util/ConversionUtils.scala | 2 +- .../OptimisticTransactionLegacySuite.scala | 738 +++++++++++++++ .../internal/OptimisticTransactionSuite.scala | 881 ++++-------------- .../OptimisticTransactionSuiteBase.scala | 151 +++ 8 files changed, 1088 insertions(+), 721 deletions(-) create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 4f8e983a43b..6a312242324 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -67,4 +67,10 @@ public interface OptimisticTransaction { * @return the latest version that has committed for the idempotent transaction with given `id`. */ long txnVersion(String id); + + /** + * @return the metadata for this transaction. The metadata refers to the metadata of the snapshot + * at the transaction's read version unless updated during the transaction. + */ + Metadata metadata(); } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index 1bd31ff84e2..b5718aafeda 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.util.*; diff --git a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java index 93b79b9a18d..cf6997c9b46 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java @@ -1,3 +1,19 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.delta.standalone.actions; import java.util.Objects; @@ -6,6 +22,12 @@ public class Protocol implements Action { private final int minReaderVersion; private final int minWriterVersion; + public Protocol() { + // TODO: have these statically and publicly defined + this.minReaderVersion = 1; + this.minWriterVersion = 2; + } + public Protocol(int minReaderVersion, int minWriterVersion) { this.minReaderVersion = minReaderVersion; this.minWriterVersion = minWriterVersion; diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 8b82a521e62..3c119feb7f0 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -74,7 +74,7 @@ private[internal] class OptimisticTransactionImpl( * Returns the metadata for this transaction. The metadata refers to the metadata of the snapshot * at the transaction's read version unless updated during the transaction. */ - def metadata: Metadata = newMetadata.getOrElse(snapshot.metadataScala) + def metadataScala: Metadata = newMetadata.getOrElse(snapshot.metadataScala) /** The version that this transaction is reading from. */ private def readVersion: Long = snapshot.version @@ -83,6 +83,8 @@ private[internal] class OptimisticTransactionImpl( // Public Java API Methods /////////////////////////////////////////////////////////////////////////// + override def metadata(): MetadataJ = ConversionUtils.convertMetadata(metadataScala) + override def commit[T <: ActionJ]( actionsJ: java.lang.Iterable[T], op: Operation, @@ -233,7 +235,7 @@ private[internal] class OptimisticTransactionImpl( s"Currently only Protocol readerVersion 1 and writerVersion 2 is supported.") } - val partitionColumns = metadata.partitionColumns.toSet + val partitionColumns = metadataScala.partitionColumns.toSet finalActions.foreach { case a: AddFile if partitionColumns != a.partitionValues.keySet => throw DeltaErrors.addFilePartitioningMismatchException( @@ -345,7 +347,7 @@ private[internal] class OptimisticTransactionImpl( readFiles = readFiles.toSet, readWholeTable = readTheWholeTable, readAppIds = readTxn.toSet, - metadata = metadata, + metadata = metadataScala, actions = actions, deltaLog = deltaLog) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index 6c68126fa9f..e1821fd9837 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -278,7 +278,7 @@ private[internal] object ConversionUtils { external.getName, external.getDescription, convertFormatJ(external.getFormat), - external.getSchema.toJson, + if (external.getSchema == null) null else external.getSchema.toJson, external.getPartitionColumns.asScala, external.getConfiguration.asScala.toMap, external.getCreatedTime // implicit check this! diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala new file mode 100644 index 00000000000..057d2b98c5f --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -0,0 +1,738 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import io.delta.standalone.{DeltaLog, Operation} +import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDeleteDeleteException, ConcurrentDeleteReadException, ConcurrentTransactionException, MetadataChangedException, ProtocolChangedException} +import io.delta.standalone.expressions.{EqualTo, Expression, Literal} +import io.delta.standalone.internal.actions._ +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.util.{ConversionUtils, SchemaUtils} +import io.delta.standalone.types._ +import io.delta.standalone.internal.util.TestUtils._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class OptimisticTransactionLegacySuite extends FunSuite { + // scalastyle:on funsuite + + val engineInfo = "test-engine-info" + val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + + val A_P1 = "part=1/a" + val B_P1 = "part=1/b" + val C_P1 = "part=1/c" + val C_P2 = "part=2/c" + val D_P2 = "part=2/d" + val E_P3 = "part=3/e" + val F_P3 = "part=3/f" + val G_P4 = "part=4/g" + + private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) + + def withLog( + actions: Seq[Action], + partitionCols: Seq[String] = "part" :: Nil)( + test: DeltaLog => Unit): Unit = { + val schemaFields = partitionCols.map { p => new StructField(p, new StringType()) }.toArray + val schema = new StructType(schemaFields) + val metadata = Metadata(partitionColumns = partitionCols, schemaString = schema.toJson) + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + log.startTransaction().commit(actions, manualUpdate, engineInfo) + + test(log) + } + } + + /** + * @tparam T expected exception type + */ + def testMetadata[T <: Throwable : ClassTag]( + metadata: Metadata, + expectedExceptionMessageSubStr: String): Unit = { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val e1 = intercept[T] { + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + } + assert(e1.getMessage.contains(expectedExceptionMessageSubStr)) + + val e2 = intercept[T] { + log.startTransaction().updateMetadata(ConversionUtils.convertMetadata(metadata)) + } + assert(e2.getMessage.contains(expectedExceptionMessageSubStr)) + } + } + + test("basic commit") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + // [...] is what is automatically added by OptimisticTransaction + // 0 -> metadata [CommitInfo, Protocol] + // 1 -> addA_P1, addB_P1 [CommitInfo] + // 2 -> removeA_P1 [CommitInfo] + val versionLogs = log.getChanges(0, true).asScala.toList + + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[MetadataJ]) == 1) + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) + assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[ProtocolJ]) == 1) + + assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[AddFileJ]) == 2) + assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) + + assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[RemoveFileJ]) == 1) + assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) + } + } + + test("basic checkpoint") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + (1 to 15).foreach { i => + val meta = if (i == 1) Metadata() :: Nil else Nil + val txn = log.startTransaction() + val file = AddFile(i.toString, Map.empty, 1, 1, dataChange = true) :: Nil + val delete: Seq[Action] = if (i > 1) { + RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil + } else { + Nil + } + txn.commit(meta ++ delete ++ file, manualUpdate, engineInfo) + } + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + assert(log2.snapshot.getVersion == 14) + assert(log2.snapshot.getAllFiles.size == 1) + } + } + + /////////////////////////////////////////////////////////////////////////// + // prepareCommit() tests + /////////////////////////////////////////////////////////////////////////// + + test("committing twice in the same transaction should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) + val e = intercept[AssertionError] { + txn.commit(Iterable().asJava, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("Transaction already committed.")) + } + } + + test("user cannot commit their own CommitInfo") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + val e = intercept[AssertionError] { + log.startTransaction().commit(CommitInfo.empty() :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("Cannot commit a custom CommitInfo in a transaction.")) + } + } + + test("commits shouldn't have more than one Metadata") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val e = intercept[AssertionError] { + txn.commit(Metadata() :: Metadata() :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) + } + } + + // DeltaLog::ensureLogDirectoryExists + test("transaction should throw if it cannot read log directory during first commit") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + dir.setReadOnly() + + val txn = log.startTransaction() + val e = intercept[java.io.IOException] { + txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) + } + + val logPath = new Path(log.getPath, "_delta_log") + assert(e.getMessage == s"Cannot create ${logPath.toString}") + } + } + + test("initial commit without metadata should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val e = intercept[IllegalStateException] { + txn.commit(Iterable().asJava, manualUpdate, engineInfo) + } + assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) + } + } + + test("AddFile with different partition schema compared to metadata should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + + // Note that Metadata() has no partition schema specified and addA_P1 does + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + val e = intercept[IllegalStateException] { + log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + + "table's partitioning schema")) + } + } + + test("Can't create table with invalid protocol version") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + + Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => + val e = intercept[AssertionError] { + log.startTransaction().commit(Metadata() :: protocol :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("Invalid Protocol")) + } + } + } + + test("can't change protocol to invalid version") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Protocol() :: Nil, manualUpdate, engineInfo) + + Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => + val e = intercept[AssertionError] { + log.startTransaction().commit(protocol :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("Invalid Protocol")) + } + } + } + + test("Removing from an append-only table") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val metadata = Metadata(configuration = Map("appendOnly" -> "true")) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + + val removeWithDataChange = addA_P1.remove.copy(dataChange = true) + val e = intercept[UnsupportedOperationException] { + log.startTransaction().commit(removeWithDataChange :: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains("This table is configured to only allow appends")) + + val removeWithoutDataChange = addA_P1.remove.copy(dataChange = false) + log.startTransaction().commit(removeWithoutDataChange :: Nil, manualUpdate, engineInfo) + } + } + + /////////////////////////////////////////////////////////////////////////// + // updateMetadata() tests + /////////////////////////////////////////////////////////////////////////// + + test("can't update metadata more than once in a transaction") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + val e = intercept[AssertionError] { + txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) + } + + assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) + } + } + + test("Protocol Action should be automatically added to transaction for new table") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + assert(log.getChanges(0, true).asScala.next().getActions.contains(new ProtocolJ(1, 2))) + } + } + + test("updateMetadata removes Protocol properties from metadata config") { + // Note: These Protocol properties are not currently exposed to the user. However, they + // might be in the future, and nothing is stopping the user now from seeing these + // properties in Delta OSS and adding them to the config map here. + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn = log.startTransaction() + val metadata = Metadata(configuration = Map( + Protocol.MIN_READER_VERSION_PROP -> "1", + Protocol.MIN_WRITER_VERSION_PROP -> "2" + )) + txn.updateMetadata(ConversionUtils.convertMetadata(metadata)) + txn.commit(Iterable().asJava, manualUpdate, engineInfo) + + val writtenConfig = log.update().getMetadata.getConfiguration + assert(!writtenConfig.containsKey(Protocol.MIN_READER_VERSION_PROP)) + assert(!writtenConfig.containsKey(Protocol.MIN_WRITER_VERSION_PROP)) + } + } + + test("commit new metadata with Protocol properties should fail") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + val newMetadata = Metadata(configuration = Map( + Protocol.MIN_READER_VERSION_PROP -> "1", + Protocol.MIN_WRITER_VERSION_PROP -> "2" + )) + + val e = intercept[AssertionError] { + log.startTransaction().commit(newMetadata:: Nil, manualUpdate, engineInfo) + } + assert(e.getMessage.contains(s"Should not have the protocol version " + + s"(${Protocol.MIN_READER_VERSION_PROP}) as part of table properties")) + } + } + + test("unenforceable not null constraints") { + val validSchema = new StructType(Array( + new StructField( + "col1", + new MapType(new ArrayType(new StringType(), true), new IntegerType(), true), + true + ), + new StructField( + "col2", + new MapType(new IntegerType(), new ArrayType(new StringType(), true), true), + true + ), + new StructField( + "col3", + new ArrayType(new MapType(new StringType(), new IntegerType(), true), + true) + ) + )) + + SchemaUtils.checkUnenforceableNotNullConstraints(validSchema) // should not throw + + // case 1: not null within array + val inValidSchema1 = new StructType( + Array( + new StructField( + "arr", + new ArrayType( + new StructType( + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + false // arr (ArrayType) containsNull + ) + ) + ) + ) + + val e1 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema1) + }.getMessage + + assert(e1.contains("The element type of the field arr contains a NOT NULL constraint.")) + + // case 2: null within map key + val inValidSchema2 = new StructType( + Array( + new StructField( + "m", + new MapType( + new StructType( // m.key + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + new IntegerType(), // m.value + false // m (MapType) valueContainsNull + ) + ) + ) + ) + + val e2 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema2) + }.getMessage + + assert(e2.contains("The key type of the field m contains a NOT NULL constraint.")) + + // case 3: null within map key + val inValidSchema3 = new StructType( + Array( + new StructField( + "m", + new MapType( + new IntegerType(), // m.key + new StructType( // m.value + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + false // m (MapType) valueContainsNull + ) + ) + ) + ) + + val e3 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema3) + }.getMessage + + assert(e3.contains("The value type of the field m contains a NOT NULL constraint.")) + + // case 4: not null within nested array + val inValidSchema4 = new StructType( + Array( + new StructField( + "s", + new StructType( + Array( + new StructField("n", new IntegerType, false), + new StructField( + "arr", + new ArrayType( + new StructType( + Array( + new StructField("name", new StringType(), true), + new StructField("mailbox", new StringType(), false) + ) + ), + true // arr (ArrayType) containsNull + ), + false // arr (StructField) nullable + ) + ) + ), + true // s (StructField) nullable + ) + ) + ) + + val e4 = intercept[RuntimeException] { + SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema4) + }.getMessage + + assert(e4.contains("The element type of the field s.arr contains a NOT NULL constraint.")) + } + + test("updateMetadata withGlobalConfigDefaults") { + // TODO: use DeltaConfigs... + withTempDir { dir => + // note that the default for logRetentionDuration is 2592000000 + val hadoopConf = new Configuration() + hadoopConf.set("logRetentionDuration", "1000") + val metadata = Metadata(configuration = Map("logRetentionDuration" -> "2000")) + + val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath) + log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) + + assert(log.deltaRetentionMillis == 2000) + } + } + + /////////////////////////////////////////////////////////////////////////// + // verifyNewMetadata() tests + /////////////////////////////////////////////////////////////////////////// + + test("can't have duplicate column names") { + // TODO: just call myStruct.getJson() + // scalastyle:off + val schemaStr = """{"type":"struct","fields":[{"name":"col1","type":"integer","nullable":true,"metadata":{}},{"name":"col1","type":"integer","nullable":true,"metadata":{}}]}""" + // scalastyle:on + testMetadata[RuntimeException](Metadata(schemaString = schemaStr), "Found duplicate column(s)") + } + + test("column names (both data and partition) must be acceptable by parquet") { + // TODO: just call myStruct.getJson() + // test DATA columns + // scalastyle:off + val schemaStr1 = """{"type":"struct","fields":[{"name":"bad;column,name","type":"integer","nullable":true,"metadata":{}}]}""" + // scalastyle:on + testMetadata[RuntimeException](Metadata(schemaString = schemaStr1), + """Attribute name "bad;column,name" contains invalid character(s)""") + + // test PARTITION columns + testMetadata[RuntimeException](Metadata(partitionColumns = "bad;column,name" :: Nil), + "Found partition columns having invalid character(s)") + } + + /////////////////////////////////////////////////////////////////////////// + // commit() tests + /////////////////////////////////////////////////////////////////////////// + + test("CommitInfo operation and engineInfo is persisted to the delta log") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val commitInfo = log2.getCommitInfoAt(0) + assert(commitInfo.getEngineInfo.isPresent) + assert(commitInfo.getEngineInfo.get() == engineInfo) + assert(commitInfo.getOperation == manualUpdate.getName.toString) + + // TODO: test commitInfo.operationParameters + } + } + + test("CommitInfo isBlindAppend is correctly set") { + withTempDir { dir => + def verifyIsBlindAppend(version: Int, expected: Boolean): Unit = { + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + assert(log.getCommitInfoAt(version).getIsBlindAppend.get() == expected) + } + + val add = AddFile("test", Map.empty, 1, 1, dataChange = false) + + val log0 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log0.startTransaction().commit(Metadata() :: add :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(0, expected = true) + + val log1 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log1.startTransaction().commit(add.remove :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(1, expected = false) + + val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + val txn2 = log2.startTransaction() + txn2.markFilesAsRead(Literal.True) + txn2.commit(add :: Nil, manualUpdate, engineInfo) + verifyIsBlindAppend(2, expected = false) + } + } + + /////////////////////////////////////////////////////////////////////////// + // checkForConflicts() tests + /////////////////////////////////////////////////////////////////////////// + + private def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { + fileActions.map { + case a: AddFile => a.copy(dataChange = false) + case r: RemoveFile => r.copy(dataChange = false) + case cdc: AddCDCFile => cdc // change files are always dataChange = false + } + } + + ////////////////////////////////// + // protocolChangedException tests + ////////////////////////////////// + + test("concurrent protocol update should fail") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + val tx2 = log.startTransaction() + tx2.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) + + assertThrows[ProtocolChangedException] { + tx1.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // metadataChangedException tests + ////////////////////////////////// + + test("concurrent metadata update should fail") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + val tx2 = log.startTransaction() + tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "foo"))) + tx2.commit(Iterable().asJava, manualUpdate, engineInfo) + + assertThrows[MetadataChangedException] { + tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "bar"))) + tx1.commit(Iterable().asJava, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentAppend tests + ////////////////////////////////// + + test("block concurrent commit when read partition was appended to by concurrent write") { + withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // TX1 reads only P1 + val tx1Read = tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) + assert(tx1Read.getFiles.asScala.toSeq.map(_.getPath) == A_P1 :: Nil) + + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True) + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentAppendException] { + // P1 was modified + tx1.commit(addC_P2 :: addE_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("block concurrent commit on full table scan") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.markFilesAsRead(Literal.True) + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) + + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True) + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentAppendException] { + tx1.commit(addE_P3 :: addF_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("no data change: allow data rearrange when new files concurrently added") { + // This tests the case when isolationLevel == SnapshotIsolation + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.markFilesAsRead(Literal.True) + + val tx2 = log.startTransaction() + tx1.markFilesAsRead(Literal.True) + tx2.commit(addE_P3 :: Nil, manualUpdate, engineInfo) + + // tx1 rearranges files (dataChange = false) + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + manualUpdate, engineInfo) + + assert(log.update().getAllFiles.asScala.map(_.getPath) == C_P1 :: E_P3 :: Nil) + } + } + + ////////////////////////////////// + // concurrentDeleteRead tests + ////////////////////////////////// + + test("block concurrent commit on read & delete conflicting partitions") { + // txn.readFiles will be non-empty, so this covers the first ConcurrentDeleteReadException + // case in checkForDeletedFilesAgainstCurrentTxnReadFiles + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val schema = log.update().getMetadata.getSchema + val tx1 = log.startTransaction() + // read P1 + tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.markFilesAsRead(Literal.True) + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteReadException] { + // P1 read by TX1 was removed by TX2 + tx1.commit(addE_P3 :: Nil, manualUpdate, engineInfo) + } + } + } + + test("readWholeTable should block concurrent delete") { + // txn.readFiles will be empty, so this covers the second ConcurrentDeleteReadException + // case in checkForDeletedFilesAgainstCurrentTxnReadFiles + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.readWholeTable() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteReadException] { + // tx1 reads the whole table but tx2 removes files before tx1 commits + tx1.commit(addB_P1 :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentDeleteDelete tests + ////////////////////////////////// + + test("block commit with concurrent removes on same file") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentDeleteDeleteException] { + // tx1 tries to remove the same file + tx1.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) + } + } + } + + ////////////////////////////////// + // concurrentTransaction tests + ////////////////////////////////// + + test("block concurrent set-txns with the same app id") { + withLog(Nil) { log => + val tx1 = log.startTransaction() + tx1.txnVersion("t1") + + val winningTxn = log.startTransaction() + winningTxn.txnVersion("t1") + winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, manualUpdate, engineInfo) + + intercept[ConcurrentTransactionException] { + tx1.commit(Iterable().asJava, manualUpdate, engineInfo) + } + } + } + + // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) + + // TODO: readWholeTable tests + + // TODO: test Checkpoint > partialWriteVisible (==> useRename) + + // TODO: test Checkpoint > !partialWriteVisible (==> !useRename) + + // TODO: test Checkpoint > correctly checkpoints all action types +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index bbad3b3f178..2142f63acf7 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -16,723 +16,170 @@ package io.delta.standalone.internal -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import io.delta.standalone.{DeltaLog, Operation} -import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} -import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDeleteDeleteException, ConcurrentDeleteReadException, ConcurrentTransactionException, MetadataChangedException, ProtocolChangedException} -import io.delta.standalone.expressions.{EqualTo, Expression, Literal} -import io.delta.standalone.internal.actions._ -import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.util.{ConversionUtils, SchemaUtils} -import io.delta.standalone.types._ -import io.delta.standalone.internal.util.TestUtils._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -// scalastyle:off funsuite -import org.scalatest.FunSuite - -class OptimisticTransactionSuite extends FunSuite { - // scalastyle:on funsuite - - val engineInfo = "test-engine-info" - val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) - - val A_P1 = "part=1/a" - val B_P1 = "part=1/b" - val C_P1 = "part=1/c" - val C_P2 = "part=2/c" - val D_P2 = "part=2/d" - val E_P3 = "part=3/e" - val F_P3 = "part=3/f" - val G_P4 = "part=4/g" - - private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) - private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) - private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) - private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) - private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) - - def withLog( - actions: Seq[Action], - partitionCols: Seq[String] = "part" :: Nil)( - test: DeltaLog => Unit): Unit = { - val schemaFields = partitionCols.map { p => new StructField(p, new StringType()) }.toArray - val schema = new StructType(schemaFields) - val metadata = Metadata(partitionColumns = partitionCols, schemaString = schema.toJson) - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) - log.startTransaction().commit(actions, manualUpdate, engineInfo) - - test(log) - } - } - - /** - * @tparam T expected exception type - */ - def testMetadata[T <: Throwable : ClassTag]( - metadata: Metadata, - expectedExceptionMessageSubStr: String): Unit = { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val e1 = intercept[T] { - log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) - } - assert(e1.getMessage.contains(expectedExceptionMessageSubStr)) - - val e2 = intercept[T] { - log.startTransaction().updateMetadata(ConversionUtils.convertMetadata(metadata)) - } - assert(e2.getMessage.contains(expectedExceptionMessageSubStr)) - } - } - - test("basic commit") { - withLog(addA_P1 :: addB_P1 :: Nil) { log => - log.startTransaction().commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) - - // [...] is what is automatically added by OptimisticTransaction - // 0 -> metadata [CommitInfo, Protocol] - // 1 -> addA_P1, addB_P1 [CommitInfo] - // 2 -> removeA_P1 [CommitInfo] - val versionLogs = log.getChanges(0, true).asScala.toList - - assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[MetadataJ]) == 1) - assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) - assert(versionLogs(0).getActions.asScala.count(_.isInstanceOf[ProtocolJ]) == 1) - - assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[AddFileJ]) == 2) - assert(versionLogs(1).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) - - assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[RemoveFileJ]) == 1) - assert(versionLogs(2).getActions.asScala.count(_.isInstanceOf[CommitInfoJ]) == 1) - } - } - - test("basic checkpoint") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - (1 to 15).foreach { i => - val meta = if (i == 1) Metadata() :: Nil else Nil - val txn = log.startTransaction() - val file = AddFile(i.toString, Map.empty, 1, 1, dataChange = true) :: Nil - val delete: Seq[Action] = if (i > 1) { - RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil - } else { - Nil - } - txn.commit(meta ++ delete ++ file, manualUpdate, engineInfo) - } - - val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - assert(log2.snapshot.getVersion == 14) - assert(log2.snapshot.getAllFiles.size == 1) - } - } - - /////////////////////////////////////////////////////////////////////////// - // prepareCommit() tests - /////////////////////////////////////////////////////////////////////////// - - test("committing twice in the same transaction should fail") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) - val e = intercept[AssertionError] { - txn.commit(Iterable().asJava, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("Transaction already committed.")) - } - } - - test("user cannot commit their own CommitInfo") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) - val e = intercept[AssertionError] { - log.startTransaction().commit(CommitInfo.empty() :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("Cannot commit a custom CommitInfo in a transaction.")) - } - } - - test("commits shouldn't have more than one Metadata") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - val e = intercept[AssertionError] { - txn.commit(Metadata() :: Metadata() :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) - } - } - - // DeltaLog::ensureLogDirectoryExists - test("transaction should throw if it cannot read log directory during first commit") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - dir.setReadOnly() - - val txn = log.startTransaction() - val e = intercept[java.io.IOException] { - txn.commit(Metadata() :: Nil, manualUpdate, engineInfo) - } - - val logPath = new Path(log.getPath, "_delta_log") - assert(e.getMessage == s"Cannot create ${logPath.toString}") - } - } - - test("initial commit without metadata should fail") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - val e = intercept[IllegalStateException] { - txn.commit(Iterable().asJava, manualUpdate, engineInfo) - } - assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) - } - } - - test("AddFile with different partition schema compared to metadata should fail") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - - // Note that Metadata() has no partition schema specified and addA_P1 does - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) - val e = intercept[IllegalStateException] { - log.startTransaction().commit(addA_P1 :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("The AddFile contains partitioning schema different from the " + - "table's partitioning schema")) - } - } - - test("Can't create table with invalid protocol version") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - - Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => - val e = intercept[AssertionError] { - log.startTransaction().commit(Metadata() :: protocol :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("Invalid Protocol")) - } - } - } - - test("can't change protocol to invalid version") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Protocol() :: Nil, manualUpdate, engineInfo) - - Seq(Protocol(1, 3), Protocol(1, 1), Protocol(2, 2)).foreach { protocol => - val e = intercept[AssertionError] { - log.startTransaction().commit(protocol :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("Invalid Protocol")) - } - } - } - - test("Removing from an append-only table") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val metadata = Metadata(configuration = Map("appendOnly" -> "true")) - log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) - - val removeWithDataChange = addA_P1.remove.copy(dataChange = true) - val e = intercept[UnsupportedOperationException] { - log.startTransaction().commit(removeWithDataChange :: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains("This table is configured to only allow appends")) - - val removeWithoutDataChange = addA_P1.remove.copy(dataChange = false) - log.startTransaction().commit(removeWithoutDataChange :: Nil, manualUpdate, engineInfo) - } - } - - /////////////////////////////////////////////////////////////////////////// - // updateMetadata() tests - /////////////////////////////////////////////////////////////////////////// - - test("can't update metadata more than once in a transaction") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) - val e = intercept[AssertionError] { - txn.updateMetadata(ConversionUtils.convertMetadata(Metadata())) - } - - assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) - } - } - - test("Protocol Action should be automatically added to transaction for new table") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) - assert(log.getChanges(0, true).asScala.next().getActions.contains(new ProtocolJ(1, 2))) - } - } - - test("updateMetadata removes Protocol properties from metadata config") { - // Note: These Protocol properties are not currently exposed to the user. However, they - // might be in the future, and nothing is stopping the user now from seeing these - // properties in Delta OSS and adding them to the config map here. - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn = log.startTransaction() - val metadata = Metadata(configuration = Map( - Protocol.MIN_READER_VERSION_PROP -> "1", - Protocol.MIN_WRITER_VERSION_PROP -> "2" - )) - txn.updateMetadata(ConversionUtils.convertMetadata(metadata)) - txn.commit(Iterable().asJava, manualUpdate, engineInfo) - - val writtenConfig = log.update().getMetadata.getConfiguration - assert(!writtenConfig.containsKey(Protocol.MIN_READER_VERSION_PROP)) - assert(!writtenConfig.containsKey(Protocol.MIN_WRITER_VERSION_PROP)) - } - } - - test("commit new metadata with Protocol properties should fail") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) - val newMetadata = Metadata(configuration = Map( - Protocol.MIN_READER_VERSION_PROP -> "1", - Protocol.MIN_WRITER_VERSION_PROP -> "2" - )) - - val e = intercept[AssertionError] { - log.startTransaction().commit(newMetadata:: Nil, manualUpdate, engineInfo) - } - assert(e.getMessage.contains(s"Should not have the protocol version " + - s"(${Protocol.MIN_READER_VERSION_PROP}) as part of table properties")) - } - } - - test("unenforceable not null constraints") { - val validSchema = new StructType(Array( - new StructField( - "col1", - new MapType(new ArrayType(new StringType(), true), new IntegerType(), true), - true - ), - new StructField( - "col2", - new MapType(new IntegerType(), new ArrayType(new StringType(), true), true), - true - ), - new StructField( - "col3", - new ArrayType(new MapType(new StringType(), new IntegerType(), true), - true) - ) - )) - - SchemaUtils.checkUnenforceableNotNullConstraints(validSchema) // should not throw - - // case 1: not null within array - val inValidSchema1 = new StructType( - Array( - new StructField( - "arr", - new ArrayType( - new StructType( - Array( - new StructField("name", new StringType(), true), - new StructField("mailbox", new StringType(), false) - ) - ), - false // arr (ArrayType) containsNull - ) - ) - ) - ) - - val e1 = intercept[RuntimeException] { - SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema1) - }.getMessage - - assert(e1.contains("The element type of the field arr contains a NOT NULL constraint.")) - - // case 2: null within map key - val inValidSchema2 = new StructType( - Array( - new StructField( - "m", - new MapType( - new StructType( // m.key - Array( - new StructField("name", new StringType(), true), - new StructField("mailbox", new StringType(), false) - ) - ), - new IntegerType(), // m.value - false // m (MapType) valueContainsNull - ) - ) - ) - ) - - val e2 = intercept[RuntimeException] { - SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema2) - }.getMessage - - assert(e2.contains("The key type of the field m contains a NOT NULL constraint.")) - - // case 3: null within map key - val inValidSchema3 = new StructType( - Array( - new StructField( - "m", - new MapType( - new IntegerType(), // m.key - new StructType( // m.value - Array( - new StructField("name", new StringType(), true), - new StructField("mailbox", new StringType(), false) - ) - ), - false // m (MapType) valueContainsNull - ) - ) - ) - ) - - val e3 = intercept[RuntimeException] { - SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema3) - }.getMessage - - assert(e3.contains("The value type of the field m contains a NOT NULL constraint.")) - - // case 4: not null within nested array - val inValidSchema4 = new StructType( - Array( - new StructField( - "s", - new StructType( - Array( - new StructField("n", new IntegerType, false), - new StructField( - "arr", - new ArrayType( - new StructType( - Array( - new StructField("name", new StringType(), true), - new StructField("mailbox", new StringType(), false) - ) - ), - true // arr (ArrayType) containsNull - ), - false // arr (StructField) nullable - ) - ) - ), - true // s (StructField) nullable - ) - ) - ) +import java.util.Collections - val e4 = intercept[RuntimeException] { - SchemaUtils.checkUnenforceableNotNullConstraints(inValidSchema4) - }.getMessage - - assert(e4.contains("The element type of the field s.arr contains a NOT NULL constraint.")) - } - - test("updateMetadata withGlobalConfigDefaults") { - // TODO: use DeltaConfigs... - withTempDir { dir => - // note that the default for logRetentionDuration is 2592000000 - val hadoopConf = new Configuration() - hadoopConf.set("logRetentionDuration", "1000") - val metadata = Metadata(configuration = Map("logRetentionDuration" -> "2000")) - - val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath) - log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) - - assert(log.deltaRetentionMillis == 2000) - } - } - - /////////////////////////////////////////////////////////////////////////// - // verifyNewMetadata() tests - /////////////////////////////////////////////////////////////////////////// - - test("can't have duplicate column names") { - // TODO: just call myStruct.getJson() - // scalastyle:off - val schemaStr = """{"type":"struct","fields":[{"name":"col1","type":"integer","nullable":true,"metadata":{}},{"name":"col1","type":"integer","nullable":true,"metadata":{}}]}""" - // scalastyle:on - testMetadata[RuntimeException](Metadata(schemaString = schemaStr), "Found duplicate column(s)") - } - - test("column names (both data and partition) must be acceptable by parquet") { - // TODO: just call myStruct.getJson() - // test DATA columns - // scalastyle:off - val schemaStr1 = """{"type":"struct","fields":[{"name":"bad;column,name","type":"integer","nullable":true,"metadata":{}}]}""" - // scalastyle:on - testMetadata[RuntimeException](Metadata(schemaString = schemaStr1), - """Attribute name "bad;column,name" contains invalid character(s)""") - - // test PARTITION columns - testMetadata[RuntimeException](Metadata(partitionColumns = "bad;column,name" :: Nil), - "Found partition columns having invalid character(s)") - } - - /////////////////////////////////////////////////////////////////////////// - // commit() tests - /////////////////////////////////////////////////////////////////////////// - - test("CommitInfo operation and engineInfo is persisted to the delta log") { - withTempDir { dir => - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) - - val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val commitInfo = log2.getCommitInfoAt(0) - assert(commitInfo.getEngineInfo.isPresent) - assert(commitInfo.getEngineInfo.get() == engineInfo) - assert(commitInfo.getOperation == manualUpdate.getName.toString) - - // TODO: test commitInfo.operationParameters - } - } - - test("CommitInfo isBlindAppend is correctly set") { - withTempDir { dir => - def verifyIsBlindAppend(version: Int, expected: Boolean): Unit = { - val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - assert(log.getCommitInfoAt(version).getIsBlindAppend.get() == expected) - } - - val add = AddFile("test", Map.empty, 1, 1, dataChange = false) - - val log0 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log0.startTransaction().commit(Metadata() :: add :: Nil, manualUpdate, engineInfo) - verifyIsBlindAppend(0, expected = true) - - val log1 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log1.startTransaction().commit(add.remove :: Nil, manualUpdate, engineInfo) - verifyIsBlindAppend(1, expected = false) - - val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val txn2 = log2.startTransaction() - txn2.markFilesAsRead(Literal.True) - txn2.commit(add :: Nil, manualUpdate, engineInfo) - verifyIsBlindAppend(2, expected = false) - } - } - - /////////////////////////////////////////////////////////////////////////// - // checkForConflicts() tests - /////////////////////////////////////////////////////////////////////////// - - private def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { - fileActions.map { - case a: AddFile => a.copy(dataChange = false) - case r: RemoveFile => r.copy(dataChange = false) - case cdc: AddCDCFile => cdc // change files are always dataChange = false - } - } - - ////////////////////////////////// - // protocolChangedException tests - ////////////////////////////////// - - test("concurrent protocol update should fail") { - withLog(Nil) { log => - val tx1 = log.startTransaction() - val tx2 = log.startTransaction() - tx2.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) - - assertThrows[ProtocolChangedException] { - tx1.commit(Protocol(1, 2) :: Nil, manualUpdate, engineInfo) - } - } - } - - ////////////////////////////////// - // metadataChangedException tests - ////////////////////////////////// - - test("concurrent metadata update should fail") { - withLog(Nil) { log => - val tx1 = log.startTransaction() - val tx2 = log.startTransaction() - tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "foo"))) - tx2.commit(Iterable().asJava, manualUpdate, engineInfo) - - assertThrows[MetadataChangedException] { - tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "bar"))) - tx1.commit(Iterable().asJava, manualUpdate, engineInfo) - } - } - } - - ////////////////////////////////// - // concurrentAppend tests - ////////////////////////////////// - - test("block concurrent commit when read partition was appended to by concurrent write") { - withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => - val schema = log.update().getMetadata.getSchema - val tx1 = log.startTransaction() - // TX1 reads only P1 - val tx1Read = tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) - assert(tx1Read.getFiles.asScala.toSeq.map(_.getPath) == A_P1 :: Nil) - - val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True) - // TX2 modifies only P1 - tx2.commit(addB_P1 :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentAppendException] { - // P1 was modified - tx1.commit(addC_P2 :: addE_P3 :: Nil, manualUpdate, engineInfo) - } - } - } - - test("block concurrent commit on full table scan") { - withLog(addA_P1 :: addD_P2 :: Nil) { log => - val schema = log.update().getMetadata.getSchema - val tx1 = log.startTransaction() - // TX1 full table scan - tx1.markFilesAsRead(Literal.True) - tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) - - val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True) - tx2.commit(addC_P2 :: addD_P2.remove :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentAppendException] { - tx1.commit(addE_P3 :: addF_P3 :: Nil, manualUpdate, engineInfo) - } - } - } - - test("no data change: allow data rearrange when new files concurrently added") { - // This tests the case when isolationLevel == SnapshotIsolation - withLog(addA_P1 :: addB_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.markFilesAsRead(Literal.True) - - val tx2 = log.startTransaction() - tx1.markFilesAsRead(Literal.True) - tx2.commit(addE_P3 :: Nil, manualUpdate, engineInfo) - - // tx1 rearranges files (dataChange = false) - tx1.commit( - setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), - manualUpdate, engineInfo) - - assert(log.update().getAllFiles.asScala.map(_.getPath) == C_P1 :: E_P3 :: Nil) - } - } - - ////////////////////////////////// - // concurrentDeleteRead tests - ////////////////////////////////// - - test("block concurrent commit on read & delete conflicting partitions") { - // txn.readFiles will be non-empty, so this covers the first ConcurrentDeleteReadException - // case in checkForDeletedFilesAgainstCurrentTxnReadFiles - withLog(addA_P1 :: addB_P1 :: Nil) { log => - val schema = log.update().getMetadata.getSchema - val tx1 = log.startTransaction() - // read P1 - tx1.markFilesAsRead(new EqualTo(schema.column("part"), Literal.of("1"))) - - // tx2 commits before tx1 - val tx2 = log.startTransaction() - tx2.markFilesAsRead(Literal.True) - tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentDeleteReadException] { - // P1 read by TX1 was removed by TX2 - tx1.commit(addE_P3 :: Nil, manualUpdate, engineInfo) - } - } - } - - test("readWholeTable should block concurrent delete") { - // txn.readFiles will be empty, so this covers the second ConcurrentDeleteReadException - // case in checkForDeletedFilesAgainstCurrentTxnReadFiles - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.readWholeTable() - - // tx2 removes file - val tx2 = log.startTransaction() - tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentDeleteReadException] { - // tx1 reads the whole table but tx2 removes files before tx1 commits - tx1.commit(addB_P1 :: Nil, manualUpdate, engineInfo) - } - } - } - - ////////////////////////////////// - // concurrentDeleteDelete tests - ////////////////////////////////// - - test("block commit with concurrent removes on same file") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - - // tx2 removes file - val tx2 = log.startTransaction() - tx2.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentDeleteDeleteException] { - // tx1 tries to remove the same file - tx1.commit(addA_P1.remove :: Nil, manualUpdate, engineInfo) - } - } - } - - ////////////////////////////////// - // concurrentTransaction tests - ////////////////////////////////// - - test("block concurrent set-txns with the same app id") { - withLog(Nil) { log => - val tx1 = log.startTransaction() - tx1.txnVersion("t1") - - val winningTxn = log.startTransaction() - winningTxn.txnVersion("t1") - winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, manualUpdate, engineInfo) - - intercept[ConcurrentTransactionException] { - tx1.commit(Iterable().asJava, manualUpdate, engineInfo) - } - } - } - - // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) - - // TODO: readWholeTable tests - - // TODO: test Checkpoint > partialWriteVisible (==> useRename) - - // TODO: test Checkpoint > !partialWriteVisible (==> !useRename) +import scala.collection.JavaConverters._ - // TODO: test Checkpoint > correctly checkpoints all action types +import io.delta.standalone.actions.{Protocol, AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} +import io.delta.standalone.expressions.{EqualTo, Literal} +import io.delta.standalone.types.{IntegerType, StructField, StructType} + +class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { + private val addA = new AddFileJ("a", Collections.emptyMap(), 1, 1, true, null, null) + private val addB = new AddFileJ("b", Collections.emptyMap(), 1, 1, true, null, null) + + private val addA_partX1 = new AddFileJ("a", Map("x" -> "1").asJava, 1, 1, true, null, null) + private val addA_partX2 = new AddFileJ("a", Map("x" -> "2").asJava, 1, 1, true, null, null) + private val schema = new StructType(Array(new StructField("x", new IntegerType()))) + + /* ************************** * + * Allowed concurrent actions * + * ************************** */ + + check( + "append / append", + conflicts = false, + reads = Seq(t => t.metadata()), + concurrentWrites = Seq(addA), + actions = Seq(addB)) + + check( + "disjoint txns", + conflicts = false, + reads = Seq( + t => t.txnVersion("t1") + ), + concurrentWrites = Seq( + new SetTransactionJ("t2", 0, java.util.Optional.of(1234L))), + actions = Nil) + + check( + "disjoint delete / read", + conflicts = false, + setup = Seq( + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build(), + addA_partX2 + ), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) + ), + concurrentWrites = Seq( + RemoveFileJ.builder("a").deletionTimestamp(4L).build() + ), + actions = Seq() + ) + + check( + "disjoint add / read", + conflicts = false, + setup = Seq( + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() + ), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) + ), + concurrentWrites = Seq( + addA_partX2 + ), + actions = Seq() + ) + + check( + "add / read + no write", // no write = no real conflicting change even though data was added + conflicts = false, // so this should not conflict + setup = Seq( + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() + ), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) + ), + concurrentWrites = Seq(addA_partX1), + actions = Seq()) + + /* ***************************** * + * Disallowed concurrent actions * + * ***************************** */ + + check( + "delete / delete", + conflicts = true, + reads = Nil, + concurrentWrites = Seq( + RemoveFileJ.builder("a").deletionTimestamp(4L).build()), + actions = Seq( + RemoveFileJ.builder("a").deletionTimestamp(4L).build())) + + check( + "add / read + write", + conflicts = true, + setup = Seq( + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() + ), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) + ), + concurrentWrites = Seq(addA_partX1), + actions = Seq(addA_partX1), + // commit info should show operation as "Manual Update", because that's the operation used by + // the harness + errorMessageHint = Some("[x=1]" :: "Manual Update" :: Nil)) + + check( + "delete / read", + conflicts = true, + setup = Seq( + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build(), + addA_partX1 + ), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) + ), + concurrentWrites = Seq( + RemoveFileJ.builder("a").deletionTimestamp(4L).build() + ), + actions = Seq(), + errorMessageHint = Some("a in partition [x=1]" :: "Manual Update" :: Nil)) + + check( + "schema change", + conflicts = true, + reads = Seq( + t => t.metadata + ), + concurrentWrites = Seq(MetadataJ.builder().build()), + actions = Nil) + + check( + "conflicting txns", + conflicts = true, + reads = Seq( + t => t.txnVersion("t1") + ), + concurrentWrites = Seq( + new SetTransactionJ("t1", 0, java.util.Optional.of(1234L)) + ), + actions = Nil) + + check( + "upgrade / upgrade", + conflicts = true, + reads = Seq( + t => t.metadata + ), + concurrentWrites = Seq( + new Protocol()), + actions = Seq( + new Protocol())) + + // TODO: taint whole table + + // TODO: taint whole table + concurrent remove + + // TODO: initial commit without metadata should fail + + // TODO: initial commit with multiple metadata actions should fail + + // TODO: AddFile with different partition schema compared to metadata should fail + + // TODO: isolation level shouldn't be null ? } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala new file mode 100644 index 00000000000..4ca8238feec --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala @@ -0,0 +1,151 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.ConcurrentModificationException + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{DeltaLog, Operation, OptimisticTransaction} +import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ, Protocol => ProtocolJ} +import io.delta.standalone.internal.util.TestUtils._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.FunSuite + +trait OptimisticTransactionSuiteBase extends FunSuite { + + val op = new Operation(Operation.Name.MANUAL_UPDATE) + val engineInfo = "test-engine-info" + + /** + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - setup (including setting table isolation level + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param name test name + * @param conflicts should test transaction is expected to conflict or not + * @param setup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentWrites writes made by concurrent transactions after the test txn reads + * @param actions actions to be committed by the test transaction + * @param errorMessageHint What to expect in the error message + * @param exceptionClass A substring to expect in the exception class name + */ + protected def check( + name: String, + conflicts: Boolean, + setup: Seq[ActionJ] = Seq(MetadataJ.builder().build(), new ProtocolJ()), + reads: Seq[OptimisticTransaction => Unit], + concurrentWrites: Seq[ActionJ], + actions: Seq[ActionJ], + errorMessageHint: Option[Seq[String]] = None, + exceptionClass: Option[String] = None): Unit = { + + val concurrentTxn: OptimisticTransaction => Unit = + (opt: OptimisticTransaction) => + opt.commit(concurrentWrites.asJava, op, engineInfo) + + def initialSetup(log: DeltaLog): Unit = { + // Setup the log + setup.foreach { action => + log.startTransaction().commit(Seq(action).asJava, op, engineInfo) + } + } + check( + name, + conflicts, + initialSetup _, + reads, + Seq(concurrentTxn), + actions.asJava, + errorMessageHint, + exceptionClass + ) + } + + /** + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - sets up the initial delta log state using `initialSetup` (set schema, partitioning, etc.) + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param name test name + * @param conflicts should test transaction is expected to conflict or not + * @param initialSetup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentTxns concurrent txns that may write data after the test txn reads + * @param actions actions to be committed by the test transaction + * @param errorMessageHint What to expect in the error message + * @param exceptionClass A substring to expect in the exception class name + */ + protected def check( + name: String, + conflicts: Boolean, + initialSetup: DeltaLog => Unit, + reads: Seq[OptimisticTransaction => Unit], + concurrentTxns: Seq[OptimisticTransaction => Unit], + actions: java.util.List[ActionJ], + errorMessageHint: Option[Seq[String]], + exceptionClass: Option[String]): Unit = { + + val conflict = if (conflicts) "should conflict" else "should not conflict" + test(s"$name - $conflict") { + withTempDir { tempDir => + val log = + DeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath + "/_delta_log")) + + // Setup the log + initialSetup(log) + + // Perform reads + val txn = log.startTransaction() + reads.foreach(_ (txn)) + + // Execute concurrent txn while current transaction is active + concurrentTxns.foreach(txn => txn(log.startTransaction())) + + // Try commit and check expected conflict behavior + if (conflicts) { + val e = intercept[ConcurrentModificationException] { + txn.commit(actions, op, engineInfo) + } + errorMessageHint.foreach { expectedParts => + assert(expectedParts.forall(part => e.getMessage.contains(part))) + } + if (exceptionClass.nonEmpty) { + assert(e.getClass.getName.contains(exceptionClass.get)) + } + } else { + txn.commit(actions, op, engineInfo) + } + } + } + } +} From 8dc5830d4f606a25659871b768d89170bd1b2ac9 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 8 Oct 2021 14:43:22 -0700 Subject: [PATCH 096/291] [DSW] [17] LogStore write tests (#163) --- .../standalone/internal/Checkpoints.scala | 3 +- .../standalone/internal/LogStoreSuite.scala | 53 +++++++++++++++---- 2 files changed, 46 insertions(+), 10 deletions(-) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index cbf8be9dec6..79f32da755d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -22,13 +22,14 @@ import java.util.UUID import scala.collection.JavaConverters._ import scala.util.control.NonFatal +import io.delta.standalone.data.CloseableIterator import io.delta.standalone.internal.actions.SingleAction import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.internal.util.FileNames._ + import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.metadata.CompressionCodecName import com.github.mjakubowski84.parquet4s.ParquetWriter -import io.delta.standalone.data.CloseableIterator /** * Records information about a checkpoint. diff --git a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala index 86c40563dc7..e5ac79d0d71 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala @@ -25,6 +25,7 @@ import io.delta.standalone.storage.LogStore import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.storage.{HDFSLogStore, LogStoreProvider} import io.delta.standalone.internal.util.GoldenTableUtils._ +import io.delta.standalone.internal.util.TestUtils._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -54,21 +55,57 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { conf } - test("instantiation") { + test("instantiation through HadoopConf") { val expectedClassName = logStoreClassName.getOrElse(LogStoreProvider.defaultLogStoreClassName) assert(createLogStore(hadoopConf).getClass.getName == expectedClassName) } - test("read") { - withGoldenTable("log-store-read") { tablePath => + test("read / write") { + def assertNoLeakedCrcFiles(dir: File): Unit = { + // crc file should not be leaked when origin file doesn't exist. + // The implementation of Hadoop filesystem may filter out checksum file, so + // listing files from local filesystem. + val fileNames = dir.listFiles().toSeq.filter(p => p.isFile).map(p => p.getName) + val crcFiles = fileNames.filter(n => n.startsWith(".") && n.endsWith(".crc")) + val originFileNamesForExistingCrcFiles = crcFiles.map { name => + // remove first "." and last ".crc" + name.substring(1, name.length - 4) + } + + // Check all origin files exist for all crc files. + assert(originFileNamesForExistingCrcFiles.toSet.subsetOf(fileNames.toSet), + s"Some of origin files for crc files don't exist - crc files: $crcFiles / " + + s"expected origin files: $originFileNamesForExistingCrcFiles / actual files: $fileNames") + } + + withTempDir { dir => import io.delta.standalone.internal.util.Implicits._ - val logStore = createLogStore(hadoopConf) - val deltas = Seq(0, 1).map(i => new File(tablePath, i.toString)).map(_.getCanonicalPath) + val store = createLogStore(hadoopConf) - assert(logStore.read(new Path(deltas.head), hadoopConf).toArray sameElements + val deltas = Seq(0, 1).map(i => new File(dir, i.toString)).map(_.getCanonicalPath) + store.write(new Path(deltas.head), Iterator("zero", "none").asJava, false, hadoopConf) + store.write(new Path(deltas(1)), Iterator("one").asJava, false, hadoopConf) + + assert(store.read(new Path(deltas.head), hadoopConf).toArray sameElements Array("zero", "none")) - assert(logStore.read(new Path(deltas(1)), hadoopConf).toArray sameElements Array("one")) + assert(store.read(new Path(deltas(1)), hadoopConf).toArray sameElements Array("one")) + + assertNoLeakedCrcFiles(dir) + } + } + + test("detects conflict") { + withTempDir { dir => + val store = createLogStore(hadoopConf) + + val deltas = Seq(0, 1).map(i => new File(dir, i.toString)).map(_.getCanonicalPath) + store.write(new Path(deltas.head), Iterator("zero").asJava, false, hadoopConf) + store.write(new Path(deltas(1)), Iterator("one").asJava, false, hadoopConf) + + intercept[java.nio.file.FileAlreadyExistsException] { + store.write(new Path(deltas(1)), Iterator("uno").asJava, false, hadoopConf) + } } } @@ -93,8 +130,6 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { .filterNot(_ == "_delta_log").toArray === Nil) } } - - // TODO: log store write tests } /** From a248425596e271cfc4b9a93447ff9350cb894a5f Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 8 Oct 2021 14:43:43 -0700 Subject: [PATCH 097/291] [DSW] [20] Opt Txn Delta OSS tests v2 (#169) --- .../standalone/internal/isolationLevels.scala | 2 +- .../OptimisticTransactionLegacySuite.scala | 2 +- .../internal/OptimisticTransactionSuite.scala | 119 +++++++++++------- 3 files changed, 79 insertions(+), 44 deletions(-) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala b/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala index d35149a482e..bf29961ab59 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/isolationLevels.scala @@ -17,7 +17,7 @@ package io.delta.standalone.internal sealed trait IsolationLevel { - // TODO override def toString: String = this.getClass.getSimpleName.stripSuffix("$") + override def toString: String = this.getClass.getSimpleName.stripSuffix("$") } case object Serializable extends IsolationLevel diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 057d2b98c5f..fb8aa05ce29 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -265,7 +265,7 @@ class OptimisticTransactionLegacySuite extends FunSuite { // updateMetadata() tests /////////////////////////////////////////////////////////////////////////// - test("can't update metadata more than once in a transaction") { + test("initial commit with multiple metadata actions should fail") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val txn = log.startTransaction() diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 2142f63acf7..b57b85f41d7 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -20,17 +20,29 @@ import java.util.Collections import scala.collection.JavaConverters._ -import io.delta.standalone.actions.{Protocol, AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} +import io.delta.standalone.actions.{CommitInfo, Protocol, AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} import io.delta.standalone.expressions.{EqualTo, Literal} import io.delta.standalone.types.{IntegerType, StructField, StructType} +import io.delta.standalone.internal.util.TestUtils._ +import io.delta.standalone.DeltaLog + +import org.apache.hadoop.conf.Configuration class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { private val addA = new AddFileJ("a", Collections.emptyMap(), 1, 1, true, null, null) private val addB = new AddFileJ("b", Collections.emptyMap(), 1, 1, true, null, null) + private val removeA = RemoveFileJ.builder("a").deletionTimestamp(4L).build() private val addA_partX1 = new AddFileJ("a", Map("x" -> "1").asJava, 1, 1, true, null, null) private val addA_partX2 = new AddFileJ("a", Map("x" -> "2").asJava, 1, 1, true, null, null) + private val addB_partX1 = new AddFileJ("b", Map("x" -> "1").asJava, 1, 1, true, null, null) + private val addB_partX3 = new AddFileJ("b", Map("x" -> "3").asJava, 1, 1, true, null, null) + private val addC_partX4 = new AddFileJ("c", Map("x" -> "4").asJava, 1, 1, true, null, null) + private val schema = new StructType(Array(new StructField("x", new IntegerType()))) + private val metadata_colX = MetadataJ.builder().schema(schema).build() + private val metadata_partX = + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() /* ************************** * * Allowed concurrent actions * @@ -56,40 +68,29 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { check( "disjoint delete / read", conflicts = false, - setup = Seq( - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build(), - addA_partX2 - ), + setup = Seq(metadata_partX, addA_partX2), reads = Seq( t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) ), - concurrentWrites = Seq( - RemoveFileJ.builder("a").deletionTimestamp(4L).build() - ), + concurrentWrites = Seq(removeA), actions = Seq() ) check( "disjoint add / read", conflicts = false, - setup = Seq( - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() - ), + setup = Seq(metadata_partX), reads = Seq( t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) ), - concurrentWrites = Seq( - addA_partX2 - ), + concurrentWrites = Seq(addA_partX2), actions = Seq() ) check( "add / read + no write", // no write = no real conflicting change even though data was added conflicts = false, // so this should not conflict - setup = Seq( - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() - ), + setup = Seq(metadata_partX), reads = Seq( t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) ), @@ -104,22 +105,19 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { "delete / delete", conflicts = true, reads = Nil, - concurrentWrites = Seq( - RemoveFileJ.builder("a").deletionTimestamp(4L).build()), - actions = Seq( - RemoveFileJ.builder("a").deletionTimestamp(4L).build())) + concurrentWrites = Seq(removeA), + actions = Seq(RemoveFileJ.builder("a").deletionTimestamp(5L).build()) + ) check( "add / read + write", conflicts = true, - setup = Seq( - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() - ), + setup = Seq(metadata_partX), reads = Seq( t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) ), concurrentWrites = Seq(addA_partX1), - actions = Seq(addA_partX1), + actions = Seq(addB_partX1), // commit info should show operation as "Manual Update", because that's the operation used by // the harness errorMessageHint = Some("[x=1]" :: "Manual Update" :: Nil)) @@ -127,16 +125,11 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { check( "delete / read", conflicts = true, - setup = Seq( - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build(), - addA_partX1 - ), + setup = Seq(metadata_partX, addA_partX1), reads = Seq( t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) ), - concurrentWrites = Seq( - RemoveFileJ.builder("a").deletionTimestamp(4L).build() - ), + concurrentWrites = Seq(removeA), actions = Seq(), errorMessageHint = Some("a in partition [x=1]" :: "Manual Update" :: Nil)) @@ -166,20 +159,62 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { reads = Seq( t => t.metadata ), - concurrentWrites = Seq( - new Protocol()), - actions = Seq( - new Protocol())) + concurrentWrites = Seq(new Protocol()), + actions = Seq(new Protocol())) + + check( + "taint whole table", + conflicts = true, + setup = Seq(metadata_partX, addA_partX2), + reads = Seq( + t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))), + // `readWholeTable` should disallow any concurrent change, even if the change + // is disjoint with the earlier filter + t => t.readWholeTable() + ), + concurrentWrites = Seq(addB_partX3), + actions = Seq(addC_partX4) + ) + + check( + "taint whole table + concurrent remove", + conflicts = true, + setup = Seq(metadata_colX, addA), + reads = Seq( + // `readWholeTable` should disallow any concurrent `RemoveFile`s. + t => t.readWholeTable() + ), + concurrentWrites = Seq(removeA), + actions = Seq(addB)) + + // initial commit without metadata should fail + // --> see OptimisticTransactionLegacySuite - // TODO: taint whole table + // initial commit with multiple metadata actions should fail + // --> see OptimisticTransactionLegacySuite - // TODO: taint whole table + concurrent remove + // AddFile with different partition schema compared to metadata should fail + // --> see OptimisticTransactionLegacySuite - // TODO: initial commit without metadata should fail + test("isolation level shouldn't be null") { + withTempDir { dir => + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + log.startTransaction().commit((MetadataJ.builder().build() :: Nil).asJava, op, engineInfo) + log.startTransaction().commit((addA :: Nil).asJava, op, engineInfo) - // TODO: initial commit with multiple metadata actions should fail + val versionLogs = log.getChanges(0, true).asScala.toList - // TODO: AddFile with different partition schema compared to metadata should fail + def getIsolationLevel(version: Int): String = { + versionLogs(version) + .getActions + .asScala + .collectFirst { case c: CommitInfo => c } + .map(_.getIsolationLevel.orElseGet(null)) + .get + } - // TODO: isolation level shouldn't be null ? + assert(getIsolationLevel(0) == "SnapshotIsolation") + assert(getIsolationLevel(1) == "Serializable") + } + } } From d7f1e7823d98986d9fa17acf14e82d9e5ecfefdc Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 11 Oct 2021 10:15:26 -0700 Subject: [PATCH 098/291] [#125] Fix a bug in CloseableParquetDataIterator where a single empty file would stop iteration (#170) --- ...-933d-5bf9b7d0b223-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-8d0e-707edf853376-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9d15-54fa113e8c21-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a0f7-c503fe640c6a-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-954a-c5b22f5b157a-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a2c6-7e90df860b5a-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-b31a-5602a972ffa4-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9596-d543afa47196-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a526-08a9a957ad58-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a25f-84acb1240f3a-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-8ebc-9235c74408d2-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9d76-22addae085d1-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...0000000000000000010.checkpoint.parquet.crc | Bin 0 -> 96 bytes .../_delta_log/00000000000000000000.json | 4 +++ .../_delta_log/00000000000000000001.json | 2 ++ .../_delta_log/00000000000000000002.json | 2 ++ .../_delta_log/00000000000000000003.json | 2 ++ .../_delta_log/00000000000000000004.json | 2 ++ .../_delta_log/00000000000000000005.json | 2 ++ .../_delta_log/00000000000000000006.json | 2 ++ .../_delta_log/00000000000000000007.json | 2 ++ .../_delta_log/00000000000000000008.json | 2 ++ .../_delta_log/00000000000000000009.json | 2 ++ .../00000000000000000010.checkpoint.parquet | Bin 0 -> 10814 bytes .../_delta_log/00000000000000000010.json | 2 ++ .../_delta_log/00000000000000000011.json | 2 ++ .../_delta_log/_last_checkpoint | 1 + ...490b-933d-5bf9b7d0b223-c000.snappy.parquet | Bin 0 -> 433 bytes ...4068-8d0e-707edf853376-c000.snappy.parquet | Bin 0 -> 433 bytes ...4e8a-9d15-54fa113e8c21-c000.snappy.parquet | Bin 0 -> 433 bytes ...423c-a0f7-c503fe640c6a-c000.snappy.parquet | Bin 0 -> 303 bytes ...4fa7-954a-c5b22f5b157a-c000.snappy.parquet | Bin 0 -> 303 bytes ...4cfa-a2c6-7e90df860b5a-c000.snappy.parquet | Bin 0 -> 303 bytes ...4519-b31a-5602a972ffa4-c000.snappy.parquet | Bin 0 -> 303 bytes ...4caa-9596-d543afa47196-c000.snappy.parquet | Bin 0 -> 433 bytes ...4b53-a526-08a9a957ad58-c000.snappy.parquet | Bin 0 -> 433 bytes ...455b-a25f-84acb1240f3a-c000.snappy.parquet | Bin 0 -> 303 bytes ...4476-8ebc-9235c74408d2-c000.snappy.parquet | Bin 0 -> 303 bytes ...4b40-9d76-22addae085d1-c000.snappy.parquet | Bin 0 -> 303 bytes .../scala/io/delta/golden/GoldenTables.scala | 16 ++++++++++ .../data/CloseableParquetDataIterator.scala | 29 +++++++++++------- .../internal/DeltaDataReaderSuite.scala | 26 ++++++++++++++-- 42 files changed, 85 insertions(+), 13 deletions(-) create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-15088d9b-5348-490b-933d-5bf9b7d0b223-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-1b8ea57e-424b-4068-8d0e-707edf853376-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-223768c3-2e58-4e8a-9d15-54fa113e8c21-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-2a248db5-8f96-423c-a0f7-c503fe640c6a-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-3f0f0396-41aa-4fa7-954a-c5b22f5b157a-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-66d18d0c-8cab-4cfa-a2c6-7e90df860b5a-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-93beced9-3a9d-4519-b31a-5602a972ffa4-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-c4738537-d851-4caa-9596-d543afa47196-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-c855206c-f42a-4b53-a526-08a9a957ad58-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-d8e947c6-4f26-455b-a25f-84acb1240f3a-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-f0b12818-15f5-4476-8ebc-9235c74408d2-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-f9490ff6-f374-4b40-9d76-22addae085d1-c000.snappy.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/.00000000000000000010.checkpoint.parquet.crc create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000000.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000001.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000002.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000003.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000004.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000005.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000006.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000007.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000008.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000009.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000010.checkpoint.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000010.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000011.json create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/_last_checkpoint create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-15088d9b-5348-490b-933d-5bf9b7d0b223-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-1b8ea57e-424b-4068-8d0e-707edf853376-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-223768c3-2e58-4e8a-9d15-54fa113e8c21-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-2a248db5-8f96-423c-a0f7-c503fe640c6a-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-3f0f0396-41aa-4fa7-954a-c5b22f5b157a-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-66d18d0c-8cab-4cfa-a2c6-7e90df860b5a-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-93beced9-3a9d-4519-b31a-5602a972ffa4-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-c4738537-d851-4caa-9596-d543afa47196-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-c855206c-f42a-4b53-a526-08a9a957ad58-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-d8e947c6-4f26-455b-a25f-84acb1240f3a-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-f0b12818-15f5-4476-8ebc-9235c74408d2-c000.snappy.parquet create mode 100644 golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-f9490ff6-f374-4b40-9d76-22addae085d1-c000.snappy.parquet diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-15088d9b-5348-490b-933d-5bf9b7d0b223-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-15088d9b-5348-490b-933d-5bf9b7d0b223-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..865bb2ac260e03a886356ab11df2524b8d1adc19 GIT binary patch literal 12 TcmYc;N@ieSU}E^DcZ2}|6KVq^ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-1b8ea57e-424b-4068-8d0e-707edf853376-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-1b8ea57e-424b-4068-8d0e-707edf853376-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..4a70137d3cc0a38a39fb7870debdaa00a4f2a67f GIT binary patch literal 12 TcmYc;N@ieSU}CsrXf*0 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-223768c3-2e58-4e8a-9d15-54fa113e8c21-c000.snappy.parquet.crc b/golden-tables/src/test/resources/golden/125-iterator-bug/.part-00000-223768c3-2e58-4e8a-9d15-54fa113e8c21-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..846c3b73ba77e88af1b47a9e016257589787f90a GIT binary patch literal 12 TcmYc;N@ieSU}Cs=h$<<=9rzD zmZto|>y%ZtXmJg_c$5IVPty_+F{&3IFKwTfH97#O6^C^?U$$$mO}xxv6q^xyMx*1Z C!73R5 literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000000.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..79bec19a1bb --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000000.json @@ -0,0 +1,4 @@ +{"commitInfo":{"timestamp":1633728454095,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"col1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1633728453099}} +{"add":{"path":"part-00000-2a248db5-8f96-423c-a0f7-c503fe640c6a-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728454000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000001.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..8094d0bafda --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728458439,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":0,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"433","numOutputRows":"1"}}} +{"add":{"path":"part-00000-15088d9b-5348-490b-933d-5bf9b7d0b223-c000.snappy.parquet","partitionValues":{},"size":433,"modificationTime":1633728458000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000002.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000..0bd34c97bf0 --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000002.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728459288,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"433","numOutputRows":"1"}}} +{"add":{"path":"part-00000-c855206c-f42a-4b53-a526-08a9a957ad58-c000.snappy.parquet","partitionValues":{},"size":433,"modificationTime":1633728459000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000003.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000..2a5e730b747 --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000003.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728460020,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"add":{"path":"part-00000-3f0f0396-41aa-4fa7-954a-c5b22f5b157a-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728460000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000004.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000004.json new file mode 100644 index 00000000000..08a1c313d3e --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000004.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728460726,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":3,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"433","numOutputRows":"1"}}} +{"add":{"path":"part-00000-c4738537-d851-4caa-9596-d543afa47196-c000.snappy.parquet","partitionValues":{},"size":433,"modificationTime":1633728460000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000005.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000005.json new file mode 100644 index 00000000000..1d73654a2fc --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000005.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728461405,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":4,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"add":{"path":"part-00000-f9490ff6-f374-4b40-9d76-22addae085d1-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728461000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000006.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000006.json new file mode 100644 index 00000000000..09668d42414 --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000006.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728462063,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":5,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"add":{"path":"part-00000-66d18d0c-8cab-4cfa-a2c6-7e90df860b5a-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728462000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000007.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000007.json new file mode 100644 index 00000000000..cde15e89a1b --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000007.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728462739,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":6,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"433","numOutputRows":"1"}}} +{"add":{"path":"part-00000-1b8ea57e-424b-4068-8d0e-707edf853376-c000.snappy.parquet","partitionValues":{},"size":433,"modificationTime":1633728462000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000008.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000008.json new file mode 100644 index 00000000000..892404a7f9e --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000008.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728463394,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":7,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"add":{"path":"part-00000-93beced9-3a9d-4519-b31a-5602a972ffa4-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728463000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000009.json b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000009.json new file mode 100644 index 00000000000..fefb3fb496c --- /dev/null +++ b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000009.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1633728464026,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":8,"isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputBytes":"303","numOutputRows":"0"}}} +{"add":{"path":"part-00000-d8e947c6-4f26-455b-a25f-84acb1240f3a-c000.snappy.parquet","partitionValues":{},"size":303,"modificationTime":1633728464000,"dataChange":true}} diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000010.checkpoint.parquet b/golden-tables/src/test/resources/golden/125-iterator-bug/_delta_log/00000000000000000010.checkpoint.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d54ab2bccae049adb75b8acd4e6b085134d581a5 GIT binary patch literal 10814 zcmds7U1%KF6`ozMR_jQ1B-=9{DNCv@@@DO5HnY1kyEA4VT)V_}P>M<8G?1Eb=TF{d z|FS!B?ZyT@1rdTUp%ha}@lz1N1-F=BLLTyHN@Dub(moW738hdf{7{N9dg!@#X70?+ ztfXBzveO7jJ9For@BH3#&poqee(+j>W!Q0cVve1LizJ&$m!}zqNgjy9|7qs29^M#d zAC3lq6S)&@)UgE-{|J^UsiI|-1*=q%1WA)bK{YC6p;$zQflN`6je?-V%zU?r+U-ku zpnZ4IwD}-NK^7IoP&Gj=Ns0h~nxK|Sh9GN}s#Oe8D;7)rKtQ=)B~OjO$H?_hK8c@z z3rDnA?4Gb)(4!}rf0iuK5=&}XkO~M1l7%XQDoaSvWe~-ZwSruMo2XVGL4Jj~s+KfU zHw{%NA=MBhxu6PKselBzEEbVkDOwhig!fM}S9PgUQsh!aFci5UNIEc6WngQ_QVGGN zN&#*@ILRz4vRo9)x?o8~kVlhC0+NemK~#{6R1n6Hl@HRftA=8#Qbh;3EqE%)vL>LS zYzc~lbgfX7M5}~^_d!BSgj=PcfR!v+kR}3ySsq|IQanx`0Hh0`?M1mRXiWy}a^g zX1QE83P7z3ijDwGw-AU2(pF4W1hBHG$tz%$fnw^FbPU4!c{byPX3~lIXB8*rX;i?h z44Vzjnmp#r8pW}*G<*7^S%yupGwkd+)sd+W}n9}39`WWlsAJ< zd2_f#uFoid>=CIru@P$&AeLh6&*s?)%Y=fxmcX&D2Ow8jEb{sWV!f@6&R=ImMQpDiqZ?4#M!`!L!*mh*X&+_ig#@N;o?D7S42 z%d=@uCO5Xm=bWgaS#^LZPk=?1J>m%pj!W8R*FJ4{pB$eHjes-vJ4rJImr3|@C^mBg zgz2Vka-Zkexqu|2Uat2pKo;+k_bveVz&vsFI9%fO=u&MRJv>Y%Vr>y+bl#t0>>pzR z0pEHrV_#~UnWLGm-C5M_%v{E*nRTO^ISP-OsDU5pt$G0-xD#tl+gva^Fwk7A*O69- zDL`w^Wg4c93}mCs(f6)gId_HMNpLXaPQve~?BKHVe!`BCPz}E3I z0lWvvWmwK3&;rp2%Q5w@zd0G(zXI>|83iV8UdH@9D&XsGJ5F84C-8pc{0E%C9SpnM z3{FpYR+JV$;S=~jruFY&AV7=9f%XYxt{rC)pISeDYP<_6<(;>?`gyZ~*l|XV^SkVe zO*RQS9*XlLP?w%I4uQ@IxEsA-cDl7z)43a~qpp3n4eQ!82pv?l>|Td1(MI-pAOpB( zaoJh3f$R-bUo^XHoQ*Sbg5Uj)dFky7cnCInV8-yfU<9)wjL8#^aiUxMh3RmeXta!) zRnrkhe7)9yVFpBd2B()3=TURP#7q-5S{RxW;oG&5-E|~ScFacWf*IC{m}1f}>!v3H z&@|dnIf!5f%!{_!G)&{=THSor&4b^pUACz&kfm^9{vlY7op8(^uMuZsOC)fP_uMgN zm|eY7Yh#UF<5;au1KE!60IA(+U8osm$9X#1B2%~Jf^jDpe{6SZ%>@T;g2vE^R(-M2 z#JEWuM2UK>YePJdm>C6^G|Z;$(CpD$O{=!B*ddX!ov0hrJ0=cp5@pz0prhS_Jr64# z2NO*+YR%V71X{o0Eevn?C!Lz@4+HTo-O$;TBj}E-j%`K(H$9b}TF+%uX?A={P9>SC z98Lhv!e`^$G*{-1W>ZxtJg3r-`9O0>f!X6}HW8TaWd=%cJWO!8mmrutDADCG(WkM1 zRFGbX;mGv0z1(NdWIv5n%ipFmchEu)#<1*%z@Vpgf#~5h=%+uN_J~fWxcTgZv9Jw3 zA0H5B#7U;uW%Jyb?C0ZQq!0HHA0$#q@G-U~&Z>m--5B?mSoXs`)gex9Q*-z@h}RxI z9+IHAzqgk`Bv%t=x8vN$@$CJ*z$|(K+u&T*huQKZgrgUNpiN*0AZ*^`ewtmLtcDJ2 zK@WWm2mud~J!1p6d_5U9=FNSZ5r=$nb6=QzX+I`+*BeMbLiQaZc$GVrUD_We_#!~y zZcP4az)SRt2+1QL@fN8|65X~4OHBVLLVxRkZ()}{Y-CJ$>p)o6wJ2fVZIJR_i$JWh z)Vwb10T$44?&a(%8gfkp}%!- z3;K}Iw+@CSctjAKV53_;Z1u$cS7Y2SW7$UstKO;D`-Cz$d^!%Uo)x(9BA9}bqJc*7 z%2ZgAyZwlQ#Ra9k8=+m9rVJIzFzJ*bwEQ6Szm9Rgjb&G+tMnvIFF+~pbWO*4Hntnj zdDeb2%)~2d`gjf}+#An@wOtV*)P-89X+Y; zC^$Ij|Mv(VTq8GTy;Jr=2DNe%1OuEd2Tnr0)>)Ca7$$n6uDxTaW-K;1bMm1`rY zP+Z_YwyulsTSnbr$ldji8~CC%uil+qB?EN_jCaGW_AljPAR2@7-Iz)B1Vd zYgh8Wn;xY92FE4Z1RZ2P>8Vf(++T%*?p9sXIofn}_3;7ftzpOmU7O(`&RyA}vFnQn z1uexM#3Hy`LC&N2M4rIEIK-r^!bd zP&GGs?eTcw8tEKT-b2R9H{#=d?MYvtp+n}ov=*OY_zw}kUJI8xZIB4A`T2Rk73F6I z^6rnm^c!~f3N(_b#6yeV9Qr8zAR2r9X-YISQ@5o}0Vvlevb_uu5yAdBm3C-#=}LB6 z?i7jYjXE8435MOadw3s({`_W)FufpYZ_PV?R^PzKb@n`}hk z_YagfGvKRWUq^uIIq(b(HH5BkDxvU$sQGyNEe+)FKzaCW-jdE;Vfa_T*!U}qLZdU| zo>OKD`GS})%;dDiTHTmIiV9z6vsATI)XJt+F2k1@#j;pdO(o~Vr|L!+8K@hxr zB=5a^?|olZcaI@K_R?mr zvJwOXc!lXOYgCAA?6{6Qwp=4+kdPa7vexS@BuMKj!$W$*IaA_ zSn+RH{w{S}HZ?tsc)$Zhr+*afR&fZp?WEzl`QVi_A{R!`ryVn&8_AZeHM+|T^I1_! zmA7mOR4uAKwI2hBqp+-vTuTi=-6+e?$a zs!|dR;uWUDwAKN#vEw@K*m41!StH~ebb@VcSGE~jxBo;rE{p@+AO~QK9G}wKthwAs zh~nR={9T!*s4IFJa=`_n(?3MB)f@)gcB4TeWj@Ob zrL%@D!K!4rxAxZn;|La2tJcbZP<2|%L<0h6mCt^4TeWj@Ob zrL%@Dz^Y_`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx57#r#A{3kX{`ffW5;#ek>vt7wT8$!>IB=^u52^1?*55#To?zsK@PweIX4TeWj@Ob zrL%@Dz^Y_(o~VL|L#{eK@hxr zB=5a^@8$becaI@K z;@__PUFx=MYI+*+fCq?9|0vq6;t+7#NyByX!7FJ*E{vd0J7zvNk}X+lbe9?Cv!awL zZ`l&4T2wvNe*=i4u&j+-OASEXDJ4<`5_l_p_S2Oucl-VG9?^%R=XIsMV@NOLVZcLg yl6IxaJ+VmgVo@YXbd$!ih~p^AgE)v2ISaBRohS1km1!PM9oK~!r~yJNSk@QWrekFQ literal 0 HcmV?d00001 diff --git a/golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-d8e947c6-4f26-455b-a25f-84acb1240f3a-c000.snappy.parquet b/golden-tables/src/test/resources/golden/125-iterator-bug/part-00000-d8e947c6-4f26-455b-a25f-84acb1240f3a-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..5b5d160be232e1c7d1cf0a824274913a36699471 GIT binary patch literal 303 zcmZXQ&r8EF6vt!H+pZlc>`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5`)?tgBlWx-O`&k@ie>+k)(N@#q`ILRFTpD-AwAq>-&7( z_xpIe#}^()N%pjGgR~!Su6l=7#u196n+RW4z2Rtyl0}kX+}i_{gA4}!mFXu~*Yq8} z$|<7^;q^bm9Sse+#}PcW{TFS)OD(;``xEis1CWw9Z^z1$jMNPn?MN9=C)+bRw+Aqb zP3PeNHVnGa7^zH9!Y*XwFO`(1X?mRyex5t;TAR*Crkrv{d3vkH#^_XTg|4@?5c$0- ipe~9$*Q{WLfCp9ywGkVpKxw{CRx5 #125: iterator bug */ + generateGoldenTable("125-iterator-bug") { tablePath => + val datas = Seq( + Seq(), + Seq(1), + Seq(2), Seq(), + Seq(3), Seq(), Seq(), + Seq(4), Seq(), Seq(), Seq(), + Seq(5) + ) + datas.foreach { data => + data.toDF("col1").write.format("delta").mode("append").save(tablePath) + } + } + generateGoldenTable("deltatbl-not-allow-write", createHiveGoldenTableFile) { tablePath => val data = (0 until 10).map(x => (x, s"foo${x % 2}")) data.toDF("a", "b").write.format("delta").save(tablePath) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala index bd9cafa82e9..37bc516b385 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/CloseableParquetDataIterator.scala @@ -69,20 +69,27 @@ private[internal] case class CloseableParquetDataIterator( return false } - // More rows in current file - if (parquetRowsIter.hasNext) return true + // We need to search for the next non-empty file + while (true) { + // More rows in current file + if (parquetRowsIter.hasNext) return true - // No more rows in current file and no more files - if (!dataFilePathsIter.hasNext) { - close() - return false + // No more rows in current file and no more files + if (!dataFilePathsIter.hasNext) { + close() + return false + } + + // No more rows in this file, but there is a next file + parquetRows.close() + + // Repeat the search at the next file + parquetRows = readNextFile + parquetRowsIter = parquetRows.iterator } - // No more rows in this file, but there is a next file - parquetRows.close() - parquetRows = readNextFile - parquetRowsIter = parquetRows.iterator - parquetRowsIter.hasNext + // Impossible + throw new RuntimeException("Some bug in CloseableParquetDataIterator::hasNext") } /** diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala index 579529071be..80162b60e99 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaDataReaderSuite.scala @@ -22,15 +22,15 @@ import java.util.{TimeZone, List => JList, Map => JMap} import java.util.Arrays.{asList => asJList} import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer -import io.delta.standalone.data.{RowRecord => JRowRecord} +import io.delta.standalone.data.{CloseableIterator, RowRecord => JRowRecord} import io.delta.standalone.DeltaLog import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.util.DataTypeParser import io.delta.standalone.internal.util.GoldenTableUtils._ import io.delta.standalone.types._ import org.apache.hadoop.conf.Configuration - import org.scalatest.FunSuite /** @@ -361,5 +361,27 @@ class DeltaDataReaderSuite extends FunSuite { val struct = new StructType(field_array) assert(struct == DataTypeParser.fromJson(struct.toJson())) } + + // scalastyle:off line.size.limit + test("#125: CloseableParquetDataIterator should not stop iteration when processing an empty file") { + // scalastyle:on line.size.limit + withLogForGoldenTable("125-iterator-bug") { log => + var datas = new ListBuffer[Int]() + var dataIter: CloseableIterator[JRowRecord] = null + try { + dataIter = log.update().open() + while (dataIter.hasNext) { + datas += dataIter.next().getInt("col1") + } + + assert(datas.length == 5) + assert(datas.toSet == Set(1, 2, 3, 4, 5)) + } finally { + if (null != dataIter) { + dataIter.close() + } + } + } + } } From d69c76c84bd9d2c7b7dd323f186abeeecc4a5600 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 13 Oct 2021 09:16:04 -0700 Subject: [PATCH 099/291] [DSW] [18] OSS Compatibility Tests v3 (#167) --- build.sbt | 3 +- .../tests/OSSCompatibilitySuite.scala | 266 +++++++++++++++++- .../tests/OssCompatibilitySuiteBase.scala | 201 +++++++++++++ .../internal/util/ComparisonUtil.scala | 1 - .../standalone/internal/util/OSSUtil.scala | 33 +++ .../internal/util/StandaloneUtil.scala | 13 +- .../OptimisticTransactionLegacySuite.scala | 1 - .../internal/OptimisticTransactionSuite.scala | 76 ++--- .../OptimisticTransactionSuiteBase.scala | 3 +- .../OptimisticTransactionSuiteTestVals.scala | 50 ++++ 10 files changed, 572 insertions(+), 75 deletions(-) create mode 100644 oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala diff --git a/build.sbt b/build.sbt index 0b7a5c1b962..2e78636da16 100644 --- a/build.sbt +++ b/build.sbt @@ -335,7 +335,8 @@ lazy val mimaSettings = Seq( ) lazy val compatibility = (project in file("oss-compatibility-tests")) - .dependsOn(standalone) + // depend on standalone test codes as well + .dependsOn(standalone % "compile->compile;test->test") .settings( name := "compatibility", commonSettings, diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala index f74bf269a5d..1e3f6d4989b 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala @@ -24,19 +24,13 @@ import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog => StandaloneDeltaLog} import io.delta.standalone.internal.{DeltaLogImpl => InternalStandaloneDeltaLog} -import io.delta.standalone.internal.util.{ComparisonUtil, OSSUtil, StandaloneUtil} +import io.delta.standalone.internal.util.ComparisonUtil import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.delta.{DeltaLog => OSSDeltaLog} -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.SharedSparkSession -class OSSCompatibilitySuite extends QueryTest with SharedSparkSession with ComparisonUtil { - - private val now = System.currentTimeMillis() - private val ss = new StandaloneUtil(now) - private val oo = new OSSUtil(now) +class OSSCompatibilitySuite extends OssCompatibilitySuiteBase with ComparisonUtil { /** * Creates a temporary directory, a public Standalone DeltaLog, an internal Standalone DeltaLog, @@ -199,11 +193,257 @@ class OSSCompatibilitySuite extends QueryTest with SharedSparkSession with Compa } } - test("Standalone (with fixed Protocol(1, 2)) read from higher protocol OSS table") { - // TODO - } + test("Standalone writer write to higher protocol OSS table should fail") { + withTempDirAndLogs { (_, standaloneLog, _, ossLog) => + ossLog.startTransaction().commit(oo.metadata :: oo.protocol13 :: Nil, oo.op) + + // scalastyle:off line.size.limit + val e = intercept[io.delta.standalone.internal.exception.DeltaErrors.InvalidProtocolVersionException] { + // scalastyle:on line.size.limit + standaloneLog.startTransaction().commit(Iterable().asJava, ss.op, ss.engineInfo) + } - test("concurrency conflicts") { - // TODO + assert(e.getMessage.contains( + """ + |Delta protocol version (1,3) is too new for this version of Delta + |Standalone Reader/Writer (1,2). Please upgrade to a newer release. + |""".stripMargin)) + } } + + /////////////////////////////////////////////////////////////////////////// + // Allowed concurrent actions + /////////////////////////////////////////////////////////////////////////// + + checkStandalone( + "append / append", + conflicts = false, + reads = Seq(t => t.metadata()), + concurrentOSSWrites = Seq(oo.conflict.addA), + actions = Seq(ss.conflict.addB)) + + checkOSS( + "append / append", + conflicts = false, + reads = Seq(t => t.metadata), + concurrentStandaloneWrites = Seq(ss.conflict.addA), + actions = Seq(oo.conflict.addB)) + + checkStandalone( + "disjoint txns", + conflicts = false, + reads = Seq(t => t.txnVersion("foo")), + concurrentOSSWrites = Seq(oo.setTransaction), + actions = Nil) + + checkOSS( + "disjoint txns", + conflicts = false, + reads = Seq(t => t.txnVersion("foo")), + concurrentStandaloneWrites = Seq(ss.setTransaction), + actions = Nil) + + checkStandalone( + "disjoint delete / read", + conflicts = false, + setup = Seq(ss.conflict.metadata_partX, ss.conflict.addA_partX2), + reads = Seq(t => t.markFilesAsRead(ss.conflict.colXEq1Filter)), + concurrentOSSWrites = Seq(oo.conflict.removeA), + actions = Seq() + ) + + checkOSS( + "disjoint delete / read", + conflicts = false, + setup = Seq(oo.conflict.metadata_partX, oo.conflict.addA_partX2), + reads = Seq(t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil)), + concurrentStandaloneWrites = Seq(ss.conflict.removeA), + actions = Seq() + ) + + checkStandalone( + "disjoint add / read", + conflicts = false, + setup = Seq(ss.conflict.metadata_partX), + reads = Seq(t => t.markFilesAsRead(ss.conflict.colXEq1Filter)), + concurrentOSSWrites = Seq(oo.conflict.addA_partX2), + actions = Seq() + ) + + checkOSS( + "disjoint add / read", + conflicts = false, + setup = Seq(oo.conflict.metadata_partX), + reads = Seq(t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil)), + concurrentStandaloneWrites = Seq(ss.conflict.addA_partX2), + actions = Seq() + ) + + checkStandalone( + "add / read + no write", // no write = no real conflicting change even though data was added + conflicts = false, // so this should not conflict + setup = Seq(ss.conflict.metadata_partX), + reads = Seq(t => t.markFilesAsRead(ss.conflict.colXEq1Filter)), + concurrentOSSWrites = Seq(oo.conflict.addA_partX1), + actions = Seq()) + + checkOSS( + "add / read + no write", // no write = no real conflicting change even though data was added + conflicts = false, // so this should not conflict + setup = Seq(oo.conflict.metadata_partX), + reads = Seq(t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil)), + concurrentStandaloneWrites = Seq(ss.conflict.addA_partX1), + actions = Seq()) + + /////////////////////////////////////////////////////////////////////////// + // Disallowed concurrent actions + /////////////////////////////////////////////////////////////////////////// + + checkStandalone( + "delete / delete", + conflicts = true, + reads = Nil, + concurrentOSSWrites = Seq(oo.conflict.removeA), + actions = Seq(ss.conflict.removeA_time5) + ) + + checkOSS( + "delete / delete", + conflicts = true, + reads = Nil, + concurrentStandaloneWrites = Seq(ss.conflict.removeA), + actions = Seq(oo.conflict.removeA_time5) + ) + + checkStandalone( + "add / read + write", + conflicts = true, + setup = Seq(ss.conflict.metadata_partX), + reads = Seq(t => t.markFilesAsRead(ss.conflict.colXEq1Filter)), + concurrentOSSWrites = Seq(oo.conflict.addA_partX1), + actions = Seq(ss.conflict.addB_partX1), + // commit info should show operation as "Manual Update", because that's the operation used by + // the harness + errorMessageHint = Some("[x=1]" :: "Manual Update" :: Nil)) + + checkOSS( + "add / read + write", + conflicts = true, + setup = Seq(oo.conflict.metadata_partX), + reads = Seq(t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil)), + concurrentStandaloneWrites = Seq(ss.conflict.addA_partX1), + actions = Seq(oo.conflict.addB_partX1), + // commit info should show operation as "Manual Update", because that's the operation used by + // the harness + errorMessageHint = Some("[x=1]" :: "Manual Update" :: Nil)) + + checkStandalone( + "delete / read", + conflicts = true, + setup = Seq(ss.conflict.metadata_partX, ss.conflict.addA_partX1), + reads = Seq(t => t.markFilesAsRead(ss.conflict.colXEq1Filter)), + concurrentOSSWrites = Seq(oo.conflict.removeA), + actions = Seq(), + errorMessageHint = Some("a in partition [x=1]" :: "Manual Update" :: Nil)) + + checkOSS( + "delete / read", + conflicts = true, + setup = Seq(oo.conflict.metadata_partX, oo.conflict.addA_partX1), + reads = Seq(t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil)), + concurrentStandaloneWrites = Seq(ss.conflict.removeA), + actions = Seq(), + errorMessageHint = Some("a in partition [x=1]" :: "Manual Update" :: Nil)) + + checkStandalone( + "schema change", + conflicts = true, + reads = Seq(t => t.metadata), + concurrentOSSWrites = Seq(oo.metadata), + actions = Nil) + + checkOSS( + "schema change", + conflicts = true, + reads = Seq(t => t.metadata), + concurrentStandaloneWrites = Seq(ss.metadata), + actions = Nil) + + checkStandalone( + "conflicting txns", + conflicts = true, + reads = Seq(t => t.txnVersion(oo.setTransaction.appId)), + concurrentOSSWrites = Seq(oo.setTransaction), + actions = Nil) + + checkOSS( + "conflicting txns", + conflicts = true, + reads = Seq(t => t.txnVersion(ss.setTransaction.getAppId)), + concurrentStandaloneWrites = Seq(ss.setTransaction), + actions = Nil) + + checkStandalone( + "upgrade / upgrade", + conflicts = true, + reads = Seq(t => t.metadata), + concurrentOSSWrites = Seq(oo.protocol12), + actions = Seq(ss.protocol12)) + + checkOSS( + "upgrade / upgrade", + conflicts = true, + reads = Seq(t => t.metadata), + concurrentStandaloneWrites = Seq(ss.protocol12), + actions = Seq(oo.protocol12)) + + checkStandalone( + "taint whole table", + conflicts = true, + setup = Seq(ss.conflict.metadata_partX, ss.conflict.addA_partX2), + reads = Seq( + t => t.markFilesAsRead(ss.conflict.colXEq1Filter), + // `readWholeTable` should disallow any concurrent change, even if the change + // is disjoint with the earlier filter + t => t.readWholeTable() + ), + concurrentOSSWrites = Seq(oo.conflict.addB_partX3), + actions = Seq(ss.conflict.addC_partX4) + ) + + checkOSS( + "taint whole table", + conflicts = true, + setup = Seq(oo.conflict.metadata_partX, oo.conflict.addA_partX2), + reads = Seq( + t => t.filterFiles(oo.conflict.colXEq1Filter :: Nil), + // `readWholeTable` should disallow any concurrent change, even if the change + // is disjoint with the earlier filter + t => t.readWholeTable() + ), + concurrentStandaloneWrites = Seq(ss.conflict.addB_partX3), + actions = Seq(oo.conflict.addC_partX4) + ) + + checkStandalone( + "taint whole table + concurrent remove", + conflicts = true, + setup = Seq(ss.conflict.metadata_colX, ss.conflict.addA), + reads = Seq( + // `readWholeTable` should disallow any concurrent `RemoveFile`s. + t => t.readWholeTable() + ), + concurrentOSSWrites = Seq(oo.conflict.removeA), + actions = Seq(ss.conflict.addB)) + + checkOSS( + "taint whole table + concurrent remove", + conflicts = true, + setup = Seq(oo.conflict.metadata_colX, oo.conflict.addA), + reads = Seq( + // `readWholeTable` should disallow any concurrent `RemoveFile`s. + t => t.readWholeTable() + ), + concurrentStandaloneWrites = Seq(ss.conflict.removeA), + actions = Seq(oo.conflict.addB)) } diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala new file mode 100644 index 00000000000..399a736d212 --- /dev/null +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala @@ -0,0 +1,201 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.compatibility.tests + +import java.util.ConcurrentModificationException + +import scala.collection.JavaConverters._ + +import io.delta.standalone.{Operation => StandaloneOperation, actions => StandaloneActions, DeltaLog => StandaloneDeltaLog, OptimisticTransaction => StandaloneOptTxn} +import io.delta.standalone.internal.util.{OSSUtil, StandaloneUtil} + +import org.apache.spark.sql.delta.{DeltaOperations, actions => OSSActions, DeltaLog => OSSDeltaLog, OptimisticTransaction => OSSOptTxn} +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +trait OssCompatibilitySuiteBase extends QueryTest with SharedSparkSession { + + protected val now = System.currentTimeMillis() + protected val ss = new StandaloneUtil(now) + protected val oo = new OSSUtil(now) + + private val standaloneConflictOp = new StandaloneOperation(StandaloneOperation.Name.MANUAL_UPDATE) + private val ossConflictOp = DeltaOperations.ManualUpdate + + /** + * Tests a DELTA STANDALONE transaction getting conflicted by a DELTA OSS commit (i.e. during the + * DSW transaction, a Delta OSS commit occurs and wins). + * + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - setup (including setting table isolation level + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[java.util.ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param testName test name + * @param conflicts should test transaction is expected to conflict or not + * @param setup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentOSSWrites writes made by concurrent transactions after the test txn reads + * @param actions actions to be committed by the test transaction + * @param exceptionClass A substring to expect in the exception class name + */ + protected def checkStandalone( + testName: String, + conflicts: Boolean, + setup: Seq[StandaloneActions.Action] = + Seq(StandaloneActions.Metadata.builder().build(), new StandaloneActions.Protocol()), + reads: Seq[StandaloneOptTxn => Unit], + concurrentOSSWrites: Seq[OSSActions.Action], + actions: Seq[StandaloneActions.Action], + errorMessageHint: Option[Seq[String]] = None, + exceptionClass: Option[String] = None): Unit = { + + val concurrentTxn: OSSOptTxn => Unit = + (opt: OSSOptTxn) => opt.commit(concurrentOSSWrites, ossConflictOp) + + def initialSetup(log: StandaloneDeltaLog): Unit = { + setup.foreach { action => + log.startTransaction().commit(Seq(action).asJava, standaloneConflictOp, ss.engineInfo) + } + } + + val conflictMsg = if (conflicts) "should conflict" else "should not conflict" + test(s"checkStandalone - $testName - $conflictMsg") { + withTempDir { tempDir => + // Standalone loses + val losingLog = + StandaloneDeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath)) + + // OSS wins + val winningLog = OSSDeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) + + // Setup the log + initialSetup(losingLog) + + // Perform reads + val standaloneTxn = losingLog.startTransaction() + reads.foreach(_ (standaloneTxn)) + + // Execute concurrent txn while current transaction is active + concurrentTxn(winningLog.startTransaction()) + + // Try commit and check expected conflict behavior + if (conflicts) { + val e = intercept[ConcurrentModificationException] { + standaloneTxn.commit(actions.asJava, standaloneConflictOp, ss.engineInfo) + } + errorMessageHint.foreach { expectedParts => + assert(expectedParts.forall(part => e.getMessage.contains(part))) + } + if (exceptionClass.nonEmpty) { + assert(e.getClass.getName.contains(exceptionClass.get)) + } + } else { + standaloneTxn.commit(actions.asJava, standaloneConflictOp, ss.engineInfo) + } + } + } + } + + /** + * Tests a DELTA OSS transaction getting conflicted by a DELTA STANDALONE commit (i.e. during the + * Delta OSS transaction, a Delta Standalone commit occurs and wins). + * + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - setup (including setting table isolation level + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[java.util.ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param testName test name + * @param conflicts should test transaction is expected to conflict or not + * @param setup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentStandaloneWrites writes made by concurrent transactions after the test txn + * reads + * @param actions actions to be committed by the test transaction + * @param exceptionClass A substring to expect in the exception class name + */ + protected def checkOSS( + testName: String, + conflicts: Boolean, + setup: Seq[OSSActions.Action] = Seq(OSSActions.Metadata(), OSSActions.Protocol(1, 2)), + reads: Seq[OSSOptTxn => Unit], + concurrentStandaloneWrites: Seq[StandaloneActions.Action], // winning Delta Standalone writes + actions: Seq[OSSActions.Action], + errorMessageHint: Option[Seq[String]] = None, + exceptionClass: Option[String] = None): Unit = { + val concurrentTxn: StandaloneOptTxn => Unit = + (opt: StandaloneOptTxn) => + opt.commit(concurrentStandaloneWrites.asJava, standaloneConflictOp, ss.engineInfo) + + def initialSetup(log: OSSDeltaLog): Unit = { + setup.foreach { action => + log.startTransaction().commit(Seq(action), ossConflictOp) + } + } + + val conflictMsg = if (conflicts) "should conflict" else "should not conflict" + test(s"checkOSS - $testName - $conflictMsg") { + withTempDir { tempDir => + // OSS loses + val losingLog = OSSDeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) + + // Standalone wins + val winningLog = + StandaloneDeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath)) + + // Setup the log + initialSetup(losingLog) + + // Perform reads + val ossTxn = losingLog.startTransaction() + reads.foreach(_ (ossTxn)) + + // Execute concurrent txn while current transaction is active + concurrentTxn(winningLog.startTransaction()) + + // Try commit and check expected conflict behavior + if (conflicts) { + val e = intercept[ConcurrentModificationException] { + ossTxn.commit(actions, ossConflictOp) + } + errorMessageHint.foreach { expectedParts => + assert(expectedParts.forall(part => e.getMessage.contains(part))) + } + if (exceptionClass.nonEmpty) { + assert(e.getClass.getName.contains(exceptionClass.get)) + } + } else { + ossTxn.commit(actions, ossConflictOp) + } + } + } + } + +} diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala index ef6ed10502b..cbd7bea85be 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala @@ -14,7 +14,6 @@ * limitations under the License. */ - package io.delta.standalone.internal.util import scala.collection.JavaConverters._ diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala index 51d87ba31b0..3283e5bf125 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations} import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} class OSSUtil(now: Long) { @@ -43,6 +44,10 @@ class OSSUtil(now: Long) { createdTime = Some(now) ) + val protocol12: Protocol = Protocol(1, 2) + + val protocol13: Protocol = Protocol(1, 3) + val addFiles: Seq[AddFile] = (0 until 50).map { i => AddFile( path = i.toString, @@ -73,4 +78,32 @@ class OSSUtil(now: Long) { commitInfoOpt.get } + + val col1PartitionFilter = + EqualTo(AttributeReference("col1_part", IntegerType, nullable = true)(), Literal(1)) + + val conflict = new ConflictVals() + + class ConflictVals { + val addA = AddFile("a", Map.empty, 1, 1, dataChange = true) + val addB = AddFile("b", Map.empty, 1, 1, dataChange = true) + + val removeA = RemoveFile("a", Some(4)) + val removeA_time5 = RemoveFile("a", Some(5)) + + val addA_partX1 = AddFile("a", Map("x" -> "1"), 1, 1, dataChange = true) + val addA_partX2 = AddFile("a", Map("x" -> "2"), 1, 1, dataChange = true) + val addB_partX1 = AddFile("b", Map("x" -> "1"), 1, 1, dataChange = true) + val addB_partX3 = AddFile("b", Map("x" -> "2"), 1, 1, dataChange = true) + val addC_partX4 = AddFile("c", Map("x" -> "4"), 1, 1, dataChange = true) + + val metadata_colX = Metadata(schemaString = new StructType().add("x", IntegerType).json) + + val metadata_partX = Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x") + ) + + val colXEq1Filter = EqualTo(AttributeReference("x", IntegerType, nullable = true)(), Literal(1)) + } } diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala index a899e469bed..bb0a65ff488 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala @@ -14,16 +14,17 @@ * limitations under the License. */ - package io.delta.standalone.internal.util import java.util.Collections import scala.collection.JavaConverters._ -import io.delta.standalone.actions.{AddFile, Format, Metadata, RemoveFile, SetTransaction} +import io.delta.standalone.actions.{AddFile, Format, Metadata, Protocol, RemoveFile, SetTransaction} import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} import io.delta.standalone.Operation +import io.delta.standalone.expressions.{EqualTo, Literal} +import io.delta.standalone.internal.OptimisticTransactionSuiteTestVals class StandaloneUtil(now: Long) { @@ -53,6 +54,8 @@ class StandaloneUtil(now: Long) { .createdTime(now) .build() + val protocol12: Protocol = new Protocol(1, 2) + val addFiles: Seq[AddFile] = (0 until 50).map { i => new AddFile( i.toString, // path @@ -78,4 +81,10 @@ class StandaloneUtil(now: Long) { val setTransaction: SetTransaction = new SetTransaction("appId", 123, java.util.Optional.of(now + 200)) + + val col1PartitionFilter = new EqualTo(schema.column("col1_part"), Literal.of(1)) + + val conflict = new ConflictVals() + + class ConflictVals extends OptimisticTransactionSuiteTestVals } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index fb8aa05ce29..38b2225131b 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -717,7 +717,6 @@ class OptimisticTransactionLegacySuite extends FunSuite { tx1.txnVersion("t1") val winningTxn = log.startTransaction() - winningTxn.txnVersion("t1") winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, manualUpdate, engineInfo) intercept[ConcurrentTransactionException] { diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index b57b85f41d7..3c9ebdcdcee 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -16,37 +16,21 @@ package io.delta.standalone.internal -import java.util.Collections - import scala.collection.JavaConverters._ -import io.delta.standalone.actions.{CommitInfo, Protocol, AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} -import io.delta.standalone.expressions.{EqualTo, Literal} -import io.delta.standalone.types.{IntegerType, StructField, StructType} +import io.delta.standalone.actions.{CommitInfo, Protocol, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} import io.delta.standalone.internal.util.TestUtils._ import io.delta.standalone.DeltaLog import org.apache.hadoop.conf.Configuration -class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { - private val addA = new AddFileJ("a", Collections.emptyMap(), 1, 1, true, null, null) - private val addB = new AddFileJ("b", Collections.emptyMap(), 1, 1, true, null, null) - - private val removeA = RemoveFileJ.builder("a").deletionTimestamp(4L).build() - private val addA_partX1 = new AddFileJ("a", Map("x" -> "1").asJava, 1, 1, true, null, null) - private val addA_partX2 = new AddFileJ("a", Map("x" -> "2").asJava, 1, 1, true, null, null) - private val addB_partX1 = new AddFileJ("b", Map("x" -> "1").asJava, 1, 1, true, null, null) - private val addB_partX3 = new AddFileJ("b", Map("x" -> "3").asJava, 1, 1, true, null, null) - private val addC_partX4 = new AddFileJ("c", Map("x" -> "4").asJava, 1, 1, true, null, null) - - private val schema = new StructType(Array(new StructField("x", new IntegerType()))) - private val metadata_colX = MetadataJ.builder().schema(schema).build() - private val metadata_partX = - MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() +class OptimisticTransactionSuite + extends OptimisticTransactionSuiteBase + with OptimisticTransactionSuiteTestVals { - /* ************************** * - * Allowed concurrent actions * - * ************************** */ + /////////////////////////////////////////////////////////////////////////// + // Allowed concurrent actions + /////////////////////////////////////////////////////////////////////////// check( "append / append", @@ -58,9 +42,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { check( "disjoint txns", conflicts = false, - reads = Seq( - t => t.txnVersion("t1") - ), + reads = Seq(t => t.txnVersion("t1")), concurrentWrites = Seq( new SetTransactionJ("t2", 0, java.util.Optional.of(1234L))), actions = Nil) @@ -69,9 +51,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { "disjoint delete / read", conflicts = false, setup = Seq(metadata_partX, addA_partX2), - reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) - ), + reads = Seq(t => t.markFilesAsRead(colXEq1Filter)), concurrentWrites = Seq(removeA), actions = Seq() ) @@ -80,9 +60,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { "disjoint add / read", conflicts = false, setup = Seq(metadata_partX), - reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) - ), + reads = Seq(t => t.markFilesAsRead(colXEq1Filter)), concurrentWrites = Seq(addA_partX2), actions = Seq() ) @@ -91,31 +69,27 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { "add / read + no write", // no write = no real conflicting change even though data was added conflicts = false, // so this should not conflict setup = Seq(metadata_partX), - reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) - ), + reads = Seq(t => t.markFilesAsRead(colXEq1Filter)), concurrentWrites = Seq(addA_partX1), actions = Seq()) - /* ***************************** * - * Disallowed concurrent actions * - * ***************************** */ + /////////////////////////////////////////////////////////////////////////// + // Disallowed concurrent actions + /////////////////////////////////////////////////////////////////////////// check( "delete / delete", conflicts = true, reads = Nil, concurrentWrites = Seq(removeA), - actions = Seq(RemoveFileJ.builder("a").deletionTimestamp(5L).build()) + actions = Seq(removeA_time5) ) check( "add / read + write", conflicts = true, setup = Seq(metadata_partX), - reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) - ), + reads = Seq(t => t.markFilesAsRead(colXEq1Filter)), concurrentWrites = Seq(addA_partX1), actions = Seq(addB_partX1), // commit info should show operation as "Manual Update", because that's the operation used by @@ -126,9 +100,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { "delete / read", conflicts = true, setup = Seq(metadata_partX, addA_partX1), - reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))) - ), + reads = Seq(t => t.markFilesAsRead(colXEq1Filter)), concurrentWrites = Seq(removeA), actions = Seq(), errorMessageHint = Some("a in partition [x=1]" :: "Manual Update" :: Nil)) @@ -136,18 +108,14 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { check( "schema change", conflicts = true, - reads = Seq( - t => t.metadata - ), + reads = Seq(t => t.metadata), concurrentWrites = Seq(MetadataJ.builder().build()), actions = Nil) check( "conflicting txns", conflicts = true, - reads = Seq( - t => t.txnVersion("t1") - ), + reads = Seq(t => t.txnVersion("t1")), concurrentWrites = Seq( new SetTransactionJ("t1", 0, java.util.Optional.of(1234L)) ), @@ -156,9 +124,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { check( "upgrade / upgrade", conflicts = true, - reads = Seq( - t => t.metadata - ), + reads = Seq(t => t.metadata), concurrentWrites = Seq(new Protocol()), actions = Seq(new Protocol())) @@ -167,7 +133,7 @@ class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase { conflicts = true, setup = Seq(metadata_partX, addA_partX2), reads = Seq( - t => t.markFilesAsRead(new EqualTo(schema.column("x"), Literal.of(1))), + t => t.markFilesAsRead(colXEq1Filter), // `readWholeTable` should disallow any concurrent change, even if the change // is disjoint with the earlier filter t => t.readWholeTable() diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala index 4ca8238feec..9c972cf36a4 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteBase.scala @@ -118,8 +118,7 @@ trait OptimisticTransactionSuiteBase extends FunSuite { val conflict = if (conflicts) "should conflict" else "should not conflict" test(s"$name - $conflict") { withTempDir { tempDir => - val log = - DeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath + "/_delta_log")) + val log = DeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath)) // Setup the log initialSetup(log) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala new file mode 100644 index 00000000000..adbe473398b --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala @@ -0,0 +1,50 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.Collections + +import scala.collection.JavaConverters._ + +import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.expressions.{EqualTo, Literal} +import io.delta.standalone.types.{IntegerType, StructField, StructType} + +/** + * By bundling these variables into a trait, we make it easier for other projects (specifically, the + * Delta Standalone / Delta OSS compatibility project) to reuse these variables in concurrent write + * tests. + */ +trait OptimisticTransactionSuiteTestVals { + val addA = new AddFileJ("a", Collections.emptyMap(), 1, 1, true, null, null) + val addB = new AddFileJ("b", Collections.emptyMap(), 1, 1, true, null, null) + + val removeA = RemoveFileJ.builder("a").deletionTimestamp(4L).build() + val removeA_time5 = RemoveFileJ.builder("a").deletionTimestamp(5L).build() + + val addA_partX1 = new AddFileJ("a", Map("x" -> "1").asJava, 1, 1, true, null, null) + val addA_partX2 = new AddFileJ("a", Map("x" -> "2").asJava, 1, 1, true, null, null) + val addB_partX1 = new AddFileJ("b", Map("x" -> "1").asJava, 1, 1, true, null, null) + val addB_partX3 = new AddFileJ("b", Map("x" -> "3").asJava, 1, 1, true, null, null) + val addC_partX4 = new AddFileJ("c", Map("x" -> "4").asJava, 1, 1, true, null, null) + + val schema = new StructType(Array(new StructField("x", new IntegerType()))) + val colXEq1Filter = new EqualTo(schema.column("x"), Literal.of(1)) + val metadata_colX = MetadataJ.builder().schema(schema).build() + val metadata_partX = + MetadataJ.builder().schema(schema).partitionColumns(Seq("x").asJava).build() +} From 4a28a1125c4a3cacfbc3a87ed15a60e8ade9d5d9 Mon Sep 17 00:00:00 2001 From: gurunath Date: Wed, 13 Oct 2021 23:21:00 +0530 Subject: [PATCH 100/291] Moving the Build to Github Actions (#173) Fixes #81 --- .circleci/config.yml | 19 ------------------- .github/workflows/test.yaml | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 33 insertions(+), 19 deletions(-) delete mode 100644 .circleci/config.yml create mode 100644 .github/workflows/test.yaml diff --git a/.circleci/config.yml b/.circleci/config.yml deleted file mode 100644 index 51689c86462..00000000000 --- a/.circleci/config.yml +++ /dev/null @@ -1,19 +0,0 @@ -version: 2 -jobs: - build: - docker: - - image: circleci/openjdk:8u181-jdk # java 8 - steps: - - checkout - - run: - name: Run tests - command: | - build/sbt "++ 2.12.8 standalone/test" - build/sbt "++ 2.12.8 hive/test" - build/sbt "++ 2.12.8 hiveMR/test" - build/sbt "++ 2.12.8 hiveTez/test" - build/sbt "++ 2.12.8 sqlDeltaImport/test" - build/sbt "++ 2.11.12 standalone/test" - build/sbt "++ 2.11.12 hive/test" - build/sbt "++ 2.11.12 hiveMR/test" - build/sbt "++ 2.11.12 hiveTez/test" diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml new file mode 100644 index 00000000000..7066a69ed26 --- /dev/null +++ b/.github/workflows/test.yaml @@ -0,0 +1,33 @@ +name: "Delta Lake Connectors Tests" +on: [push, pull_request] +jobs: + build: + name: "Run tests " + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v2 + - name: install java + uses: actions/setup-java@v2 + with: + distribution: 'zulu' + java-version: '8' + - name: Cache Scala, SBT + uses: actions/cache@v2 + with: + path: | + ~/.sbt + ~/.ivy2 + ~/.cache/coursier + key: delta-sbt-cache + - name: Run Scala/Java and Python tests + shell: bash -l {0} + run: | + build/sbt "++ 2.12.8 standalone/test" + build/sbt "++ 2.12.8 hive/test" + build/sbt "++ 2.12.8 hiveMR/test" + build/sbt "++ 2.12.8 hiveTez/test" + build/sbt "++ 2.12.8 sqlDeltaImport/test" + build/sbt "++ 2.11.12 standalone/test" + build/sbt "++ 2.11.12 hive/test" + build/sbt "++ 2.11.12 hiveMR/test" + build/sbt "++ 2.11.12 hiveTez/test" From ac9bc2fa990adb44f897b7bd55212a1496b1c6af Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 13 Oct 2021 12:15:40 -0700 Subject: [PATCH 101/291] Update the build status to use GitHub Actions and add license badge (#174) --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index c7b5e7e0a37..74248dc20d3 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,7 @@ # Delta Lake Logo Connectors -[![CircleCI](https://circleci.com/gh/delta-io/connectors/tree/master.svg?style=svg)](https://circleci.com/gh/delta-io/connectors/tree/master) +[![Test](https://github.com/delta-io/connectors/actions/workflows/test.yaml/badge.svg)](https://github.com/delta-io/connectors/actions/workflows/test.yaml) +[![License](https://img.shields.io/badge/license-Apache%202-brightgreen.svg)](https://github.com/delta-io/connectors/blob/master/LICENSE.txt) We are building connectors to bring [Delta Lake](https://delta.io) to popular big-data engines outside [Apache Spark](https://spark.apache.org) (e.g., [Apache Hive](https://hive.apache.org/), [Presto](https://prestodb.io/)) and also to common reporting tools like [Microsoft Power BI](https://powerbi.microsoft.com/). From c4ca3a8a8851be1efd833bf8e2f139f84f0fcdd1 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Wed, 13 Oct 2021 22:38:00 -0700 Subject: [PATCH 102/291] Add DeltaConfigs to support table properties stored in Metadata.configuration (#164) --- .../standalone/internal/DeltaConfig.scala | 237 +++++++++++++- .../standalone/internal/DeltaLogImpl.scala | 10 +- .../standalone/internal/MetadataCleanup.scala | 7 +- .../internal/OptimisticTransactionImpl.scala | 28 +- .../internal/exception/DeltaErrors.scala | 8 + .../internal/util/CalendarInterval.scala | 40 +++ .../internal/util/DateTimeConstants.scala | 48 +++ .../internal/util/IntervalUtils.scala | 302 ++++++++++++++++++ .../internal/DeltaConfigSuite.scala | 109 +++++++ .../internal/DeltaRetentionSuite.scala | 21 +- .../internal/DeltaRetentionSuiteBase.scala | 5 +- .../OptimisticTransactionLegacySuite.scala | 10 +- .../internal/OptimisticTransactionSuite.scala | 2 +- 13 files changed, 769 insertions(+), 58 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/CalendarInterval.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/DateTimeConstants.scala create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/IntervalUtils.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/DeltaConfigSuite.scala diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala index 1e6c83ce543..3697159eb49 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala @@ -14,17 +14,240 @@ * limitations under the License. */ - package io.delta.standalone.internal -object DeltaConfig { - // TODO IS_APPEND_ONLY +import java.util.{HashMap, Locale} + +import io.delta.standalone.internal.actions.{Action, Metadata, Protocol} +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.util.{CalendarInterval, IntervalUtils} +import org.apache.hadoop.conf.Configuration + +private[internal] case class DeltaConfig[T]( + key: String, + defaultValue: String, + fromString: String => T, + validationFunction: T => Boolean, + helpMessage: String, + minimumProtocolVersion: Option[Protocol] = None, + editable: Boolean = true) { + /** + * Recover the saved value of this configuration from `Metadata`. + * If undefined, return defaultValue. + */ + def fromMetadata(metadata: Metadata): T = { + // todo: how to fail gracefully for an invalid value (see validate() below) + fromString(metadata.configuration.getOrElse(key, defaultValue)) + } + + /** Validate the setting for this configuration */ + private def validate(value: String): Unit = { + if (!editable) { + throw DeltaErrors.cannotModifyTableProperty(key) + } + val onErrorMessage = s"$key $helpMessage" + try { + require(validationFunction(fromString(value)), onErrorMessage) + } catch { + case e: NumberFormatException => + throw new IllegalArgumentException(onErrorMessage, e) + } + } + + /** + * Validate this configuration and return the key - value pair to save into the metadata. + */ + def apply(value: String): (String, String) = { + validate(value) + key -> value + } +} + +/** + * Contains list of reservoir configs and validation checks. + */ +private[internal] object DeltaConfigs { + + /** + * Convert a string to [[CalendarInterval]]. This method is case-insensitive and will throw + * [[IllegalArgumentException]] when the input string is not a valid interval. + * + * @throws IllegalArgumentException if the string is not a valid internal. + */ + def parseCalendarInterval(s: String): CalendarInterval = { + if (s == null || s.trim.isEmpty) { + throw new IllegalArgumentException("Interval cannot be null or blank.") + } + val sInLowerCase = s.trim.toLowerCase(Locale.ROOT) + val interval = + if (sInLowerCase.startsWith("interval ")) sInLowerCase else "interval " + sInLowerCase + val cal = IntervalUtils.safeStringToInterval(interval) + if (cal == null) { + throw new IllegalArgumentException("Invalid interval: " + s) + } + cal + } + + /** + * A global default value set as a HadoopConf will overwrite the default value of a DeltaConfig. + * For example, user can run: + * hadoopConf.set("spark.databricks.delta.properties.defaults.isAppendOnly", "true") + * This setting will be populated to a Delta table during its creation and overwrites + * the default value of delta.isAppendOnly + * + * We accept these HadoopConfs as strings and only perform validation in DeltaConfig. All the + * DeltaConfigs set in HadoopConf should adopt the same prefix. + */ + val hadoopConfPrefix = "spark.databricks.delta.properties.defaults." + + private val entries = new HashMap[String, DeltaConfig[_]] + + protected def buildConfig[T]( + key: String, + defaultValue: String, + fromString: String => T, + validationFunction: T => Boolean, + helpMessage: String, + minimumProtocolVersion: Option[Protocol] = None, + userConfigurable: Boolean = true): DeltaConfig[T] = { + val deltaConfig = DeltaConfig( + s"delta.$key", + defaultValue, + fromString, + validationFunction, + helpMessage, + minimumProtocolVersion, + userConfigurable) + + entries.put(key.toLowerCase(Locale.ROOT), deltaConfig) + deltaConfig + } + + /** + * Validates specified configurations and returns the normalized key -> value map. + */ + def validateConfigurations(configurations: Map[String, String]): Map[String, String] = { + configurations.map { + case kv @ (key, value) if key.toLowerCase(Locale.ROOT).startsWith("delta.constraints.") => + throw new IllegalArgumentException(s"Unsupported CHECK constraint configuration ${key} set") + case (key, value) if key.toLowerCase(Locale.ROOT).startsWith("delta.") => + Option(entries.get(key.toLowerCase(Locale.ROOT).stripPrefix("delta."))) + .map(_(value)) + .getOrElse { + throw DeltaErrors.unknownConfigurationKeyException(key) + } + case keyvalue @ (key, _) => + if (entries.containsKey(key.toLowerCase(Locale.ROOT))) { + // TODO: add log +// logConsole( +// s""" +// |You are trying to set a property the key of which is the same as Delta config: $key. +// |If you are trying to set a Delta config, prefix it with "delta.", e.g. 'delta.$key'. +// """.stripMargin) + } + keyvalue + } + } + + /** + * Table properties for new tables can be specified through Hadoop configurations. This method + * checks to see if any of the configurations exist among the Hadoop configurations and merges + * them with the user provided configurations. User provided configs take precedence. + */ + def mergeGlobalConfigs( + hadoopConf: Configuration, + tableConf: Map[String, String]): Map[String, String] = { + import collection.JavaConverters._ + + val globalConfs = entries.asScala.flatMap { case (_, config) => + val hadoopConfKey = hadoopConfPrefix + config.key.stripPrefix("delta.") + Option(hadoopConf.get(hadoopConfKey, null)) match { + case Some(default) => Some(config(default)) + case _ => None + } + } + + globalConfs.toMap ++ tableConf + } + + def getMilliSeconds(i: CalendarInterval): Long = { + getMicroSeconds(i) / 1000L + } + + private def getMicroSeconds(i: CalendarInterval): Long = { + assert(i.months == 0) + i.days * util.DateTimeConstants.MICROS_PER_DAY + i.microseconds + } + + /** + * For configs accepting an interval, we require the user specified string must obey: + * + * - Doesn't use months or years, since an internal like this is not deterministic. + * - The microseconds parsed from the string value must be a non-negative value. + * + * The method returns whether a [[CalendarInterval]] satisfies the requirements. + */ + def isValidIntervalConfigValue(i: CalendarInterval): Boolean = { + i.months == 0 && getMicroSeconds(i) >= 0 + } + + /** + * The shortest duration we have to keep delta files around before deleting them. We can only + * delete delta files that are before a compaction. We may keep files beyond this duration until + * the next calendar day. + */ + val LOG_RETENTION = buildConfig[CalendarInterval]( + "logRetentionDuration", + "interval 30 days", + parseCalendarInterval, + isValidIntervalConfigValue, + "needs to be provided as a calendar interval such as '2 weeks'. Months " + + "and years are not accepted. You may specify '365 days' for a year instead.") - // TODO CHECKPOINT_INTERVAL + /** + * The shortest duration we have to keep logically deleted data files around before deleting them + * physically. This is to prevent failures in stale readers after compactions or partition + * overwrites. + * + * Note: this value should be large enough: + * - It should be larger than the longest possible duration of a job if you decide to run "VACUUM" + * when there are concurrent readers or writers accessing the table. + * - If you are running a streaming query reading from the table, you should make sure the query + * doesn't stop longer than this value. Otherwise, the query may not be able to restart as it + * still needs to read old files. + */ + val TOMBSTONE_RETENTION = buildConfig[CalendarInterval]( + "deletedFileRetentionDuration", + "interval 1 week", + parseCalendarInterval, + isValidIntervalConfigValue, + "needs to be provided as a calendar interval such as '2 weeks'. Months " + + "and years are not accepted. You may specify '365 days' for a year instead.") - // TODO: LOG_RETENTION + /** How often to checkpoint the delta log. */ + val CHECKPOINT_INTERVAL = buildConfig[Int]( + "checkpointInterval", + "10", + _.toInt, + _ > 0, + "needs to be a positive integer.") - // TODO: TOMBSTONE_RETENTION + /** Whether to clean up expired checkpoints and delta logs. */ + val ENABLE_EXPIRED_LOG_CLEANUP = buildConfig[Boolean]( + "enableExpiredLogCleanup", + "true", + _.toBoolean, + _ => true, + "needs to be a boolean.") - // TODO: ENABLE_EXPIRED_LOG_CLEANUP + /** + * Whether this Delta table is append-only. Files can't be deleted, or values can't be updated. + */ + val IS_APPEND_ONLY = buildConfig[Boolean]( + "appendOnly", + "false", + _.toBoolean, + _ => true, + "needs to be a boolean.", + Some(new Protocol(0, 2))) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index b98136408f2..e71d0bd119d 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -58,9 +58,7 @@ private[internal] class DeltaLogImpl private( /** How long to keep around logically deleted files before physically deleting them. */ def tombstoneRetentionMillis: Long = - // TODO DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata)) - // 1 week - metadata.configuration.getOrElse("deletedFileRetentionDuration", "604800000").toLong + DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetadata(metadata)) /** * Tombstones before this timestamp will be dropped from the state and the files can be @@ -75,8 +73,7 @@ private[internal] class DeltaLogImpl private( protected lazy val history = DeltaHistoryManager(this) /** Returns the checkpoint interval for this log. Not transactional. */ - // TODO: DeltaConfigs.CHECKPOINT_INTERVAL - def checkpointInterval: Int = metadata.configuration.getOrElse("checkpointInterval", "10").toInt + def checkpointInterval: Int = DeltaConfigs.CHECKPOINT_INTERVAL.fromMetadata(metadata) /** Convert the timeZoneId to an actual timeZone that can be used for decoding. */ def timezone: TimeZone = { @@ -185,8 +182,7 @@ private[internal] class DeltaLogImpl private( * can remove data such as DELETE/UPDATE/MERGE. */ def assertRemovable(): Unit = { - // TODO: DeltaConfig.IS_APPEND_ONLY - if (metadata.configuration.getOrElse("appendOnly", "false").toBoolean) { + if (DeltaConfigs.IS_APPEND_ONLY.fromMetadata(metadata)) { throw DeltaErrors.modifyAppendOnlyTableException } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala index 7fe45bda5e9..e8cac2c7472 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala @@ -30,17 +30,14 @@ private[internal] trait MetadataCleanup { /** Whether to clean up expired log files and checkpoints. */ def enableExpiredLogCleanup: Boolean = - // TODO: DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.fromMetaData(metadata) - metadata.configuration.getOrElse("enableExpiredLogCleanup", "true").toBoolean + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.fromMetadata(metadata) /** * Returns the duration in millis for how long to keep around obsolete logs. We may keep logs * beyond this duration until the next calendar day to avoid constantly creating checkpoints. */ def deltaRetentionMillis: Long = { - // TODO DeltaConfigs.getMilliSeconds(DeltaConfigs.LOG_RETENTION.fromMetaData(metadata)) - // 30 days - metadata.configuration.getOrElse("logRetentionDuration", "2592000000").toLong + DeltaConfigs.getMilliSeconds(DeltaConfigs.LOG_RETENTION.fromMetadata(metadata)) } def doLogCleanup(): Unit = { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 3c119feb7f0..8ba250fcf2c 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -396,32 +396,8 @@ private[internal] class OptimisticTransactionImpl( /** Creates new metadata with global Delta configuration defaults. */ private def withGlobalConfigDefaults(metadata: Metadata): Metadata = { - // TODO DeltaConfigs.mergeGlobalConfigs - - val defaultConfigs = Map( - "deletedFileRetentionDuration" -> "604800000", // 1 week - "checkpointInterval" -> "10", - "enableExpiredLogCleanup" -> "true", - "logRetentionDuration" -> "2592000000", // 30 days - "appendOnly" -> "false" - ) - - // Priority is: - // 1. user-provided configs (via metadata.configuration) - // 2. global hadoop configs - // 3. default configs - val newMetadataConfig = defaultConfigs.keySet.map { key => - val value = if (metadata.configuration.contains(key)) { - metadata.configuration(key) - } else { - deltaLog.hadoopConf.get(key, defaultConfigs(key)) - } - - key -> value - }.toMap - - // User provided configs take precedence. - metadata.copy(configuration = newMetadataConfig) + metadata.copy(configuration = + DeltaConfigs.mergeGlobalConfigs(deltaLog.hadoopConf, metadata.configuration)) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index fd9732f027f..023fc916545 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -273,6 +273,14 @@ private[internal] object DeltaErrors { s"Parsed $nestType type:\n${nested.toPrettyJson}") } + def cannotModifyTableProperty(prop: String): Throwable = + throw new UnsupportedOperationException( + s"The Delta table configuration $prop cannot be specified by the user") + + def unknownConfigurationKeyException(confKey: String): Throwable = { + new IllegalArgumentException(s"Unknown configuration was specified: $confKey") + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/CalendarInterval.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/CalendarInterval.scala new file mode 100644 index 00000000000..45d9f612542 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/CalendarInterval.scala @@ -0,0 +1,40 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +/** + * The class representing calendar intervals. The calendar interval is stored internally in + * three components. + *

+ * + * The `months` and `days` are not units of time with a constant length (unlike hours, seconds), so + * they are two separated fields from microseconds. One month may be equal to 28, 29, 30 or 31 days + * and one day may be equal to 23, 24 or 25 hours (daylight saving). + * + * @param months an integer value representing the number of months in this interval + * @param days an integer value representing the number of days in this interval + * @param microseconds a long value representing the number of microseconds in this interval + */ +private[internal] case class CalendarInterval( + val months: Int, + val days: Int, + val microseconds: Long) + diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/DateTimeConstants.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/DateTimeConstants.scala new file mode 100644 index 00000000000..72656b50d7e --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/DateTimeConstants.scala @@ -0,0 +1,48 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +private[internal] object DateTimeConstants { + + val MONTHS_PER_YEAR = 12 + + val DAYS_PER_WEEK = 7 + + val HOURS_PER_DAY = 24L + + val MINUTES_PER_HOUR = 60L + + val SECONDS_PER_MINUTE = 60L + val SECONDS_PER_HOUR: Long = MINUTES_PER_HOUR * SECONDS_PER_MINUTE + val SECONDS_PER_DAY: Long = HOURS_PER_DAY * SECONDS_PER_HOUR + + val MILLIS_PER_SECOND = 1000L + val MILLIS_PER_MINUTE: Long = SECONDS_PER_MINUTE * MILLIS_PER_SECOND + val MILLIS_PER_HOUR: Long = MINUTES_PER_HOUR * MILLIS_PER_MINUTE + val MILLIS_PER_DAY: Long = HOURS_PER_DAY * MILLIS_PER_HOUR + + val MICROS_PER_MILLIS = 1000L + val MICROS_PER_SECOND: Long = MILLIS_PER_SECOND * MICROS_PER_MILLIS + val MICROS_PER_MINUTE: Long = SECONDS_PER_MINUTE * MICROS_PER_SECOND + val MICROS_PER_HOUR: Long = MINUTES_PER_HOUR * MICROS_PER_MINUTE + val MICROS_PER_DAY: Long = HOURS_PER_DAY * MICROS_PER_HOUR + + val NANOS_PER_MICROS = 1000L + val NANOS_PER_MILLIS: Long = MICROS_PER_MILLIS * NANOS_PER_MICROS + val NANOS_PER_SECOND: Long = MILLIS_PER_SECOND * NANOS_PER_MILLIS + +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/IntervalUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/IntervalUtils.scala new file mode 100644 index 00000000000..8b866e24c53 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/IntervalUtils.scala @@ -0,0 +1,302 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import java.nio.charset.StandardCharsets + +private[internal] object IntervalUtils { + + object IntervalUnit extends Enumeration { + type IntervalUnit = Value + + val NANOSECOND = Value(0, "nanosecond") + val MICROSECOND = Value(1, "microsecond") + val MILLISECOND = Value(2, "millisecond") + val SECOND = Value(3, "second") + val MINUTE = Value(4, "minute") + val HOUR = Value(5, "hour") + val DAY = Value(6, "day") + val WEEK = Value(7, "week") + val MONTH = Value(8, "month") + val YEAR = Value(9, "year") + } + import IntervalUnit._ + + private object ParseState extends Enumeration { + type ParseState = Value + + val PREFIX, + TRIM_BEFORE_SIGN, + SIGN, + TRIM_BEFORE_VALUE, + VALUE, + VALUE_FRACTIONAL_PART, + TRIM_BEFORE_UNIT, + UNIT_BEGIN, + UNIT_SUFFIX, + UNIT_END = Value + } + private final val intervalStr = "interval" + private def unitToUtf8(unit: IntervalUnit): String = { + unit.toString + } + private final val yearStr = unitToUtf8(YEAR) + private final val monthStr = unitToUtf8(MONTH) + private final val weekStr = unitToUtf8(WEEK) + private final val dayStr = unitToUtf8(DAY) + private final val hourStr = unitToUtf8(HOUR) + private final val minuteStr = unitToUtf8(MINUTE) + private final val secondStr = unitToUtf8(SECOND) + private final val millisStr = unitToUtf8(MILLISECOND) + private final val microsStr = unitToUtf8(MICROSECOND) + + /** + * A safe version of `stringToInterval`. It returns null for invalid input string. + */ + def safeStringToInterval(input: String): CalendarInterval = { + try { + stringToInterval(input) + } catch { + case _: IllegalArgumentException => null + } + } + + /** + * Converts a string to [[CalendarInterval]] case-insensitively. + * + * @throws IllegalArgumentException if the input string is not in valid interval format. + */ + def stringToInterval(input: String): CalendarInterval = { + import ParseState._ + def throwIAE(msg: String, e: Exception = null) = { + throw new IllegalArgumentException(s"Error parsing '$input' to interval, $msg", e) + } + + if (input == null) { + throwIAE("interval string cannot be null") + } + // scalastyle:off caselocale .toLowerCase + val s = input.trim().toLowerCase + // scalastyle:on + val bytes = s.getBytes(StandardCharsets.UTF_8) + if (bytes.isEmpty) { + throwIAE("interval string cannot be empty") + } + var state = PREFIX + var i = 0 + var currentValue: Long = 0 + var isNegative: Boolean = false + var months: Int = 0 + var days: Int = 0 + var microseconds: Long = 0 + var fractionScale: Int = 0 + val initialFractionScale = (DateTimeConstants.NANOS_PER_SECOND / 10).toInt + var fraction: Int = 0 + var pointPrefixed: Boolean = false + + def trimToNextState(b: Byte, next: ParseState): Unit = { + if (Character.isWhitespace(b)) { + i += 1 + } else { + state = next + } + } + + def currentWord: String = { + val sep = "\\s+" + val strings = s.split(sep) + val lenRight = s.substring(i, s.length).split(sep) .length + strings(strings.length - lenRight) + } + + def matchAt(i: Int, str: String): Boolean = { + if (i + str.length > s.length) { + false + } else { + s.substring(i, i + str.length) == str + } + } + + while (i < bytes.length) { + val b = bytes(i) + state match { + case PREFIX => + if (s.startsWith(intervalStr)) { + if (s.length == + intervalStr.length) { + throwIAE("interval string cannot be empty") + } else if (!Character.isWhitespace( + bytes(i + intervalStr.length))) { + throwIAE(s"invalid interval prefix $currentWord") + } else { + i += intervalStr.length + 1 + } + } + state = TRIM_BEFORE_SIGN + case TRIM_BEFORE_SIGN => trimToNextState(b, SIGN) + case SIGN => + currentValue = 0 + fraction = 0 + // We preset next state from SIGN to TRIM_BEFORE_VALUE. If we meet '.' in the SIGN state, + // it means that the interval value we deal with here is a numeric with only fractional + // part, such as '.11 second', which can be parsed to 0.11 seconds. In this case, we need + // to reset next state to `VALUE_FRACTIONAL_PART` to go parse the fraction part of the + // interval value. + state = TRIM_BEFORE_VALUE + // We preset the scale to an invalid value to track fraction presence in the UNIT_BEGIN + // state. If we meet '.', the scale become valid for the VALUE_FRACTIONAL_PART state. + fractionScale = -1 + pointPrefixed = false + b match { + case '-' => + isNegative = true + i += 1 + case '+' => + isNegative = false + i += 1 + case _ if '0' <= b && b <= '9' => + isNegative = false + case '.' => + isNegative = false + fractionScale = initialFractionScale + pointPrefixed = true + i += 1 + state = VALUE_FRACTIONAL_PART + case _ => throwIAE( s"unrecognized number '$currentWord'") + } + case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) + case VALUE => + b match { + case _ if '0' <= b && b <= '9' => + try { + currentValue = Math.addExact(Math.multiplyExact(10, currentValue), (b - '0')) + } catch { + case e: ArithmeticException => throwIAE(e.getMessage, e) + } + case _ if Character.isWhitespace(b) => state = TRIM_BEFORE_UNIT + case '.' => + fractionScale = initialFractionScale + state = VALUE_FRACTIONAL_PART + case _ => throwIAE(s"invalid value '$currentWord'") + } + i += 1 + case VALUE_FRACTIONAL_PART => + if ('0' <= b && b <= '9' && fractionScale > 0) { + fraction += (b - '0') * fractionScale + fractionScale /= 10 + } else if (Character.isWhitespace(b) && + (!pointPrefixed || fractionScale < initialFractionScale)) { + fraction /= DateTimeConstants.NANOS_PER_MICROS.toInt + state = TRIM_BEFORE_UNIT + } else if ('0' <= b && b <= '9') { + throwIAE(s"interval can only support nanosecond precision, '$currentWord' is out" + + s" of range") + } else { + throwIAE(s"invalid value '$currentWord'") + } + i += 1 + case TRIM_BEFORE_UNIT => trimToNextState(b, UNIT_BEGIN) + case UNIT_BEGIN => + // Checks that only seconds can have the fractional part + if (b != 's' && fractionScale >= 0) { + throwIAE(s"'$currentWord' cannot have fractional part") + } + if (isNegative) { + currentValue = -currentValue + fraction = -fraction + } + try { + b match { + case 'y' if matchAt(i, yearStr) => + val monthsInYears = Math.multiplyExact( + DateTimeConstants.MONTHS_PER_YEAR, + currentValue) + months = Math.toIntExact(Math.addExact(months, monthsInYears)) + i += yearStr.length + case 'w' if matchAt(i, weekStr) => + val daysInWeeks = Math.multiplyExact(DateTimeConstants.DAYS_PER_WEEK, currentValue) + days = Math.toIntExact(Math.addExact(days, daysInWeeks)) + i += weekStr.length + case 'd' if matchAt(i, dayStr) => + days = Math.addExact(days, Math.toIntExact(currentValue)) + i += dayStr.length + case 'h' if matchAt(i, hourStr) => + val hoursUs = Math.multiplyExact(currentValue, DateTimeConstants.MICROS_PER_HOUR) + microseconds = Math.addExact(microseconds, hoursUs) + i += hourStr.length + case 's' if matchAt(i, secondStr) => + val secondsUs = Math.multiplyExact( + currentValue, + DateTimeConstants.MICROS_PER_SECOND) + microseconds = Math.addExact(Math.addExact(microseconds, secondsUs), fraction) + i += secondStr.length + case 'm' => + if (matchAt(i, monthStr)) { + months = Math.addExact(months, Math.toIntExact(currentValue)) + i += monthStr.length + } else if (matchAt(i, minuteStr)) { + val minutesUs = Math.multiplyExact( + currentValue, + DateTimeConstants.MICROS_PER_MINUTE) + microseconds = Math.addExact(microseconds, minutesUs) + i += minuteStr.length + } else if (matchAt(i, millisStr)) { + val millisUs = Math.multiplyExact( + currentValue, + DateTimeConstants.MICROS_PER_MILLIS) + microseconds = Math.addExact(microseconds, millisUs) + i += millisStr.length + } else if (matchAt(i, microsStr)) { + microseconds = Math.addExact(microseconds, currentValue) + i += microsStr.length + } else throwIAE(s"invalid unit '$currentWord'") + case _ => throwIAE(s"invalid unit '$currentWord'") + } + } catch { + case e: ArithmeticException => throwIAE(e.getMessage, e) + } + state = UNIT_SUFFIX + case UNIT_SUFFIX => + b match { + case 's' => state = UNIT_END + case _ if Character.isWhitespace(b) => state = TRIM_BEFORE_SIGN + case _ => throwIAE(s"invalid unit '$currentWord'") + } + i += 1 + case UNIT_END => + if (Character.isWhitespace(b) ) { + i += 1 + state = TRIM_BEFORE_SIGN + } else { + throwIAE(s"invalid unit '$currentWord'") + } + } + } + + val result = state match { + case UNIT_SUFFIX | UNIT_END | TRIM_BEFORE_SIGN => + new CalendarInterval(months, days, microseconds) + case TRIM_BEFORE_VALUE => throwIAE(s"expect a number after '$currentWord' but hit EOL") + case VALUE | VALUE_FRACTIONAL_PART => + throwIAE(s"expect a unit name after '$currentWord' but hit EOL") + case _ => throwIAE(s"unknown error when parsing '$currentWord'") + } + + result + } +} + diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaConfigSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaConfigSuite.scala new file mode 100644 index 00000000000..b06852440d5 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaConfigSuite.scala @@ -0,0 +1,109 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.concurrent.TimeUnit + +import org.apache.hadoop.conf.Configuration +import io.delta.standalone.internal.actions.Metadata +import io.delta.standalone.internal.DeltaConfigs.{isValidIntervalConfigValue, parseCalendarInterval} +import io.delta.standalone.internal.util.{CalendarInterval, DateTimeConstants} + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class DeltaConfigSuite extends FunSuite { + + test("mergeGlobalConfigs") { + + val hadoopConf = new Configuration() + hadoopConf.set( + DeltaConfigs.hadoopConfPrefix + DeltaConfigs.IS_APPEND_ONLY.key.stripPrefix("delta."), + "true") + hadoopConf.set( + DeltaConfigs.hadoopConfPrefix + + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key.stripPrefix("delta."), + "true") + val metadataConf = Map(DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key -> "false", + DeltaConfigs.CHECKPOINT_INTERVAL.key -> "1 day") + val mergedConf = DeltaConfigs.mergeGlobalConfigs(hadoopConf, metadataConf) + assert(mergedConf.get(DeltaConfigs.IS_APPEND_ONLY.key) == Some("true")) + assert(mergedConf.get(DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key) == Some("false")) + assert(mergedConf.get(DeltaConfigs.CHECKPOINT_INTERVAL.key) == Some("1 day")) + assert(!mergedConf.contains("delta.deletedFileRetentionDuration")) // we didn't add other keys + } + + test("check DeltaConfig defaults") { + val emptyMetadata = new Metadata() + assert( + DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetadata(emptyMetadata)) == + DateTimeConstants.MILLIS_PER_DAY*DateTimeConstants.DAYS_PER_WEEK) // default is 1 week + + assert(DeltaConfigs.getMilliSeconds(DeltaConfigs.LOG_RETENTION.fromMetadata(emptyMetadata)) == + DateTimeConstants.MILLIS_PER_DAY*30) // default is 30 days + + assert(DeltaConfigs.CHECKPOINT_INTERVAL.fromMetadata(emptyMetadata) == 10) + + assert(DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.fromMetadata(emptyMetadata)) + + assert(!DeltaConfigs.IS_APPEND_ONLY.fromMetadata(emptyMetadata)) + } + + test("parseCalendarInterval") { + for (input <- Seq("5 MINUTES", "5 minutes", "5 Minutes", "inTERval 5 minutes")) { + assert(parseCalendarInterval(input) === + new CalendarInterval(0, 0, TimeUnit.MINUTES.toMicros(5))) + } + + for (input <- Seq(null, "", " ")) { + val e = intercept[IllegalArgumentException] { + parseCalendarInterval(input) + } + assert(e.getMessage.contains("cannot be null or blank")) + } + + for (input <- Seq("interval", "interval1 day", "foo", "foo 1 day")) { + val e = intercept[IllegalArgumentException] { + parseCalendarInterval(input) + } + assert(e.getMessage.contains("Invalid interval")) + } + } + + test("isValidIntervalConfigValue") { + for (input <- Seq( + // Allow 0 microsecond because we always convert microseconds to milliseconds so 0 + // microsecond is the same as 100 microseconds. + "0 microsecond", + "1 microsecond", + "1 millisecond", + "1 day", + "-1 day 86400001 milliseconds", // This is 1 millisecond + "1 day -1 microseconds")) { + assert(isValidIntervalConfigValue(parseCalendarInterval(input))) + } + for (input <- Seq( + "-1 microseconds", + "-1 millisecond", + "-1 day", + "1 day -86400001 milliseconds", // This is -1 millisecond + "1 month", + "1 year")) { + assert(!isValidIntervalConfigValue(parseCalendarInterval(input)), s"$input") + } + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala index e0381e47667..9e2d3838d60 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuite.scala @@ -58,9 +58,10 @@ class DeltaRetentionSuite extends DeltaRetentionSuiteBase { assert(initialFiles === getLogFiles(logPath)) - // TODO clock.advance(intervalStringToMillis(DeltaConfigs.LOG_RETENTION.defaultValue) + - // intervalStringToMillis("interval 1 day")) - clock.advance(log.deltaRetentionMillis + 1000*60*60*24) // 1 day + clock.advance( + DeltaConfigs.getMilliSeconds( + DeltaConfigs.parseCalendarInterval(DeltaConfigs.LOG_RETENTION.defaultValue) + ) + util.DateTimeConstants.MILLIS_PER_DAY) // + 1 day // Shouldn't clean up, no checkpoint, although all files have expired log.cleanUpExpiredLogs() @@ -135,12 +136,16 @@ class DeltaRetentionSuite extends DeltaRetentionSuiteBase { withTempDir { tempDir => val log = DeltaLogImpl.forTable(hadoopConf, tempDir.getCanonicalPath) log.startTransaction().commit( - Metadata(configuration = Map("enableExpiredLogCleanup" -> "true")) :: Nil, + Metadata(configuration = Map( + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key ->"true" + )) :: Nil, manualUpdate, writerId) assert(log.enableExpiredLogCleanup) log.startTransaction().commit( - Metadata(configuration = Map("enableExpiredLogCleanup" -> "false")) :: Nil, + Metadata(configuration = Map( + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key -> "false" + )) :: Nil, manualUpdate, writerId) assert(!log.enableExpiredLogCleanup) @@ -181,8 +186,10 @@ class DeltaRetentionSuite extends DeltaRetentionSuiteBase { val files2 = (1 to 4).map(f => RemoveFile(f.toString, Some(clock.getTimeMillis()))) txn2.commit(files2, manualUpdate, writerId) - // TODO: intervalStringToMillis(DeltaConfigs.TOMBSTONE_RETENTION.defaultValue) + 1000000L) - clock.advance(log1.tombstoneRetentionMillis + 1000000L) + clock.advance( + DeltaConfigs.getMilliSeconds( + DeltaConfigs.parseCalendarInterval(DeltaConfigs.LOG_RETENTION.defaultValue) + ) + 1000000L) log1.checkpoint() diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala index bd4ed41e573..7fb0cf3f61e 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaRetentionSuiteBase.scala @@ -33,7 +33,10 @@ trait DeltaRetentionSuiteBase extends FunSuite { protected def hadoopConf: Configuration = { val conf = new Configuration() - conf.set("enableExpiredLogCleanup", "false") + conf.set( + DeltaConfigs.hadoopConfPrefix + + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.key.stripPrefix("delta."), + "false") conf } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 38b2225131b..5509890ccd6 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -247,7 +247,7 @@ class OptimisticTransactionLegacySuite extends FunSuite { test("Removing from an append-only table") { withTempDir { dir => val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - val metadata = Metadata(configuration = Map("appendOnly" -> "true")) + val metadata = Metadata(configuration = Map(DeltaConfigs.IS_APPEND_ONLY.key -> "true")) log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) val removeWithDataChange = addA_P1.remove.copy(dataChange = true) @@ -454,12 +454,14 @@ class OptimisticTransactionLegacySuite extends FunSuite { } test("updateMetadata withGlobalConfigDefaults") { - // TODO: use DeltaConfigs... withTempDir { dir => // note that the default for logRetentionDuration is 2592000000 val hadoopConf = new Configuration() - hadoopConf.set("logRetentionDuration", "1000") - val metadata = Metadata(configuration = Map("logRetentionDuration" -> "2000")) + hadoopConf.set( + DeltaConfigs.hadoopConfPrefix + DeltaConfigs.LOG_RETENTION.key.stripPrefix("delta."), + "1000 milliseconds") + val metadata = Metadata( + configuration = Map(DeltaConfigs.LOG_RETENTION.key -> "2000 millisecond")) val log = DeltaLogImpl.forTable(hadoopConf, dir.getCanonicalPath) log.startTransaction().commit(metadata :: Nil, manualUpdate, engineInfo) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 3c9ebdcdcee..88a2e44f72d 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase - with OptimisticTransactionSuiteTestVals { + with OptimisticTransactionSuiteTestVals { /////////////////////////////////////////////////////////////////////////// // Allowed concurrent actions From 17c549805e7b600dafd8ff44acf3d86d7a9a45b3 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 15 Oct 2021 08:58:45 -0700 Subject: [PATCH 103/291] [DSW] [21] Misc Fixes 1 (#172) * added comments for future work on operation json encoded values * added commitInfo engineInfo name & version string * remove a todo; add reader, writer version to public java Action interface * remove a todo; throw proper exception in Checkpoints.checkpoint * remove a todo; use schema.toJson instead of schemaStr in OptTxnSuite * remove old comments; minor style change; add extra oss compatibility comparison utils * add tests to DeltaLogSuite * tidy up conversionUtils * refactor action builders to their own suite * cleanup with conversion utils; add conversion utils test * update engineInfo format (slash instead of dash); rename BuildInfo to package object --- build.sbt | 15 +- .../internal/util/ComparisonUtil.scala | 23 +- .../standalone/OptimisticTransaction.java | 4 +- .../io/delta/standalone/actions/Action.java | 7 +- .../delta/standalone/actions/AddCDCFile.java | 8 +- .../io/delta/standalone/actions/AddFile.java | 4 +- .../delta/standalone/actions/CommitInfo.java | 2 +- .../io/delta/standalone/actions/Format.java | 3 +- .../io/delta/standalone/actions/JobInfo.java | 3 +- .../io/delta/standalone/actions/Metadata.java | 4 +- .../standalone/actions/NotebookInfo.java | 3 +- .../io/delta/standalone/actions/Protocol.java | 15 +- .../delta/standalone/actions/RemoveFile.java | 4 +- .../standalone/internal/Checkpoints.scala | 16 +- .../internal/OptimisticTransactionImpl.scala | 17 +- .../internal/exception/DeltaErrors.scala | 8 +- .../internal/util/ConversionUtils.scala | 77 +++-- .../internal/ActionBuildersSuite.scala | 213 ++++++++++++++ .../internal/ConversionUtilsSuite.scala | 52 +++- .../standalone/internal/DeltaLogSuite.scala | 270 +++++------------- .../OptimisticTransactionLegacySuite.scala | 33 +-- .../internal/OptimisticTransactionSuite.scala | 2 +- 22 files changed, 494 insertions(+), 289 deletions(-) create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala diff --git a/build.sbt b/build.sbt index 2e78636da16..0d8cdc58b06 100644 --- a/build.sbt +++ b/build.sbt @@ -264,7 +264,20 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("com.fasterxml.jackson.module") ), "org.scalatest" %% "scalatest" % "3.0.5" % "test" - )) + ), + sourceGenerators in Compile += Def.task { + val file = (sourceManaged in Compile).value / "meta" / "package.scala" + IO.write(file, + s"""package io.delta + | + |package object standalone { + | val VERSION = "${version.value}" + | val NAME = "Delta Standalone" + |} + |""".stripMargin) + Seq(file) + } + ) /** * Unidoc settings diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala index cbd7bea85be..8c540c343fd 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala @@ -70,8 +70,17 @@ trait ComparisonUtil { compareOptions(standalone.getUserName, oss.userName) assert(standalone.getOperation == oss.operation) compareNullableMaps(standalone.getOperationParameters, oss.operationParameters) - // TODO: job - // TODO: notebook + + assert(standalone.getJobInfo.isPresent == oss.job.isDefined) + if (standalone.getJobInfo.isPresent) { + compareJobInfo(standalone.getJobInfo.get, oss.job.get) + } + + assert(standalone.getNotebookInfo.isPresent == oss.notebook.isDefined) + if (standalone.getNotebookInfo.isPresent) { + assert(standalone.getNotebookInfo.get.getNotebookId == oss.notebook.get.notebookId) + } + compareOptions(standalone.getClusterId, oss.clusterId) compareOptions(standalone.getReadVersion, oss.readVersion) compareOptions(standalone.getIsolationLevel, oss.isolationLevel) @@ -149,4 +158,14 @@ trait ComparisonUtil { assert(standalone.getVerion == oss.version) compareOptions(standalone.getLastUpdated, oss.lastUpdated) } + + def compareJobInfo( + standalone: io.delta.standalone.actions.JobInfo, + oss: org.apache.spark.sql.delta.actions.JobInfo): Unit = { + assert(standalone.getJobId == oss.jobId) + assert(standalone.getJobName == oss.jobName) + assert(standalone.getRunId == oss.runId) + assert(standalone.getJobOwnerId == oss.jobOwnerId) + assert(standalone.getTriggerType == oss.triggerType) + } } diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 6a312242324..6362a59083e 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -17,7 +17,7 @@ public interface OptimisticTransaction { * @param actions Set of actions to commit. * @param op Details of operation that is performing this transactional commit. * @param engineInfo String used to identify the writer engine. It should resemble - * "{engineName}-{engineVersion}". + * "{engineName}/{engineVersion}". * @return a {@link CommitResult}, wrapping the table version that was committed. */ CommitResult commit(Iterable actions, Operation op, String engineInfo); @@ -63,7 +63,7 @@ public interface OptimisticTransaction { void readWholeTable(); /** - * @param id TODO + * @param id transaction id * @return the latest version that has committed for the idempotent transaction with given `id`. */ long txnVersion(String id); diff --git a/standalone/src/main/java/io/delta/standalone/actions/Action.java b/standalone/src/main/java/io/delta/standalone/actions/Action.java index d1e427bcc9d..e7378d944e7 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Action.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Action.java @@ -18,4 +18,9 @@ * }); * } */ -public interface Action { } +public interface Action { + + /** The maximum version of the protocol that this version of Delta Standalone understands. */ + int readerVersion = 1; + int writerVersion = 2; +} diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java index 5016237e501..9d83e494d13 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java @@ -1,5 +1,6 @@ package io.delta.standalone.actions; +import java.util.Collections; import java.util.Map; public class AddCDCFile implements FileAction { @@ -8,7 +9,8 @@ public class AddCDCFile implements FileAction { private final long size; private final Map tags; - public AddCDCFile(String path, Map partitionValues, long size, Map tags) { + public AddCDCFile(String path, Map partitionValues, long size, + Map tags) { this.path = path; this.partitionValues = partitionValues; this.size = size; @@ -21,7 +23,7 @@ public String getPath() { } public Map getPartitionValues() { - return partitionValues; + return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; } public long getSize() { @@ -29,7 +31,7 @@ public long getSize() { } public Map getTags() { - return tags; + return tags != null ? Collections.unmodifiableMap(tags) : null; } @Override diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java index 03322135f6c..95c936ef2ab 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.util.Collections; @@ -146,7 +147,8 @@ public static class Builder { private final long size; private final long modificationTime; private final boolean dataChange; - // optional AddFile fields + + // optional AddFile fields private String stats; private Map tags; diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index ddc09db40af..33aac13b091 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -136,7 +136,7 @@ public String getOperation() { * @return any relevant operation parameters. e.g. "mode", "partitionBy" */ public Map getOperationParameters() { - return null == operationParameters ? null : Collections.unmodifiableMap(operationParameters); + return operationParameters != null ? Collections.unmodifiableMap(operationParameters) : null; } /** diff --git a/standalone/src/main/java/io/delta/standalone/actions/Format.java b/standalone/src/main/java/io/delta/standalone/actions/Format.java index 6cb56478930..8e5f2da0e77 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Format.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Format.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.util.Collections; @@ -24,7 +25,7 @@ * * @see Delta Transaction Log Protocol */ -public final class Format implements Action { +public final class Format { private final String provider; private final Map options; diff --git a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java index 2dbe4b8f0a4..daa83a65623 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java @@ -13,12 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.util.Objects; /** Represents the Databricks Job information that committed to the Delta table. */ -public class JobInfo implements Action { +public class JobInfo { private final String jobId; private final String jobName; private final String runId; diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index b5718aafeda..52fabe01cfd 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -86,7 +86,7 @@ public Format getFormat() { * columns by which the data should be partitioned */ public List getPartitionColumns() { - return Collections.unmodifiableList(partitionColumns); + return partitionColumns != null ? Collections.unmodifiableList(partitionColumns) : null; } /** @@ -94,7 +94,7 @@ public List getPartitionColumns() { * options for this metadata */ public Map getConfiguration() { - return Collections.unmodifiableMap(configuration); + return configuration != null ? Collections.unmodifiableMap(configuration) : null; } /** diff --git a/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java b/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java index 4d1962ad6b4..cce3b54af95 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/NotebookInfo.java @@ -13,12 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone.actions; import java.util.Objects; /** Represents the Databricks Notebook information that committed to the Delta table. */ -public class NotebookInfo implements Action { +public class NotebookInfo { private final String notebookId; public NotebookInfo(String notebookId) { diff --git a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java index cf6997c9b46..1217115248a 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java @@ -18,14 +18,23 @@ import java.util.Objects; +/** + * Used to block older clients from reading or writing the log when backwards + * incompatible changes are made to the protocol. Readers and writers are + * responsible for checking that they meet the minimum versions before performing + * any other operations. + * + * Since this action allows us to explicitly block older clients in the case of a + * breaking change to the protocol, clients should be tolerant of messages and + * fields that they do not understand. + */ public class Protocol implements Action { private final int minReaderVersion; private final int minWriterVersion; public Protocol() { - // TODO: have these statically and publicly defined - this.minReaderVersion = 1; - this.minWriterVersion = 2; + this.minReaderVersion = Action.readerVersion; + this.minWriterVersion = Action.writerVersion; } public Protocol(int minReaderVersion, int minWriterVersion) { diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java index 63b7367a6cb..27f8bd64e65 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java @@ -45,7 +45,7 @@ public boolean isExtendedFileMetadata() { } public Map getPartitionValues() { - return Collections.unmodifiableMap(partitionValues); + return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; } public long getSize() { @@ -53,7 +53,7 @@ public long getSize() { } public Map getTags() { - return Collections.unmodifiableMap(tags); + return tags != null ? Collections.unmodifiableMap(tags) : null; } @Override diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 79f32da755d..6ff3b2b993a 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -26,10 +26,10 @@ import io.delta.standalone.data.CloseableIterator import io.delta.standalone.internal.actions.SingleAction import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.internal.util.FileNames._ - import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.metadata.CompressionCodecName import com.github.mjakubowski84.parquet4s.ParquetWriter +import io.delta.standalone.internal.exception.DeltaErrors /** * Records information about a checkpoint. @@ -118,7 +118,7 @@ private[internal] trait Checkpoints { */ def checkpoint(snapshotToCheckpoint: SnapshotImpl): Unit = { if (snapshotToCheckpoint.version < 0) { - // TODO throw DeltaErrors.checkpointNonExistTable(dataPath) + throw DeltaErrors.checkpointNonExistTable(dataPath) } val checkpointMetaData = Checkpoints.writeCheckpoint(this, snapshotToCheckpoint) val json = JsonUtils.toJson(checkpointMetaData) @@ -228,11 +228,13 @@ private[internal] object Checkpoints { // Use the string in the closure as Path is not Serializable. val path = checkpointFileSingular(snapshot.path, snapshot.version).toString - val actions: Seq[SingleAction] = ( - Seq(snapshot.metadataScala, snapshot.protocolScala) ++ - snapshot.setTransactionsScala ++ - snapshot.allFilesScala ++ - snapshot.tombstonesScala).map(_.wrap) + // Exclude commitInfo, CDC + val actions: Seq[SingleAction] = ( + Seq(snapshot.metadataScala, snapshot.protocolScala) ++ + snapshot.setTransactionsScala ++ + snapshot.allFilesScala ++ + snapshot.tombstonesScala + ).map(_.wrap) val writtenPath = if (useRename) { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 8ba250fcf2c..ff6a9153cce 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -21,13 +21,12 @@ import java.nio.file.FileAlreadyExistsException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction} -import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ} +import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction, NAME, VERSION} +import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ} import io.delta.standalone.expressions.{Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors -import io.delta.standalone.internal.scan.FilteredDeltaScanImpl -import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils, SchemaMergingUtils, SchemaUtils} +import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMergingUtils, SchemaUtils} private[internal] class OptimisticTransactionImpl( deltaLog: DeltaLogImpl, @@ -115,14 +114,14 @@ private[internal] class OptimisticTransactionImpl( val commitInfo = CommitInfo( deltaLog.clock.getTimeMillis(), op.getName.toString, - null, + null, // TODO: use operation jsonEncodedValues Map.empty, Some(readVersion).filter(_ >= 0), Option(isolationLevelToUse.toString), Some(isBlindAppend), Some(op.getOperationMetrics.asScala.toMap), if (op.getUserMetadata.isPresent) Some(op.getUserMetadata.get()) else None, - Some(engineInfo) // TODO: engineInfo-standalone-standaloneVersion + Some(s"$engineInfo $NAME/$VERSION") ) preparedActions = commitInfo +: preparedActions @@ -404,7 +403,7 @@ private[internal] class OptimisticTransactionImpl( private[internal] object OptimisticTransactionImpl { val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 - def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { - op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap - } +// def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { +// op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap +// } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 023fc916545..9c85d901a47 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -273,9 +273,15 @@ private[internal] object DeltaErrors { s"Parsed $nestType type:\n${nested.toPrettyJson}") } - def cannotModifyTableProperty(prop: String): Throwable = + def checkpointNonExistTable(path: Path): Throwable = { + new IllegalStateException(s"Cannot checkpoint a non-exist table $path. Did you manually " + + s"delete files in the _delta_log directory?") + } + + def cannotModifyTableProperty(prop: String): Throwable = { throw new UnsupportedOperationException( s"The Delta table configuration $prop cannot be specified by the user") + } def unknownConfigurationKeyException(confKey: String): Throwable = { new IllegalArgumentException(s"Unknown configuration was specified: $confKey") diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index e1821fd9837..54985ef5e1e 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -189,10 +189,7 @@ private[internal] object ConversionUtils { case x: AddCDCFile => convertAddCDCFile(x) case x: RemoveFile => convertRemoveFile(x) case x: CommitInfo => convertCommitInfo(x) - case x: Format => convertFormat(x) - case x: JobInfo => convertJobInfo(x) case x: Metadata => convertMetadata(x) - case x: NotebookInfo => convertNotebookInfo(x) case x: SetTransaction => convertSetTransaction(x) case x: Protocol => convertProtocol(x) } @@ -201,31 +198,24 @@ private[internal] object ConversionUtils { // Java to Scala conversions /////////////////////////////////////////////////////////////////////////// -// private implicit def toScalaLongOption(opt: OptionalJ[java.lang.Long]): Option[Long] = -// if (opt.isPresent) Some(opt.get()) else None -// -// private implicit def toScalaStringOption(opt: OptionalJ[StringJ]): Option[String] = -// if (opt.isPresent) Some(opt.get()) else None - - // TODO verify this actually works private implicit def toScalaOption[J, S](opt: OptionalJ[J]): Option[S] = if (opt.isPresent) Some(opt.get().asInstanceOf[S]) else None def convertActionJ(external: ActionJ): Action = external match { case x: AddFileJ => convertAddFileJ(x) + case x: AddCDCFileJ => convertAddCDCFileJ(x) case x: RemoveFileJ => convertRemoveFileJ(x) case x: CommitInfoJ => convertCommitInfoJ(x) case x: MetadataJ => convertMetadataJ(x) - case x: ProtocolJ => convertProtocolJ(x) case x: SetTransactionJ => convertSetTransactionJ(x) - // TODO others + case x: ProtocolJ => convertProtocolJ(x) case _ => throw new UnsupportedOperationException("cannot convert this Java Action") } def convertAddFileJ(external: AddFileJ): AddFile = { AddFile( external.getPath, - external.getPartitionValues.asScala.toMap, + if (external.getPartitionValues == null) null else external.getPartitionValues.asScala.toMap, external.getSize, external.getModificationTime, external.isDataChange, @@ -234,10 +224,19 @@ private[internal] object ConversionUtils { ) } + def convertAddCDCFileJ(external: AddCDCFileJ): AddCDCFile = { + AddCDCFile( + external.getPath, + if (external.getPartitionValues == null) null else external.getPartitionValues.asScala.toMap, + external.getSize, + if (external.getTags == null) null else external.getTags.asScala.toMap + ) + } + def convertRemoveFileJ(external: RemoveFileJ): RemoveFile = { RemoveFile( external.getPath, - external.getDeletionTimestamp, // implicit check this! + external.getDeletionTimestamp, external.isDataChange, external.isExtendedFileMetadata, if (external.isExtendedFileMetadata && external.getPartitionValues != null) { @@ -252,23 +251,29 @@ private[internal] object ConversionUtils { def convertCommitInfoJ(external: CommitInfoJ): CommitInfo = { CommitInfo( - external.getVersion, // implicit check this! + external.getVersion, external.getTimestamp, - external.getUserId, // implicit check this! - external.getUserName, // implicit check this! + external.getUserId, + external.getUserName, external.getOperation, if (external.getOperationParameters != null) { external.getOperationParameters.asScala.toMap } else null, - None, // TODO: Option[JobInfo] - None, // TODO: Option[NotebookInfo] - external.getClusterId, // implicit check this! - external.getReadVersion, // implicit check this! - external.getIsolationLevel, // implicit check this! - external.getIsBlindAppend, // implicit check this! - external.getOperationMetrics, // implicit check this! - external.getUserMetadata, // implicit check this! - external.getEngineInfo // implicit check this! + if (external.getJobInfo.isDefined) { + Some(convertJobInfoJ(external.getJobInfo.get())) + } else None, + if (external.getNotebookInfo.isDefined) { + Some(convertNotebookInfoJ(external.getNotebookInfo.get())) + } else None, + external.getClusterId, + external.getReadVersion, + external.getIsolationLevel, + external.getIsBlindAppend, + if (external.getOperationMetrics.isDefined) { + Some(external.getOperationMetrics.get.asScala.toMap) + } else None, + external.getUserMetadata, + external.getEngineInfo ) } @@ -280,8 +285,8 @@ private[internal] object ConversionUtils { convertFormatJ(external.getFormat), if (external.getSchema == null) null else external.getSchema.toJson, external.getPartitionColumns.asScala, - external.getConfiguration.asScala.toMap, - external.getCreatedTime // implicit check this! + if (external.getConfiguration == null) null else external.getConfiguration.asScala.toMap, + external.getCreatedTime ) } @@ -303,8 +308,22 @@ private[internal] object ConversionUtils { SetTransaction( external.getAppId, external.getVerion, - external.getLastUpdated // implicit check this! + external.getLastUpdated + ) + } + + def convertJobInfoJ(external: JobInfoJ): JobInfo = { + JobInfo( + external.getJobId, + external.getJobName, + external.getRunId, + external.getJobOwnerId, + external.getTriggerType ) } + def convertNotebookInfoJ(external: NotebookInfoJ): NotebookInfo = { + NotebookInfo(external.getNotebookId) + } + } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala new file mode 100644 index 00000000000..42489f8c188 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala @@ -0,0 +1,213 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.sql.Timestamp +import java.util.{Collections, Optional} + +import scala.collection.JavaConverters._ + +import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.types.{IntegerType, StructField => StructFieldJ, StructType => StructTypeJ} + +import org.scalatest.FunSuite + +class ActionBuildersSuite extends FunSuite { + test("builder action class constructor for Metadata") { + val metadataFromBuilderDefaults = MetadataJ.builder().build() + val metadataFromConstructorDefaults = new MetadataJ( + metadataFromBuilderDefaults.getId(), + null, + null, + new FormatJ("parquet", Collections.emptyMap()), + Collections.emptyList(), + Collections.emptyMap(), + metadataFromBuilderDefaults.getCreatedTime(), + null); + assert(metadataFromBuilderDefaults == metadataFromConstructorDefaults) + + val metadataFromBuilder = MetadataJ.builder() + .id("test_id") + .name("test_name") + .description("test_description") + .format(new FormatJ("csv", Collections.emptyMap())) + .partitionColumns(List("id", "name").asJava) + .configuration(Map("test"->"foo").asJava) + .createdTime(0L) + .schema(new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) + .build() + val metadataFromConstructor = new MetadataJ( + "test_id", + "test_name", + "test_description", + new FormatJ("csv", Collections.emptyMap()), + List("id", "name").asJava, + Map("test"->"foo").asJava, + Optional.of(0L), + new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) + assert(metadataFromBuilder == metadataFromConstructor) + } + + test("builder action class constructor for AddFile") { + val addFileFromBuilderDefaults = AddFileJ.builder( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true).build() + val addFileFromConstructorDefaults = new AddFileJ( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true, + null, + null) + assert(addFileFromBuilderDefaults == addFileFromConstructorDefaults) + + val addFileFromBuilder = AddFileJ.builder( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true) + .stats("test_stats") + .tags(Map("test"->"foo").asJava) + .build() + val addFileFromConstructor = new AddFileJ( + "/test", + Collections.emptyMap(), + 0L, + 0L, + true, + "test_stats", + Map("test"->"foo").asJava) + assert(addFileFromBuilder == addFileFromConstructor) + } + + test("builder action class constructor for JobInfo") { + val jobInfoFromBuilderDefaults = JobInfoJ.builder("test").build() + val jobInfoFromConstructorDefaults = new JobInfoJ( + "test", + null, + null, + null, + null) + assert(jobInfoFromBuilderDefaults == jobInfoFromConstructorDefaults) + + val jobInfoFromBuilder = JobInfoJ.builder("test") + .jobName("test_name") + .runId("test_id") + .jobOwnerId("test_job_id") + .triggerType("test_trigger_type") + .build() + val jobInfoFromConstructor = new JobInfoJ( + "test", + "test_name", + "test_id", + "test_job_id", + "test_trigger_type") + assert(jobInfoFromBuilder == jobInfoFromConstructor) + } + + test("builder action class constructor for RemoveFile") { + val removeFileJFromBuilderDefaults = RemoveFileJ.builder("/test").build() + val removeFileJFromConstructorDefaults = new RemoveFileJ( + "/test", + Optional.empty(), + true, + false, + null, + 0L, + null) + assert(removeFileJFromBuilderDefaults == removeFileJFromConstructorDefaults) + + val removeFileJFromBuilder = RemoveFileJ.builder("/test") + .deletionTimestamp(0L) + .dataChange(false) + .extendedFileMetadata(true) + .partitionValues(Map("test"->"foo").asJava) + .size(1L) + .tags(Map("tag"->"foo").asJava) + .build() + val removeFileJFromConstructor = new RemoveFileJ( + "/test", + Optional.of(0L), + false, + true, + Map("test"->"foo").asJava, + 1L, + Map("tag"->"foo").asJava) + assert(removeFileJFromBuilder == removeFileJFromConstructor) + } + + test("builder action class constructor for CommitInfo") { + val commitInfoFromBuilderDefaults = CommitInfoJ.builder().build() + val commitInfoFromConstructorDefaults = new CommitInfoJ( + Optional.empty(), + null, + Optional.empty(), + Optional.empty(), + null, + null, + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty()) + assert(commitInfoFromBuilderDefaults == commitInfoFromConstructorDefaults) + + val commitInfoFromBuilder = CommitInfoJ.builder() + .version(0L) + .timestamp(new Timestamp(1540415658000L)) + .userId("test_id") + .userName("test_name") + .operation("test_op") + .operationParameters(Map("test"->"op").asJava) + .jobInfo(JobInfoJ.builder("test").build()) + .notebookInfo(new NotebookInfoJ("test")) + .clusterId("test_clusterId") + .readVersion(0L) + .isolationLevel("test_level") + .isBlindAppend(true) + .operationMetrics(Map("test"->"metric").asJava) + .userMetadata("user_metadata") + .engineInfo("engine_info") + .build() + val commitInfoFromConstructor = new CommitInfoJ( + Optional.of(0L), + new Timestamp(1540415658000L), + Optional.of("test_id"), + Optional.of("test_name"), + "test_op", + Map("test"->"op").asJava, + Optional.of(JobInfoJ.builder("test").build()), + Optional.of(new NotebookInfoJ("test")), + Optional.of("test_clusterId"), + Optional.of(0L), + Optional.of("test_level"), + Optional.of(true), + Optional.of(Map("test"->"metric").asJava), + Optional.of("user_metadata"), + Optional.of("engine_info")) + assert(commitInfoFromBuilder == commitInfoFromConstructor) + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala index b7f7d17c634..420c30abc7b 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ConversionUtilsSuite.scala @@ -16,12 +16,58 @@ package io.delta.standalone.internal -// scalastyle:off funsuite +import java.sql.Timestamp + +import io.delta.standalone.internal.actions._ +import io.delta.standalone.internal.util.ConversionUtils.{convertAction, convertActionJ} +import io.delta.standalone.types.{IntegerType, StructField, StructType} + import org.scalatest.FunSuite class ConversionUtilsSuite extends FunSuite { - // scalastyle:on funsuite + private val schema = new StructType(Array( + new StructField("col1", new IntegerType()), + new StructField("col2", new IntegerType()) + )) + + private val addFile = AddFile("path", Map("col1" -> "val2", "col2" -> "val2"), 123L, 456L, + dataChange = true, "stats", Map("tagKey" -> "tagVal")) + + private val cdcFile = AddCDCFile("path", Map("col1" -> "val2", "col2" -> "val2"), 700L, + Map("tagKey" -> "tagVal")) + + private val removeFile = addFile.removeWithTimestamp() + + private val metadata = Metadata("id", "name", "desc", Format(), schema.toJson, + Seq("col1", "col2"), Map("configKey" -> "configVal"), Some(789L)) + + private val jobInfo = JobInfo("jobId", "jobName", "runId", "jobOwnerId", "triggerType") + + private val notebookInfo = NotebookInfo("notebookId") + + private val commitInfo = CommitInfo(Some(1L), new Timestamp(1000000), Some("userId"), + Some("userName"), "WRITE", Map("paramKey" -> "paramVal"), Some(jobInfo), Some(notebookInfo), + Some("clusterId"), Some(9L), Some("Serializable"), Some(true), + Some(Map("opMetricKey" -> "opMetricVal")), Some("userMetadata"), Some("engineInfo")) + + private val setTransaction = SetTransaction("appId", 1L, Some(2000L)) + + private val protocol = Protocol() + + private val actions = + Seq(addFile, cdcFile, removeFile, metadata, commitInfo, setTransaction, protocol) - // TODO: test convertAction and convertActionJ + test("convert actions") { + actions.foreach { scalaAction => + val javaAction = convertAction(scalaAction) + val newScalaAction = convertActionJ(javaAction) + assert(newScalaAction == scalaAction, + s""" + |New Scala action: ${newScalaAction.toString} + |did not equal + |Original Scala action ${scalaAction.toString} + |""".stripMargin) + } + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index 14b9a2d1d2b..0aeb788b4db 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -19,24 +19,21 @@ package io.delta.standalone.internal import java.io.File import java.nio.file.Files import java.sql.Timestamp -import java.util.Collections -import java.util.Optional import java.util.UUID import scala.collection.JavaConverters._ -import io.delta.standalone.{DeltaLog, Snapshot} -import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, - Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, - RemoveFile => RemoveFileJ} -import io.delta.standalone.internal.actions.{Action, Protocol} +import io.delta.standalone.{DeltaLog, Operation, Snapshot} +import io.delta.standalone.actions.{JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.util.{ConversionUtils, FileNames} import io.delta.standalone.internal.util.GoldenTableUtils._ -import io.delta.standalone.types.{IntegerType, StructField => StructFieldJ, StructType => StructTypeJ} +import io.delta.standalone.internal.util.TestUtils._ + import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - import org.scalatest.FunSuite /** @@ -49,6 +46,10 @@ import org.scalatest.FunSuite * been generated. */ class DeltaLogSuite extends FunSuite { + + val engineInfo = "test-engine-info" + val manualUpdate = new Operation(Operation.Name.MANUAL_UPDATE) + // scalastyle:on funsuite test("checkpoint") { withLogForGoldenTable("checkpoint") { log => @@ -58,8 +59,6 @@ class DeltaLogSuite extends FunSuite { } } - // TODO: another checkpoint test - test("snapshot") { def getDirDataFiles(tablePath: String): Array[File] = { val correctTablePath = @@ -148,7 +147,49 @@ class DeltaLogSuite extends FunSuite { } } - // TODO: update should pick up checkpoints + test("update shouldn't pick up delta files earlier than checkpoint") { + withTempDir { tempDir => + val log1 = DeltaLog.forTable(new Configuration(), new Path(tempDir.getCanonicalPath)) + + (1 to 5).foreach { i => + val txn = log1.startTransaction() + val metadata = if (i == 1) Metadata() :: Nil else Nil + val file = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil + val delete: Seq[Action] = if (i > 1) { + RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil + } else { + Nil + } + + val filesToCommit = (metadata ++ delete ++ file).map(ConversionUtils.convertAction) + + txn.commit(filesToCommit.asJava, manualUpdate, engineInfo) + } + + // DeltaOSS performs `DeltaLog.clearCache()` here, but we can't + val log2 = DeltaLogImpl.forTable(new Configuration(), new Path(tempDir.getCanonicalPath)) + + (6 to 15).foreach { i => + val txn = log1.startTransaction() + val file = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil + val delete = RemoveFile(i - 1 toString, Some(System.currentTimeMillis()), true) :: Nil + + val filesToCommit = (delete ++ file).map(ConversionUtils.convertAction) + + txn.commit(filesToCommit.asJava, manualUpdate, engineInfo) + } + + // Since log2 is a separate instance, it shouldn't be updated to version 15 + assert(log2.snapshot.getVersion == 4) + val updateLog2 = log2.update() + assert(updateLog2.getVersion == log1.snapshot.getVersion, "Did not update to correct version") + + val deltas = log2.snapshot.logSegment.deltas + assert(deltas.length === 4, "Expected 4 files starting at version 11 to 14") + val versions = deltas.map(f => FileNames.deltaVersion(f.getPath)).sorted + assert(versions === Seq[Long](11, 12, 13, 14), "Received the wrong files for update") + } + } test("handle corrupted '_last_checkpoint' file") { withLogImplForGoldenTable("corrupted-last-checkpoint") { log1 => @@ -195,7 +236,26 @@ class DeltaLogSuite extends FunSuite { } } - // TODO: do not relativize paths in RemoveFiles + test("do not relativize paths in RemoveFiles") { + withTempDir { dir => + val log = DeltaLogImpl.forTable(new Configuration(), dir.getCanonicalPath) + assert(new File(log.logPath.toUri).mkdirs()) + val path = new File(dir, "a/b/c").getCanonicalPath + + val removeFile = RemoveFileJ + .builder(path) + .deletionTimestamp(System.currentTimeMillis()) + .dataChange(true) + .build() + val metadata = MetadataJ.builder().build() + val actions = java.util.Arrays.asList(removeFile, metadata) + + log.startTransaction().commit(actions, manualUpdate, engineInfo) + + val committedRemove = log.update().tombstonesScala + assert(committedRemove.head.path === s"file://$path") + } + } test("delete and re-add the same file in different transactions") { withLogForGoldenTable("delete-re-add-same-file-different-transactions") { log => @@ -356,188 +416,4 @@ class DeltaLogSuite extends FunSuite { } } } - - test("builder action class constructor for Metadata") { - val metadataFromBuilderDefaults = MetadataJ.builder().build() - val metadataFromConstructorDefaults = new MetadataJ( - metadataFromBuilderDefaults.getId(), - null, - null, - new FormatJ("parquet", Collections.emptyMap()), - Collections.emptyList(), - Collections.emptyMap(), - metadataFromBuilderDefaults.getCreatedTime(), - null); - assert(metadataFromBuilderDefaults == metadataFromConstructorDefaults) - - val metadataFromBuilder = MetadataJ.builder() - .id("test_id") - .name("test_name") - .description("test_description") - .format(new FormatJ("csv", Collections.emptyMap())) - .partitionColumns(List("id", "name").asJava) - .configuration(Map("test"->"foo").asJava) - .createdTime(0L) - .schema(new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) - .build() - val metadataFromConstructor = new MetadataJ( - "test_id", - "test_name", - "test_description", - new FormatJ("csv", Collections.emptyMap()), - List("id", "name").asJava, - Map("test"->"foo").asJava, - Optional.of(0L), - new StructTypeJ(Array(new StructFieldJ("test_field", new IntegerType())))) - assert(metadataFromBuilder == metadataFromConstructor) - } - - test("builder action class constructor for AddFile") { - val addFileFromBuilderDefaults = AddFileJ.builder( - "/test", - Collections.emptyMap(), - 0L, - 0L, - true).build() - val addFileFromConstructorDefaults = new AddFileJ( - "/test", - Collections.emptyMap(), - 0L, - 0L, - true, - null, - null) - assert(addFileFromBuilderDefaults == addFileFromConstructorDefaults) - - val addFileFromBuilder = AddFileJ.builder( - "/test", - Collections.emptyMap(), - 0L, - 0L, - true) - .stats("test_stats") - .tags(Map("test"->"foo").asJava) - .build() - val addFileFromConstructor = new AddFileJ( - "/test", - Collections.emptyMap(), - 0L, - 0L, - true, - "test_stats", - Map("test"->"foo").asJava) - assert(addFileFromBuilder == addFileFromConstructor) - } - - test("builder action class constructor for JobInfo") { - val jobInfoFromBuilderDefaults = JobInfoJ.builder("test").build() - val jobInfoFromConstructorDefaults = new JobInfoJ( - "test", - null, - null, - null, - null) - assert(jobInfoFromBuilderDefaults == jobInfoFromConstructorDefaults) - - val jobInfoFromBuilder = JobInfoJ.builder("test") - .jobName("test_name") - .runId("test_id") - .jobOwnerId("test_job_id") - .triggerType("test_trigger_type") - .build() - val jobInfoFromConstructor = new JobInfoJ( - "test", - "test_name", - "test_id", - "test_job_id", - "test_trigger_type") - assert(jobInfoFromBuilder == jobInfoFromConstructor) - } - - test("builder action class constructor for RemoveFile") { - val removeFileJFromBuilderDefaults = RemoveFileJ.builder("/test").build() - val removeFileJFromConstructorDefaults = new RemoveFileJ( - "/test", - Optional.empty(), - true, - false, - null, - 0L, - null) - assert(removeFileJFromBuilderDefaults == removeFileJFromConstructorDefaults) - - val removeFileJFromBuilder = RemoveFileJ.builder("/test") - .deletionTimestamp(0L) - .dataChange(false) - .extendedFileMetadata(true) - .partitionValues(Map("test"->"foo").asJava) - .size(1L) - .tags(Map("tag"->"foo").asJava) - .build() - val removeFileJFromConstructor = new RemoveFileJ( - "/test", - Optional.of(0L), - false, - true, - Map("test"->"foo").asJava, - 1L, - Map("tag"->"foo").asJava) - assert(removeFileJFromBuilder == removeFileJFromConstructor) - } - - test("builder action class constructor for CommitInfo") { - val commitInfoFromBuilderDefaults = CommitInfoJ.builder().build() - val commitInfoFromConstructorDefaults = new CommitInfoJ( - Optional.empty(), - null, - Optional.empty(), - Optional.empty(), - null, - null, - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty()) - assert(commitInfoFromBuilderDefaults == commitInfoFromConstructorDefaults) - - val commitInfoFromBuilder = CommitInfoJ.builder() - .version(0L) - .timestamp(new Timestamp(1540415658000L)) - .userId("test_id") - .userName("test_name") - .operation("test_op") - .operationParameters(Map("test"->"op").asJava) - .jobInfo(JobInfoJ.builder("test").build()) - .notebookInfo(new NotebookInfoJ("test")) - .clusterId("test_clusterId") - .readVersion(0L) - .isolationLevel("test_level") - .isBlindAppend(true) - .operationMetrics(Map("test"->"metric").asJava) - .userMetadata("user_metadata") - .engineInfo("engine_info") - .build() - val commitInfoFromConstructor = new CommitInfoJ( - Optional.of(0L), - new Timestamp(1540415658000L), - Optional.of("test_id"), - Optional.of("test_name"), - "test_op", - Map("test"->"op").asJava, - Optional.of(JobInfoJ.builder("test").build()), - Optional.of(new NotebookInfoJ("test")), - Optional.of("test_clusterId"), - Optional.of(0L), - Optional.of("test_level"), - Optional.of(true), - Optional.of(Map("test"->"metric").asJava), - Optional.of("user_metadata"), - Optional.of("engine_info")) - assert(commitInfoFromBuilder == commitInfoFromConstructor) - } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 5509890ccd6..841400d19d8 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -19,16 +19,15 @@ package io.delta.standalone.internal import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import io.delta.standalone.{DeltaLog, Operation} +import io.delta.standalone.{DeltaLog, Operation, NAME, VERSION} import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDeleteDeleteException, ConcurrentDeleteReadException, ConcurrentTransactionException, MetadataChangedException, ProtocolChangedException} -import io.delta.standalone.expressions.{EqualTo, Expression, Literal} +import io.delta.standalone.expressions.{EqualTo, Literal} import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.{ConversionUtils, SchemaUtils} import io.delta.standalone.types._ import io.delta.standalone.internal.util.TestUtils._ - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -57,7 +56,6 @@ class OptimisticTransactionLegacySuite extends FunSuite { private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) - private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) def withLog( actions: Seq[Action], @@ -475,20 +473,19 @@ class OptimisticTransactionLegacySuite extends FunSuite { /////////////////////////////////////////////////////////////////////////// test("can't have duplicate column names") { - // TODO: just call myStruct.getJson() - // scalastyle:off - val schemaStr = """{"type":"struct","fields":[{"name":"col1","type":"integer","nullable":true,"metadata":{}},{"name":"col1","type":"integer","nullable":true,"metadata":{}}]}""" - // scalastyle:on - testMetadata[RuntimeException](Metadata(schemaString = schemaStr), "Found duplicate column(s)") + val schema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col1", new StringType(), true) + )) + testMetadata[RuntimeException](Metadata(schemaString = schema.toJson), + "Found duplicate column(s)") } test("column names (both data and partition) must be acceptable by parquet") { - // TODO: just call myStruct.getJson() + val schema = new StructType(Array(new StructField("bad;column,name", new IntegerType(), true))) + // test DATA columns - // scalastyle:off - val schemaStr1 = """{"type":"struct","fields":[{"name":"bad;column,name","type":"integer","nullable":true,"metadata":{}}]}""" - // scalastyle:on - testMetadata[RuntimeException](Metadata(schemaString = schemaStr1), + testMetadata[RuntimeException](Metadata(schemaString = schema.toJson), """Attribute name "bad;column,name" contains invalid character(s)""") // test PARTITION columns @@ -508,7 +505,7 @@ class OptimisticTransactionLegacySuite extends FunSuite { val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val commitInfo = log2.getCommitInfoAt(0) assert(commitInfo.getEngineInfo.isPresent) - assert(commitInfo.getEngineInfo.get() == engineInfo) + assert(commitInfo.getEngineInfo.get() == s"$engineInfo $NAME/$VERSION") assert(commitInfo.getOperation == manualUpdate.getName.toString) // TODO: test commitInfo.operationParameters @@ -727,13 +724,7 @@ class OptimisticTransactionLegacySuite extends FunSuite { } } - // TODO multiple concurrent commits, not just one (i.e. 1st doesn't conflict, 2nd does) - - // TODO: readWholeTable tests - // TODO: test Checkpoint > partialWriteVisible (==> useRename) // TODO: test Checkpoint > !partialWriteVisible (==> !useRename) - - // TODO: test Checkpoint > correctly checkpoints all action types } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 88a2e44f72d..3c9ebdcdcee 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration class OptimisticTransactionSuite extends OptimisticTransactionSuiteBase - with OptimisticTransactionSuiteTestVals { + with OptimisticTransactionSuiteTestVals { /////////////////////////////////////////////////////////////////////////// // Allowed concurrent actions From 56f40e38329d728794452a414676161d737baea6 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 18 Oct 2021 10:41:33 -0700 Subject: [PATCH 104/291] [DSW] [22] Misc Fixes 2 (#176) --- .../java/io/delta/standalone/Operation.java | 38 +++++++++-- .../exceptions/DeltaStandaloneException.java | 31 +++++++++ .../delta/standalone/types/StructField.java | 25 ++++---- .../standalone/internal/DeltaConfig.scala | 1 - .../standalone/internal/MetadataCleanup.scala | 2 - .../internal/OptimisticTransactionImpl.scala | 4 +- .../internal/exception/DeltaErrors.scala | 36 +++++------ .../sources/StandaloneHadoopConf.scala | 4 +- .../internal/storage/HDFSLogStore.scala | 2 +- .../internal/storage/LogStoreProvider.scala | 4 +- .../internal/util/PartitionUtils.scala | 12 ++-- .../internal/util/SchemaMergingUtils.scala | 6 +- .../internal/util/SchemaUtils.scala | 14 ++-- .../standalone/internal/DeltaLogSuite.scala | 3 +- .../internal/DeltaTimeTravelSuite.scala | 8 +-- .../standalone/internal/LogStoreSuite.scala | 64 ++++++++++++++++--- .../OptimisticTransactionLegacySuite.scala | 3 - 17 files changed, 178 insertions(+), 79 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index c55a1392edb..f66efcb4872 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -32,7 +32,38 @@ public enum Name { /** Recorded when converting a table into a Delta table. */ CONVERT("CONVERT"), - // TODO: the rest + /** Recorded when a merge operation is committed to the table. */ + MERGE("MERGE"), + + /** Recorded when an update operation is committed to the table. */ + UPDATE("UPDATE"), + + /** Recorded when the table is created. */ + CREATE_TABLE("CREATE TABLE"), + + /** Recorded when the table is replaced. */ + REPLACE_TABLE("REPLACE TABLE"), + + /** Recorded when the table properties are set. */ + SET_TABLE_PROPERTIES("SET TBLPROPERTIES"), + + /** Recorded when the table properties are unset. */ + UNSET_TABLE_PROPERTIES("UNSET TBLPROPERTIES"), + + /** Recorded when columns are added. */ + ADD_COLUMNS("ADD COLUMNS"), + + /** Recorded when columns are changed. */ + CHANGE_COLUMN("CHANGE COLUMN"), + + /** Recorded when columns are replaced. */ + REPLACE_COLUMNS("REPLACE COLUMNS"), + + /** Recorded when the table protocol is upgraded. */ + UPGRADE_PROTOCOL("UPGRADE PROTOCOL"), + + /** Recorded when the table schema is upgraded. */ + UPGRADE_SCHEMA("UPDATE SCHEMA"), MANUAL_UPDATE("Manual Update"); @@ -85,15 +116,14 @@ public Name getName() { * @return operation parameters */ public Map getParameters() { - // TODO: be consistent with AddFile getter ternary - return null == parameters ? null : Collections.unmodifiableMap(parameters); + return parameters != null ? Collections.unmodifiableMap(parameters) : null; } /** * @return operation metrics */ public Map getOperationMetrics() { - return null == operationMetrics ? null : Collections.unmodifiableMap(operationMetrics); + return operationMetrics != null ? Collections.unmodifiableMap(operationMetrics) : null; } /** diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java new file mode 100644 index 00000000000..ee516a19273 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java @@ -0,0 +1,31 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.exceptions; + +public class DeltaStandaloneException extends RuntimeException { + public DeltaStandaloneException() { + super(); + } + + public DeltaStandaloneException(String message) { + super(message); + } + + public DeltaStandaloneException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/types/StructField.java b/standalone/src/main/java/io/delta/standalone/types/StructField.java index 6afdfd203a4..8bbee138f7c 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructField.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructField.java @@ -52,16 +52,23 @@ public final class StructField { private final boolean nullable; private final FieldMetadata metadata; + /** + * Constructor with default {@code nullable = true}. + * + * @param name the name of this field + * @param dataType the data type of this field + */ + public StructField(String name, DataType dataType) { + this(name, dataType, true); + } + /** * @param name the name of this field * @param dataType the data type of this field * @param nullable indicates if values of this field can be {@code null} values */ public StructField(String name, DataType dataType, boolean nullable) { - this.name = name; - this.dataType = dataType; - this.nullable = nullable; - this.metadata = FieldMetadata.builder().build(); + this(name, dataType, nullable, FieldMetadata.builder().build()); } /** @@ -77,16 +84,6 @@ public StructField(String name, DataType dataType, boolean nullable, FieldMetada this.metadata = metadata; } - /** - * Constructor with default {@code nullable = true}. - * - * @param name the name of this field - * @param dataType the data type of this field - */ - public StructField(String name, DataType dataType) { - this(name, dataType, true); - } - /** * @return the name of this field */ diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala index 3697159eb49..2491389cad5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala @@ -36,7 +36,6 @@ private[internal] case class DeltaConfig[T]( * If undefined, return defaultValue. */ def fromMetadata(metadata: Metadata): T = { - // todo: how to fail gracefully for an invalid value (see validate() below) fromString(metadata.configuration.getOrElse(key, defaultValue)) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala index e8cac2c7472..4c16f0ab1a7 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala @@ -83,8 +83,6 @@ private[internal] trait MetadataCleanup { private def truncateDay(timeMillis: Long): Calendar = { val date = Calendar.getInstance(TimeZone.getTimeZone("UTC")) date.setTimeInMillis(timeMillis) - - // TODO: this is using org.apache.commons.lang2.6 instead of org.apache.commons.lang3 DateUtils.truncate( date, Calendar.DAY_OF_MONTH) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index ff6a9153cce..5b5d2a73db6 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction, NAME, VERSION} import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ} +import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.expressions.{Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors @@ -373,8 +374,7 @@ private[internal] class OptimisticTransactionImpl( try { SchemaUtils.checkFieldNames(metadata.partitionColumns) } catch { - // TODO: case e: AnalysisException ? - case e: RuntimeException => throw DeltaErrors.invalidPartitionColumn(e) + case e: DeltaStandaloneException => throw DeltaErrors.invalidPartitionColumn(e) } Protocol.checkMetadataProtocolProperties(metadata, protocol) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 9c85d901a47..2cf1bff9853 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -18,10 +18,11 @@ package io.delta.standalone.internal.exception import java.io.{FileNotFoundException, IOException} -import io.delta.standalone.internal.actions.{CommitInfo, Protocol} -import io.delta.standalone.internal.util.JsonUtils + import io.delta.standalone.types.{DataType, StructType} import io.delta.standalone.exceptions._ +import io.delta.standalone.internal.actions.{CommitInfo, Protocol} +import io.delta.standalone.internal.util.JsonUtils import org.apache.hadoop.fs.Path @@ -67,9 +68,8 @@ private[internal] object DeltaErrors { s"Couldn't find all part files of the checkpoint version: $version", e) } - def noReproducibleHistoryFound(logPath: Path): Throwable = { - // TODO: AnalysisException ? - new RuntimeException(s"No reproducible commits found at $logPath") + def noReproducibleHistoryFound(logPath: Path): DeltaStandaloneException = { + new DeltaStandaloneException(s"No reproducible commits found at $logPath") } def timestampEarlierThanTableFirstCommit( @@ -84,19 +84,18 @@ private[internal] object DeltaErrors { def timestampLaterThanTableLastCommit( userTimestamp: java.sql.Timestamp, commitTs: java.sql.Timestamp): Throwable = { - new IllegalArgumentException( + new DeltaStandaloneException( s"""The provided timestamp ($userTimestamp) is after the latest version available to this |table ($commitTs). Please use a timestamp less than or equal to $commitTs. """.stripMargin) } - def noHistoryFound(logPath: Path): Throwable = { - // TODO: AnalysisException ? - new RuntimeException(s"No commits found at $logPath") + def noHistoryFound(logPath: Path): DeltaStandaloneException = { + new DeltaStandaloneException(s"No commits found at $logPath") } def versionNotExistException(userVersion: Long, earliest: Long, latest: Long): Throwable = { - new IllegalArgumentException(s"Cannot time travel Delta table to version $userVersion. " + + new DeltaStandaloneException(s"Cannot time travel Delta table to version $userVersion. " + s"Available versions: [$earliest, $latest].") } @@ -145,18 +144,15 @@ private[internal] object DeltaErrors { s"(appendOnly=false)'.") } - def invalidColumnName(name: String): Throwable = { - // TODO: AnalysisException ?? - new RuntimeException( + def invalidColumnName(name: String): DeltaStandaloneException = { + new DeltaStandaloneException( s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\\n\\t=". |Please use alias to rename it. """.stripMargin.split("\n").mkString(" ").trim) } - // TODO: AnalysisException ?? - def invalidPartitionColumn(e: RuntimeException): Throwable = { - // TODO: AnalysisException ?? - new RuntimeException( + def invalidPartitionColumn(e: RuntimeException): DeltaStandaloneException = { + new DeltaStandaloneException( """Found partition columns having invalid character(s) among " ,;{}()\n\t=". Please """ + "change the name to your partition columns. This check can be turned off by setting " + """spark.conf.set("spark.databricks.delta.partitionColumnValidity.enabled", false) """ + @@ -267,8 +263,8 @@ private[internal] object DeltaErrors { } def nestedNotNullConstraint( - parent: String, nested: DataType, nestType: String): RuntimeException = { - new RuntimeException(s"The $nestType type of the field $parent contains a NOT NULL " + + parent: String, nested: DataType, nestType: String): DeltaStandaloneException = { + new DeltaStandaloneException(s"The $nestType type of the field $parent contains a NOT NULL " + s"constraint. Delta does not support NOT NULL constraints nested within arrays or maps. " + s"Parsed $nestType type:\n${nested.toPrettyJson}") } @@ -284,7 +280,7 @@ private[internal] object DeltaErrors { } def unknownConfigurationKeyException(confKey: String): Throwable = { - new IllegalArgumentException(s"Unknown configuration was specified: $confKey") + new DeltaStandaloneException(s"Unknown configuration was specified: $confKey") } /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala index 8af33b0c9b5..ad10ad99b13 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/sources/StandaloneHadoopConf.scala @@ -21,9 +21,9 @@ package io.delta.standalone.internal.sources */ private[internal] object StandaloneHadoopConf { - /** time zone as which time-based parquet values will be encoded and decoded */ + /** Time zone as which time-based parquet values will be encoded and decoded. */ val PARQUET_DATA_TIME_ZONE_ID = "io.delta.standalone.PARQUET_DATA_TIME_ZONE_ID" - /** TODO */ + /** Class name for the desired [[LogStore]] implementation to be used. */ val LOG_STORE_CLASS_KEY = "io.delta.standalone.LOG_STORE_CLASS_KEY" } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala index efd204ef070..338fdef35b5 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala @@ -65,7 +65,7 @@ private[internal] class HDFSLogStore(override val initHadoopConf: Configuration) } override def isPartialWriteVisible( - path: Path, hadoopConf: Configuration): java.lang.Boolean = false + path: Path, hadoopConf: Configuration): java.lang.Boolean = true private def writeInternal( path: Path, diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala index e2658d65d66..828daeb99a9 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala @@ -16,6 +16,7 @@ package io.delta.standalone.internal.storage +import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.storage.LogStore import io.delta.standalone.internal.sources.StandaloneHadoopConf @@ -42,8 +43,7 @@ private[internal] trait LogStoreProvider { .newInstance(hadoopConf) .asInstanceOf[LogStore] } else { - // TODO proper error? - throw new IllegalArgumentException(s"Can't instantiate a LogStore with classname " + + throw new DeltaStandaloneException(s"Can't instantiate a LogStore with classname " + s"$logStoreClassName.") } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala index 045f46196a9..b3603161b34 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala @@ -16,6 +16,8 @@ package io.delta.standalone.internal.util +import java.util.Locale + import scala.collection.JavaConverters._ import io.delta.standalone.expressions.{And, Expression, Literal} @@ -41,8 +43,6 @@ private[internal] object PartitionUtils { partitionSchema: StructType, files: Seq[AddFile], partitionFilter: Expression): Seq[AddFile] = { - // TODO: compressedExpr = ... - files.filter { addFile => val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) val result = partitionFilter.eval(partitionRowRecord) @@ -81,8 +81,12 @@ private[internal] object PartitionUtils { * Check if condition can be evaluated using only metadata (i.e. partition columns) */ def isPredicateMetadataOnly(condition: Expression, partitionColumns: Seq[String]): Boolean = { - // TODO: name equality resolver ? - condition.references().asScala.forall(partitionColumns.contains(_)) + val lowercasePartCols = partitionColumns.map(_.toLowerCase(Locale.ROOT)) + + condition.references() + .asScala + .map(_.toLowerCase(Locale.ROOT)) + .forall(lowercasePartCols.contains(_)) } private def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala index 8831af9cff3..5ee415a1ac7 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala @@ -16,7 +16,8 @@ package io.delta.standalone.internal.util -import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType} +import io.delta.standalone.exceptions.DeltaStandaloneException +import io.delta.standalone.types.{ArrayType, DataType, MapType, StructType} /** * Utils to merge table schema with data schema. @@ -83,8 +84,7 @@ private[internal] object SchemaMergingUtils { case (x, ys) if ys.length > 1 => s"$x" } - // TODO: AnalysisException ? - throw new RuntimeException( + throw new DeltaStandaloneException( s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala index 56e3409574b..f4aaac20768 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala @@ -16,8 +16,9 @@ package io.delta.standalone.internal.util -import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType} +import io.delta.standalone.internal.exception.DeltaErrors private[internal] object SchemaUtils { @@ -109,6 +110,10 @@ private[internal] object SchemaUtils { columnPath.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") private object ParquetSchemaConverter { + def checkFieldNames(names: Seq[String]): Unit = { + names.foreach(checkFieldName) + } + def checkFieldName(name: String): Unit = { // ,;{}()\n\t= and space are special characters in Parquet schema checkConversionRequirement( @@ -118,14 +123,9 @@ private[internal] object SchemaUtils { """.stripMargin.split("\n").mkString(" ").trim) } - def checkFieldNames(names: Seq[String]): Unit = { - names.foreach(checkFieldName) - } - def checkConversionRequirement(f: => Boolean, message: String): Unit = { if (!f) { - // TODO: AnalysisException ? - throw new RuntimeException(message) + throw new DeltaStandaloneException(message) } } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index 0aeb788b4db..e5b2a0e42ce 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaLog, Operation, Snapshot} import io.delta.standalone.actions.{JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.internal.actions.{Action, AddFile, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.{ConversionUtils, FileNames} @@ -343,7 +344,7 @@ class DeltaLogSuite extends FunSuite { } // test illegal version - assertThrows[IllegalArgumentException] { + assertThrows[DeltaStandaloneException] { log.getCommitInfoAt(99) } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala index bd03aea72d3..4498300864f 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaTimeTravelSuite.scala @@ -26,13 +26,13 @@ import scala.concurrent.duration._ import scala.language.implicitConversions import io.delta.standalone.{DeltaLog, Snapshot} +import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.FileNames import io.delta.standalone.internal.util.GoldenTableUtils._ import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - import org.scalatest.FunSuite /** @@ -108,13 +108,13 @@ class DeltaTimeTravelSuite extends FunSuite { verifySnapshot(log.getSnapshotForVersionAsOf(2), data_files_version_2, 2) // Error case - version after latest commit - val e1 = intercept[IllegalArgumentException] { + val e1 = intercept[DeltaStandaloneException] { log.getSnapshotForVersionAsOf(3) } assert(e1.getMessage == DeltaErrors.versionNotExistException(3, 0, 2).getMessage) // Error case - version before earliest commit - val e2 = intercept[IllegalArgumentException] { + val e2 = intercept[DeltaStandaloneException] { log.getSnapshotForVersionAsOf(-1) } assert(e2.getMessage == DeltaErrors.versionNotExistException(-1, 0, 2).getMessage) @@ -155,7 +155,7 @@ class DeltaTimeTravelSuite extends FunSuite { new File(logDir, "00000000000000000002.json").setLastModified(start + 40.minutes) val log = DeltaLog.forTable(new Configuration(), tablePath) - val e = intercept[IllegalArgumentException] { + val e = intercept[DeltaStandaloneException] { log.getSnapshotForTimestampAsOf(start + 50.minutes) // later by 10 mins } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala index e5ac79d0d71..c101b22cdb4 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/LogStoreSuite.scala @@ -20,17 +20,18 @@ import java.io.File import scala.collection.JavaConverters._ +import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.data.{CloseableIterator => CloseableIteratorJ} import io.delta.standalone.storage.LogStore import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.storage.{HDFSLogStore, LogStoreProvider} import io.delta.standalone.internal.util.GoldenTableUtils._ import io.delta.standalone.internal.util.TestUtils._ +import io.delta.standalone.Operation import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} -// scalastyle:off funsuite import org.scalatest.FunSuite /** @@ -43,7 +44,6 @@ import org.scalatest.FunSuite * been generated. */ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { - // scalastyle:on funsuite def logStoreClassName: Option[String] @@ -55,6 +55,12 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { conf } + /** + * Whether the log store being tested should use rename to write checkpoint or not. The following + * test is using this method to verify the behavior of `checkpoint`. + */ + protected def shouldUseRenameToWriteCheckpoint: Boolean + test("instantiation through HadoopConf") { val expectedClassName = logStoreClassName.getOrElse(LogStoreProvider.defaultLogStoreClassName) assert(createLogStore(hadoopConf).getClass.getName == expectedClassName) @@ -130,6 +136,44 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { .filterNot(_ == "_delta_log").toArray === Nil) } } + + test("use isPartialWriteVisible to decide whether use rename") { + withTempDir { tempDir => + val conf = hadoopConf + conf.set("fs.file.impl", classOf[TrackingRenameFileSystem].getName) + conf.set("fs.file.impl.disable.cache", "true") + + val log = DeltaLogImpl.forTable(conf, tempDir.getCanonicalPath) + val addFile = AddFileJ.builder("/path", Map.empty[String, String].asJava, 100L, + System.currentTimeMillis(), true).build() + val metadata = MetadataJ.builder().build() + + log.startTransaction().commit((metadata :: addFile :: Nil).asJava, + new Operation(Operation.Name.MANUAL_UPDATE), "engineInfo") + + TrackingRenameFileSystem.numOfRename = 0 + + log.checkpoint() + + val expectedNumOfRename = if (shouldUseRenameToWriteCheckpoint) 1 else 0 + assert(TrackingRenameFileSystem.numOfRename === expectedNumOfRename) + } + } +} + +/** + * A file system allowing to track how many times `rename` is called. + * `TrackingRenameFileSystem.numOfRename` should be reset to 0 before starting to trace. + */ +class TrackingRenameFileSystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + TrackingRenameFileSystem.numOfRename += 1 + super.rename(src, dst) + } +} + +object TrackingRenameFileSystem { + @volatile var numOfRename = 0 } /** @@ -137,6 +181,7 @@ abstract class LogStoreSuiteBase extends FunSuite with LogStoreProvider { */ class HDFSLogStoreSuite extends LogStoreSuiteBase { override def logStoreClassName: Option[String] = Some(classOf[HDFSLogStore].getName) + override protected def shouldUseRenameToWriteCheckpoint: Boolean = true } /** @@ -145,6 +190,7 @@ class HDFSLogStoreSuite extends LogStoreSuiteBase { */ class DefaultLogStoreSuite extends LogStoreSuiteBase { override def logStoreClassName: Option[String] = None + override protected def shouldUseRenameToWriteCheckpoint: Boolean = true } /** @@ -153,11 +199,9 @@ class DefaultLogStoreSuite extends LogStoreSuiteBase { class UserDefinedLogStoreSuite extends LogStoreSuiteBase { override def logStoreClassName: Option[String] = Some(classOf[UserDefinedLogStore].getName) - override def hadoopConf: Configuration = { - val conf = new Configuration() - conf.set(StandaloneHadoopConf.LOG_STORE_CLASS_KEY, classOf[UserDefinedLogStore].getName) - conf - } + // In [[UserDefinedLogStore]], we purposefully set isPartialWriteVisible to false, so this + // should be false as well + override protected def shouldUseRenameToWriteCheckpoint: Boolean = false } /** @@ -194,6 +238,8 @@ class UserDefinedLogStore(override val initHadoopConf: Configuration) } override def isPartialWriteVisible(path: Path, hadoopConf: Configuration): java.lang.Boolean = { - mockImpl.isPartialWriteVisible(path, hadoopConf) + // mockImpl.isPartialWriteVisible is true, but let's add some test diversity for better branch + // coverage and return false instead + false } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 841400d19d8..5c759c2b7cb 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -724,7 +724,4 @@ class OptimisticTransactionLegacySuite extends FunSuite { } } - // TODO: test Checkpoint > partialWriteVisible (==> useRename) - - // TODO: test Checkpoint > !partialWriteVisible (==> !useRename) } From 935f3036b4351cc64118896135beddfd8929897a Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 20 Oct 2021 09:59:25 -0700 Subject: [PATCH 105/291] [DSW] [23] Add logging (#177) --- build.sbt | 4 +- .../standalone/internal/Checkpoints.scala | 20 +-- .../standalone/internal/ConflictChecker.scala | 129 ++++++++++------- .../standalone/internal/DeltaConfig.scala | 17 +-- .../internal/DeltaHistoryManager.scala | 7 +- .../standalone/internal/DeltaLogImpl.scala | 12 +- .../standalone/internal/MetadataCleanup.scala | 9 +- .../internal/OptimisticTransactionImpl.scala | 133 ++++++++++++++---- .../standalone/internal/SnapshotImpl.scala | 7 +- .../internal/SnapshotManagement.scala | 22 +++ .../internal/exception/DeltaErrors.scala | 7 + .../standalone/internal/logging/Logging.scala | 71 ++++++++++ .../internal/storage/HDFSLogStore.scala | 4 +- .../src/test/resources/log4j.properties | 48 +++++++ 14 files changed, 389 insertions(+), 101 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/logging/Logging.scala create mode 100644 standalone/src/test/resources/log4j.properties diff --git a/build.sbt b/build.sbt index e08fdbb146b..20193d8aa94 100644 --- a/build.sbt +++ b/build.sbt @@ -274,7 +274,9 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("com.fasterxml.jackson.core"), ExclusionRule("com.fasterxml.jackson.module") ), - "org.scalatest" %% "scalatest" % "3.0.5" % "test" + "org.scalatest" %% "scalatest" % "3.0.5" % "test", + "org.slf4j" % "slf4j-api" % "1.7.25", + "org.slf4j" % "slf4j-log4j12" % "1.7.25" ), sourceGenerators in Compile += Def.task { val file = (sourceManaged in Compile).value / "meta" / "package.scala" diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala index 6ff3b2b993a..12996cc1433 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala @@ -26,10 +26,12 @@ import io.delta.standalone.data.CloseableIterator import io.delta.standalone.internal.actions.SingleAction import io.delta.standalone.internal.util.JsonUtils import io.delta.standalone.internal.util.FileNames._ +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging + import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.metadata.CompressionCodecName import com.github.mjakubowski84.parquet4s.ParquetWriter -import io.delta.standalone.internal.exception.DeltaErrors /** * Records information about a checkpoint. @@ -139,17 +141,12 @@ private[internal] trait Checkpoints { case _: FileNotFoundException => None case NonFatal(e) if tries < 3 => - // scalastyle:off println - println(s"Failed to parse $LAST_CHECKPOINT. This may happen if there was an error " + + logWarning(s"Failed to parse $LAST_CHECKPOINT. This may happen if there was an error " + "during read operation, or a file appears to be partial. Sleeping and trying again.", e) - // scalastyle:on println - Thread.sleep(1000) loadMetadataFromFile(tries + 1) case NonFatal(e) => - // scalastyle:off println - println(s"$LAST_CHECKPOINT is corrupted. Will search the checkpoint files directly", e) - // scalastyle:on println + logWarning(s"$LAST_CHECKPOINT is corrupted. Will search the checkpoint files directly", e) // Hit a partial file. This could happen on Azure as overwriting _last_checkpoint file is // not atomic. We will try to list all files to find the latest checkpoint and restore // CheckpointMetaData from it. @@ -208,7 +205,7 @@ private[internal] trait Checkpoints { } } -private[internal] object Checkpoints { +private[internal] object Checkpoints extends Logging { /** * Writes out the contents of a [[Snapshot]] into a checkpoint file that * can be used to short-circuit future replays of the log. @@ -300,6 +297,11 @@ private[internal] object Checkpoints { "State of the checkpoint doesn't match that of the snapshot.") } + // Attempting to write empty checkpoint + if (checkpointSize == 0) { + logWarning(DeltaErrors.EmptyCheckpointErrorMessage) + } + CheckpointMetaData(snapshot.version, checkpointSize, None) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala index b9aa675a60e..d5fa1c67690 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala @@ -16,11 +16,15 @@ package io.delta.standalone.internal +import java.util.concurrent.TimeUnit + import scala.collection.JavaConverters._ +import scala.collection.mutable import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions._ import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import io.delta.standalone.internal.util.{FileNames, PartitionUtils} /** @@ -74,8 +78,10 @@ private[internal] case class WinningCommitSummary(actions: Seq[Action], commitVe private[internal] class ConflictChecker( currentTransactionInfo: CurrentTransactionInfo, winningCommitVersion: Long, - isolationLevel: IsolationLevel) { + isolationLevel: IsolationLevel, + logPrefixStr: String) extends Logging { + private val timingStats = mutable.HashMap[String, Long]() private val deltaLog = currentTransactionInfo.deltaLog private val winningCommitSummary: WinningCommitSummary = createWinningCommitSummary() @@ -86,6 +92,7 @@ private[internal] class ConflictChecker( checkForDeletedFilesAgainstCurrentTxnReadFiles() checkForDeletedFilesAgainstCurrentTxnDeletedFiles() checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn() + reportMetrics() } /** @@ -93,15 +100,17 @@ private[internal] class ConflictChecker( * transaction (winning transaction). */ private def createWinningCommitSummary(): WinningCommitSummary = { - import io.delta.standalone.internal.util.Implicits._ + recordTime("initialize-old-commit") { + import io.delta.standalone.internal.util.Implicits._ - val deltaLog = currentTransactionInfo.deltaLog - val winningCommitActions = deltaLog.store - .read(FileNames.deltaFile(deltaLog.logPath, winningCommitVersion), deltaLog.hadoopConf) - .toArray - .map(Action.fromJson) + val deltaLog = currentTransactionInfo.deltaLog + val winningCommitActions = deltaLog.store + .read(FileNames.deltaFile(deltaLog.logPath, winningCommitVersion), deltaLog.hadoopConf) + .toArray + .map(Action.fromJson) - WinningCommitSummary(winningCommitActions, winningCommitVersion) + WinningCommitSummary(winningCommitActions, winningCommitVersion) + } } /** @@ -137,27 +146,29 @@ private[internal] class ConflictChecker( * the current transaction. */ private def checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn(): Unit = { - // Fail if new files have been added that the txn should have read. - val addedFilesToCheckForConflicts = isolationLevel match { - case Serializable => - winningCommitSummary.changedDataAddedFiles ++ winningCommitSummary.blindAppendAddedFiles - case SnapshotIsolation => - Seq.empty - } + recordTime("checked-appends") { + // Fail if new files have been added that the txn should have read. + val addedFilesToCheckForConflicts = isolationLevel match { + case Serializable => + winningCommitSummary.changedDataAddedFiles ++ winningCommitSummary.blindAppendAddedFiles + case SnapshotIsolation => + Seq.empty + } - val predicatesMatchingAddedFiles = currentTransactionInfo.readPredicates.flatMap { p => - val conflictingFile = PartitionUtils.filterFileList( - currentTransactionInfo.metadata.partitionSchema, - addedFilesToCheckForConflicts, - p - ).headOption + val predicatesMatchingAddedFiles = currentTransactionInfo.readPredicates.flatMap { p => + val conflictingFile = PartitionUtils.filterFileList( + currentTransactionInfo.metadata.partitionSchema, + addedFilesToCheckForConflicts, + p + ).headOption - conflictingFile.map(f => getPrettyPartitionMessage(f.partitionValues)) - }.headOption + conflictingFile.map(f => getPrettyPartitionMessage(f.partitionValues)) + }.headOption - if (predicatesMatchingAddedFiles.isDefined) { - throw DeltaErrors.concurrentAppendException( + if (predicatesMatchingAddedFiles.isDefined) { + throw DeltaErrors.concurrentAppendException( winningCommitSummary.commitInfo, predicatesMatchingAddedFiles.get) + } } } @@ -166,21 +177,23 @@ private[internal] class ConflictChecker( * read by the current transaction. */ private def checkForDeletedFilesAgainstCurrentTxnReadFiles(): Unit = { - // Fail if files have been deleted that the txn read. - val readFilePaths = currentTransactionInfo.readFiles.map( - f => f.path -> f.partitionValues).toMap - val deleteReadOverlap = winningCommitSummary.removedFiles - .find(r => readFilePaths.contains(r.path)) - if (deleteReadOverlap.nonEmpty) { - val filePath = deleteReadOverlap.get.path - val partition = getPrettyPartitionMessage(readFilePaths(filePath)) - throw DeltaErrors.concurrentDeleteReadException( - winningCommitSummary.commitInfo, s"$filePath in $partition") - } - if (winningCommitSummary.removedFiles.nonEmpty && currentTransactionInfo.readWholeTable) { - val filePath = winningCommitSummary.removedFiles.head.path - throw DeltaErrors.concurrentDeleteReadException( - winningCommitSummary.commitInfo, s"$filePath") + recordTime("checked-deletes") { + // Fail if files have been deleted that the txn read. + val readFilePaths = currentTransactionInfo.readFiles.map( + f => f.path -> f.partitionValues).toMap + val deleteReadOverlap = winningCommitSummary.removedFiles + .find(r => readFilePaths.contains(r.path)) + if (deleteReadOverlap.nonEmpty) { + val filePath = deleteReadOverlap.get.path + val partition = getPrettyPartitionMessage(readFilePaths(filePath)) + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath in $partition") + } + if (winningCommitSummary.removedFiles.nonEmpty && currentTransactionInfo.readWholeTable) { + val filePath = winningCommitSummary.removedFiles.head.path + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath") + } } } @@ -189,14 +202,16 @@ private[internal] class ConflictChecker( * [[RemoveFile]] actions this transaction is trying to add. */ private def checkForDeletedFilesAgainstCurrentTxnDeletedFiles(): Unit = { - // Fail if a file is deleted twice. - val txnDeletes = currentTransactionInfo.actions - .collect { case r: RemoveFile => r } - .map(_.path).toSet - val deleteOverlap = winningCommitSummary.removedFiles.map(_.path).toSet intersect txnDeletes - if (deleteOverlap.nonEmpty) { - throw DeltaErrors.concurrentDeleteDeleteException( - winningCommitSummary.commitInfo, deleteOverlap.head) + recordTime("checked-2x-deletes") { + // Fail if a file is deleted twice. + val txnDeletes = currentTransactionInfo.actions + .collect { case r: RemoveFile => r } + .map(_.path).toSet + val deleteOverlap = winningCommitSummary.removedFiles.map(_.path).toSet intersect txnDeletes + if (deleteOverlap.nonEmpty) { + throw DeltaErrors.concurrentDeleteDeleteException( + winningCommitSummary.commitInfo, deleteOverlap.head) + } } } @@ -232,4 +247,22 @@ private[internal] class ConflictChecker( s"partition $partition" } } + + private def recordTime[T](phase: String)(f: => T): T = { + val startTimeNs = System.nanoTime() + val ret = f + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + timingStats += phase -> timeTakenMs + ret + } + + private def reportMetrics(): Unit = { + lazy val timingStr = timingStats.keys + .toSeq + .sorted + .map(k => s"$k=${timingStats(k)}") + .mkString(",") + + logInfo(s"[$logPrefixStr] Timing stats against $winningCommitVersion [$timingStr]") + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala index 2491389cad5..06477822d65 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala @@ -20,7 +20,9 @@ import java.util.{HashMap, Locale} import io.delta.standalone.internal.actions.{Action, Metadata, Protocol} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import io.delta.standalone.internal.util.{CalendarInterval, IntervalUtils} + import org.apache.hadoop.conf.Configuration private[internal] case class DeltaConfig[T]( @@ -65,7 +67,7 @@ private[internal] case class DeltaConfig[T]( /** * Contains list of reservoir configs and validation checks. */ -private[internal] object DeltaConfigs { +private[internal] object DeltaConfigs extends Logging { /** * Convert a string to [[CalendarInterval]]. This method is case-insensitive and will throw @@ -128,7 +130,7 @@ private[internal] object DeltaConfigs { def validateConfigurations(configurations: Map[String, String]): Map[String, String] = { configurations.map { case kv @ (key, value) if key.toLowerCase(Locale.ROOT).startsWith("delta.constraints.") => - throw new IllegalArgumentException(s"Unsupported CHECK constraint configuration ${key} set") + throw new IllegalArgumentException(s"Unsupported CHECK constraint configuration $key set") case (key, value) if key.toLowerCase(Locale.ROOT).startsWith("delta.") => Option(entries.get(key.toLowerCase(Locale.ROOT).stripPrefix("delta."))) .map(_(value)) @@ -137,12 +139,11 @@ private[internal] object DeltaConfigs { } case keyvalue @ (key, _) => if (entries.containsKey(key.toLowerCase(Locale.ROOT))) { - // TODO: add log -// logConsole( -// s""" -// |You are trying to set a property the key of which is the same as Delta config: $key. -// |If you are trying to set a Delta config, prefix it with "delta.", e.g. 'delta.$key'. -// """.stripMargin) + logWarning( + s""" + |You are trying to set a property the key of which is the same as Delta config: $key. + |If you are trying to set a Delta config, prefix it with "delta.", e.g. 'delta.$key'. + |""".stripMargin) } keyvalue } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala index 05212dd9153..1ac2e3e6b80 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala @@ -20,10 +20,11 @@ import java.sql.Timestamp import scala.collection.JavaConverters._ +import io.delta.standalone.storage.LogStore import io.delta.standalone.internal.actions.{Action, CommitInfo, CommitMarker} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.FileNames -import io.delta.standalone.storage.LogStore +import io.delta.standalone.internal.logging.Logging import org.apache.hadoop.fs.Path @@ -33,7 +34,7 @@ import org.apache.hadoop.fs.Path * * @param deltaLog the transaction log of this table */ -private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { +private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) extends Logging { /** Get the persisted commit info for the given delta file. */ def getCommitInfo(version: Long): CommitInfo = { @@ -188,6 +189,8 @@ private[internal] case class DeltaHistoryManager(deltaLog: DeltaLogImpl) { val prevTimestamp = commits(i).getTimestamp assert(commits(i).getVersion < commits(i + 1).getVersion, "Unordered commits provided.") if (prevTimestamp >= commits(i + 1).getTimestamp) { + logWarning(s"Found Delta commit ${commits(i).getVersion} with a timestamp $prevTimestamp " + + s"which is greater than the next commit timestamp ${commits(i + 1).getTimestamp}.") commits(i + 1) = commits(i + 1).withTimestamp(prevTimestamp + 1).asInstanceOf[T] } i += 1 diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index e71d0bd119d..dac49db5bba 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -22,16 +22,18 @@ import java.util.TimeZone import scala.collection.JavaConverters._ -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import io.delta.standalone.{DeltaLog, OptimisticTransaction, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} import io.delta.standalone.internal.actions.{Action, Metadata, Protocol} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import io.delta.standalone.internal.sources.StandaloneHadoopConf import io.delta.standalone.internal.storage.LogStoreProvider import io.delta.standalone.internal.util.{Clock, ConversionUtils, FileNames, SystemClock} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + /** * Scala implementation of Java interface [[DeltaLog]]. */ @@ -44,7 +46,8 @@ private[internal] class DeltaLogImpl private( with Checkpoints with MetadataCleanup with LogStoreProvider - with SnapshotManagement { + with SnapshotManagement + with Logging { /** Used to read and write physical log files and checkpoints. */ lazy val store = createLogStore(hadoopConf) @@ -66,6 +69,9 @@ private[internal] class DeltaLogImpl private( */ def minFileRetentionTimestamp: Long = clock.getTimeMillis() - tombstoneRetentionMillis + /** The unique identifier for this table. */ + def tableId: String = metadata.id + /** Use ReentrantLock to allow us to call `lockInterruptibly`. */ private val deltaLogLock = new ReentrantLock() diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala index 4c16f0ab1a7..bcd708b46b6 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala @@ -49,10 +49,17 @@ private[internal] trait MetadataCleanup { /** Clean up expired delta and checkpoint logs. Exposed for testing. */ def cleanUpExpiredLogs(): Unit = { val fileCutOffTime = truncateDay(clock.getTimeMillis() - deltaRetentionMillis).getTime + + lazy val formattedDate = fileCutOffTime.toGMTString + logInfo(s"Starting the deletion of log files older than $formattedDate") + + var numDeleted = 0 listExpiredDeltaLogs(fileCutOffTime.getTime).map(_.getPath).foreach { path => // recursive = false - fs.delete(path, false) + if (fs.delete(path, false)) numDeleted += 1 } + + logInfo(s"Deleted $numDeleted log files older than $formattedDate") } /** diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 5b5d2a73db6..2d613433d8f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -17,6 +17,7 @@ package io.delta.standalone.internal import java.nio.file.FileAlreadyExistsException +import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -27,13 +28,17 @@ import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.expressions.{Expression, Literal} import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMergingUtils, SchemaUtils} private[internal] class OptimisticTransactionImpl( deltaLog: DeltaLogImpl, - snapshot: SnapshotImpl) extends OptimisticTransaction { + snapshot: SnapshotImpl) extends OptimisticTransaction with Logging { import OptimisticTransactionImpl._ + /** Used for logging */ + private val txnId = UUID.randomUUID().toString + /** Tracks the appIds that have been seen by this transaction. */ private val readTxn = new ArrayBuffer[String] @@ -83,7 +88,7 @@ private[internal] class OptimisticTransactionImpl( // Public Java API Methods /////////////////////////////////////////////////////////////////////////// - override def metadata(): MetadataJ = ConversionUtils.convertMetadata(metadataScala) + override def metadata: MetadataJ = ConversionUtils.convertMetadata(metadataScala) override def commit[T <: ActionJ]( actionsJ: java.lang.Iterable[T], @@ -136,6 +141,8 @@ private[internal] class OptimisticTransactionImpl( postCommit(commitVersion) + logInfo(s"Committed delta #$commitVersion to ${deltaLog.logPath}") + new CommitResult(commitVersion) } @@ -177,6 +184,9 @@ private[internal] class OptimisticTransactionImpl( latestMetadata = latestMetadata.copy(configuration = noProtocolVersionConfig) verifyNewMetadata(latestMetadata) + + logInfo(s"Updated metadata from ${newMetadata.getOrElse("-")} to $latestMetadata") + newMetadata = Some(latestMetadata) } @@ -269,7 +279,7 @@ private[internal] class OptimisticTransactionImpl( while (tryCommit) { try { if (attemptNumber == 0) { - doCommit(commitVersion, actions) + doCommit(commitVersion, actions, isolationLevel) } else if (attemptNumber > DELTA_MAX_RETRY_COMMIT_ATTEMPTS) { val totalCommitAttemptTime = deltaLog.clock.getTimeMillis() - commitAttemptStartTime throw DeltaErrors.maxCommitRetriesExceededException( @@ -279,8 +289,8 @@ private[internal] class OptimisticTransactionImpl( actions.length, totalCommitAttemptTime) } else { - commitVersion = checkForConflicts(commitVersion, actions, isolationLevel) - doCommit(commitVersion, actions) + commitVersion = checkForConflicts(commitVersion, actions, attemptNumber, isolationLevel) + doCommit(commitVersion, actions, isolationLevel) } tryCommit = false } catch { @@ -299,25 +309,37 @@ private[internal] class OptimisticTransactionImpl( * @throws IllegalStateException if the attempted commit version is ahead of the current delta log * version */ - private def doCommit(attemptVersion: Long, actions: Seq[Action]): Long = - deltaLog.lockInterruptibly { - deltaLog.store.write( - FileNames.deltaFile(deltaLog.logPath, attemptVersion), - actions.map(_.json).toIterator.asJava, - false, // overwrite = false - deltaLog.hadoopConf - ) - - val postCommitSnapshot = deltaLog.update() - if (postCommitSnapshot.version < attemptVersion) { - throw new IllegalStateException( - s"The committed version is $attemptVersion " + - s"but the current version is ${postCommitSnapshot.version}.") - } + private def doCommit( + attemptVersion: Long, + actions: Seq[Action], + isolationLevel: IsolationLevel): Long = { + logInfo( + s"Attempting to commit version $attemptVersion with ${actions.size} actions with " + + s"$isolationLevel isolation level") + + if (readVersion > -1 && metadata.getId != snapshot.getMetadata.getId) { + logError(s"Change in the table id detected in txn. Table id for txn on table at " + + s"${deltaLog.dataPath} was ${snapshot.getMetadata.getId} when the txn was created and " + + s"is now changed to ${metadata.getId}.") + } + + deltaLog.store.write( + FileNames.deltaFile(deltaLog.logPath, attemptVersion), + actions.map(_.json).toIterator.asJava, + false, // overwrite = false + deltaLog.hadoopConf + ) - attemptVersion + val postCommitSnapshot = deltaLog.update() + if (postCommitSnapshot.version < attemptVersion) { + throw new IllegalStateException( + s"The committed version is $attemptVersion " + + s"but the current version is ${postCommitSnapshot.version}.") } + attemptVersion + } + /** * Perform post-commit operations */ @@ -325,9 +347,13 @@ private[internal] class OptimisticTransactionImpl( committed = true if (shouldCheckpoint(commitVersion)) { - // We checkpoint the version to be committed to so that no two transactions will checkpoint - // the same version. - deltaLog.checkpoint(deltaLog.getSnapshotForVersionAsOf(commitVersion)) + try { + // We checkpoint the version to be committed to so that no two transactions will checkpoint + // the same version. + deltaLog.checkpoint(deltaLog.getSnapshotForVersionAsOf(commitVersion)) + } catch { + case e: IllegalStateException => logWarning("Failed to checkpoint table state.", e) + } } } @@ -339,6 +365,7 @@ private[internal] class OptimisticTransactionImpl( private def checkForConflicts( checkVersion: Long, actions: Seq[Action], + attemptNumber: Int, commitIsolationLevel: IsolationLevel): Long = { val nextAttemptVersion = getNextAttemptVersion @@ -351,15 +378,39 @@ private[internal] class OptimisticTransactionImpl( actions = actions, deltaLog = deltaLog) + val logPrefixStr = s"[attempt $attemptNumber]" + val txnDetailsLogStr = { + var adds = 0L + var removes = 0L + currentTransactionInfo.actions.foreach { + case _: AddFile => adds += 1 + case _: RemoveFile => removes += 1 + case _ => + } + s"$adds adds, $removes removes, ${readPredicates.size} read predicates, " + + s"${readFiles.size} read files" + } + + logInfo(s"$logPrefixStr Checking for conflicts with versions " + + s"[$checkVersion, $nextAttemptVersion) with current txn having $txnDetailsLogStr") + (checkVersion until nextAttemptVersion).foreach { otherCommitVersion => val conflictChecker = new ConflictChecker( currentTransactionInfo, otherCommitVersion, - commitIsolationLevel) + commitIsolationLevel, + logPrefixStr) conflictChecker.checkConflicts() + + logInfo(s"$logPrefixStr No conflicts in version $otherCommitVersion, " + + s"${deltaLog.clock.getTimeMillis() - commitAttemptStartTime} ms since start") } + logInfo(s"$logPrefixStr No conflicts with versions [$checkVersion, $nextAttemptVersion) " + + s"with current txn having $txnDetailsLogStr, " + + s"${deltaLog.clock.getTimeMillis() - commitAttemptStartTime} ms since start") + nextAttemptVersion } @@ -398,10 +449,40 @@ private[internal] class OptimisticTransactionImpl( metadata.copy(configuration = DeltaConfigs.mergeGlobalConfigs(deltaLog.hadoopConf, metadata.configuration)) } + + /////////////////////////////////////////////////////////////////////////// + // Logging Override Methods + /////////////////////////////////////////////////////////////////////////// + + protected lazy val logPrefix: String = { + def truncate(uuid: String): String = uuid.split("-").head + s"[tableId=${truncate(snapshot.metadataScala.id)},txnId=${truncate(txnId)}] " + } + + override def logInfo(msg: => String): Unit = { + super.logInfo(logPrefix + msg) + } + + override def logWarning(msg: => String): Unit = { + super.logWarning(logPrefix + msg) + } + + override def logWarning(msg: => String, throwable: Throwable): Unit = { + super.logWarning(logPrefix + msg, throwable) + } + + override def logError(msg: => String): Unit = { + super.logError(logPrefix + msg) + } + + override def logError(msg: => String, throwable: Throwable): Unit = { + super.logError(logPrefix + msg, throwable) + } + } private[internal] object OptimisticTransactionImpl { - val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 + val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 // TODO: DeltaConfig this // def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { // op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 8e4faa09b6b..97b03da3086 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -21,12 +21,13 @@ import java.net.URI import scala.collection.JavaConverters._ import io.delta.standalone.{DeltaScan, Snapshot} -import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ, Protocol => ProtocolJ} +import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ} import io.delta.standalone.expressions.Expression import io.delta.standalone.internal.actions.{AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction} import io.delta.standalone.internal.data.CloseableParquetDataIterator import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import io.delta.standalone.internal.scan.{DeltaScanImpl, FilteredDeltaScanImpl} import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils} @@ -48,7 +49,7 @@ private[internal] class SnapshotImpl( val logSegment: LogSegment, val minFileRetentionTimestamp: Long, val deltaLog: DeltaLogImpl, - val timestamp: Long) extends Snapshot { + val timestamp: Long) extends Snapshot with Logging { import SnapshotImpl._ @@ -180,6 +181,8 @@ private[internal] class SnapshotImpl( lazy val transactions: Map[String, Long] = setTransactionsScala.map(t => t.appId -> t.version).toMap + logInfo(s"[tableId=${deltaLog.tableId}] Created snapshot $this") + /** Complete initialization by checking protocol version. */ deltaLog.assertProtocolRead(protocolScala) } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala index 05d09616a6b..12e8907ac4b 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala @@ -66,7 +66,19 @@ private[internal] trait SnapshotManagement { self: DeltaLogImpl => try { val segment = getLogSegmentForVersion(currentSnapshot.logSegment.checkpointVersion) if (segment != currentSnapshot.logSegment) { + val startingFrom = segment.checkpointVersion + .map(v => s" starting from checkpoint version $v.").getOrElse(".") + logInfo(s"Loading version ${segment.version}$startingFrom") + val newSnapshot = createSnapshot(segment, segment.lastCommitTimestamp) + + if (currentSnapshot.version > -1 && + currentSnapshot.metadataScala.id != newSnapshot.metadataScala.id) { + logError(s"Change in the table id detected while updating snapshot. " + + s"\nPrevious snapshot = $currentSnapshot\nNew snapshot = $newSnapshot.") + } + + logInfo(s"Updated snapshot to $newSnapshot") currentSnapshot = newSnapshot } } catch { @@ -75,6 +87,7 @@ private[internal] trait SnapshotManagement { self: DeltaLogImpl => if (Option(e.getMessage).exists(_.contains("reconstruct state at version"))) { throw e } + logInfo(s"No delta log found for the Delta table at $logPath") currentSnapshot = new InitialSnapshotImpl(hadoopConf, logPath, this) } currentSnapshot @@ -203,10 +216,19 @@ private[internal] trait SnapshotManagement { self: DeltaLogImpl => private def getSnapshotAtInit: SnapshotImpl = { try { val logSegment = getLogSegmentForVersion(lastCheckpoint.map(_.version)) + + val startCheckpoint = logSegment.checkpointVersion + .map(v => s" starting from checkpoint $v.").getOrElse(".") + logInfo(s"Loading version ${logSegment.version}$startCheckpoint") + val snapshot = createSnapshot(logSegment, logSegment.lastCommitTimestamp) + + logInfo(s"Returning initial snapshot $snapshot") + snapshot } catch { case _: FileNotFoundException => + logInfo(s"Creating initial snapshot without metadata, because the directory is empty") new InitialSnapshotImpl(hadoopConf, logPath, this) } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 2cf1bff9853..201f9f2ae7a 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -40,6 +40,13 @@ private[internal] object DeltaErrors { |Standalone Reader/Writer ${clientProtocol.simpleString}. Please upgrade to a newer release. |""".stripMargin) + val EmptyCheckpointErrorMessage = + s""" + |Attempted to write an empty checkpoint without any actions. This checkpoint will not be + |useful in recomputing the state of the table. However this might cause other checkpoints to + |get deleted based on retention settings. + """.stripMargin + def deltaVersionsNotContiguousException(deltaVersions: Seq[Long]): Throwable = { new IllegalStateException(s"Versions ($deltaVersions) are not contiguous.") } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/logging/Logging.scala b/standalone/src/main/scala/io/delta/standalone/internal/logging/Logging.scala new file mode 100644 index 00000000000..6aae7810a86 --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/logging/Logging.scala @@ -0,0 +1,71 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.logging + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows + * logging messages at different levels using methods that only evaluate parameters lazily if the + * log level is enabled. + */ +private[internal] trait Logging { + + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get the logger name for this object + private def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + + // Method to get or create the logger for this object + private def log: Logger = { + if (log_ == null) { + log_ = LoggerFactory.getLogger(logName) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String): Unit = { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logWarning(msg: => String): Unit = { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String): Unit = { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable): Unit = { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable): Unit = { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable): Unit = { + if (log.isErrorEnabled) log.error(msg, throwable) + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala index 338fdef35b5..3a093f61c42 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.internal.logging.Logging import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileContext, Options, Path, RawLocalFileSystem} @@ -40,7 +41,7 @@ import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts} * 2. Consistent file listing: HDFS file listing is consistent. */ private[internal] class HDFSLogStore(override val initHadoopConf: Configuration) - extends HadoopFileSystemLogStore(initHadoopConf) { + extends HadoopFileSystemLogStore(initHadoopConf) with Logging { val noAbstractFileSystemExceptionMessage = "No AbstractFileSystem" @@ -77,6 +78,7 @@ private[internal] class HDFSLogStore(override val initHadoopConf: Configuration) } catch { case e: IOException if e.getMessage.contains(noAbstractFileSystemExceptionMessage) => val newException = DeltaErrors.incorrectLogStoreImplementationException(e) + logError(newException.getMessage, newException.getCause) throw newException } if (!overwrite && fc.util.exists(path)) { diff --git a/standalone/src/test/resources/log4j.properties b/standalone/src/test/resources/log4j.properties new file mode 100644 index 00000000000..ed93eca2537 --- /dev/null +++ b/standalone/src/test/resources/log4j.properties @@ -0,0 +1,48 @@ +# +# Copyright (2020) The Delta Lake Project Authors. +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN From 764f7cbd8962708d509e3956e6867f39345247f9 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 20 Oct 2021 14:12:10 -0700 Subject: [PATCH 106/291] Add SchemaUtilsSuite; add two new public APIs to StructType (add); add test to OptTxnSuite (#178) --- .../io/delta/standalone/types/StructType.java | 39 +++ .../internal/util/SchemaMergingUtils.scala | 2 +- .../OptimisticTransactionLegacySuite.scala | 9 + .../internal/SchemaUtilsSuite.scala | 230 ++++++++++++++++++ 4 files changed, 279 insertions(+), 1 deletion(-) create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index ed7ee17b294..60e9dc910bd 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -53,12 +53,51 @@ public final class StructType extends DataType { private final StructField[] fields; private final HashMap nameToField; + public StructType() { + this(new StructField[0]); + } + public StructType(StructField[] fields) { this.fields = fields; this.nameToField = new HashMap<>(); Arrays.stream(fields).forEach(field -> nameToField.put(field.getName(), field)); } + /** + * Creates a new {@link StructType} by adding a new field. + * + *
{@code
+     * StructType schema = new StructType()
+     *     .add(new StructField("a", new IntegerType(), true))
+     *     .add(new StructField("b", new LongType(), false))
+     *     .add(new StructField("c", new StringType(), true))
+     * }
+ * @param field The new field to add. + * @return The new {@link StructType}. + */ + public StructType add(StructField field) { + StructField[] newFields = Arrays.copyOf(fields, fields.length + 1); + newFields[newFields.length - 1] = field; + return new StructType(newFields); + } + + /** + * Creates a new {@link StructType} by adding a new nullable field with no metadata. + * + *
{@code
+     * StructType schema = new StructType()
+     *     .add("a", new IntegerType())
+     *     .add("b", new LongType())
+     *     .add("c", new StringType())
+     * }
+ * @param fieldName The name of the new field to add. + * @return The new {@link StructType}. + */ + public StructType add(String fieldName, DataType dataType) { + StructField newField = new StructField(fieldName, dataType); + return add(newField); + } + /** * @return array of fields */ diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala index 5ee415a1ac7..8d2522936b9 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaMergingUtils.scala @@ -40,7 +40,7 @@ private[internal] object SchemaMergingUtils { def recurseIntoComplexTypes(complexType: DataType): Seq[Seq[String]] = { complexType match { case s: StructType => explode(s) - case a: ArrayType => recurseIntoComplexTypes(a.getElementType) + case a: ArrayType => recurseIntoComplexTypes(a.getElementType).map(Seq("element") ++ _) case m: MapType => recurseIntoComplexTypes(m.getKeyType).map(Seq("key") ++ _) ++ recurseIntoComplexTypes(m.getValueType).map(Seq("value") ++ _) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 5c759c2b7cb..3a4dea62410 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -481,6 +481,15 @@ class OptimisticTransactionLegacySuite extends FunSuite { "Found duplicate column(s)") } + test("can't have duplicate column names - case insensitive") { + val schema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("COL1", new StringType(), true) + )) + testMetadata[RuntimeException](Metadata(schemaString = schema.toJson), + "Found duplicate column(s)") + } + test("column names (both data and partition) must be acceptable by parquet") { val schema = new StructType(Array(new StructField("bad;column,name", new IntegerType(), true))) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala new file mode 100644 index 00000000000..e7be1c005b5 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala @@ -0,0 +1,230 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.util.Locale + +import io.delta.standalone.exceptions.DeltaStandaloneException +import io.delta.standalone.types.{ArrayType, IntegerType, MapType, StringType, StructType} +import io.delta.standalone.internal.util.SchemaUtils +import io.delta.standalone.internal.util.SchemaMergingUtils.checkColumnNameDuplication + +import org.scalatest.FunSuite + +class SchemaUtilsSuite extends FunSuite { + + private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { + val e = intercept[DeltaStandaloneException] { + f + } + val msg = e.getMessage.toLowerCase(Locale.ROOT) + assert(shouldContain.map(_.toLowerCase(Locale.ROOT)).forall(msg.contains), + s"Error message '$msg' didn't contain: $shouldContain") + } + + /////////////////////////////////////////////////////////////////////////// + // Duplicate Column Checks + /////////////////////////////////////////////////////////////////////////// + + test("duplicate column name in top level") { + val schema = new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupColName", new StringType()) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in top level - case sensitivity") { + val schema = new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupCOLNAME", new StringType()) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name for nested column + non-nested column") { + val schema = new StructType() + .add("dupColName", new StructType() + .add("a", new IntegerType()) + .add("b", new IntegerType())) + .add("dupColName", new IntegerType()) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name for nested column + non-nested column - case sensitivity") { + val schema = new StructType() + .add("dupColName", new StructType() + .add("a", new IntegerType()) + .add("b", new IntegerType())) + .add("dupCOLNAME", new IntegerType()) + expectFailure("dupCOLNAME") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested level") { + val schema = new StructType() + .add("top", new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupColName", new StringType()) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested level - case sensitivity") { + val schema = new StructType() + .add("top", new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupCOLNAME", new StringType()) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested level") { + val schema = new StructType() + .add("top", new StructType() + .add("b", new StructType() + .add("dupColName", new StringType()) + .add("c", new IntegerType()) + .add("dupColName", new StringType())) + .add("d", new IntegerType()) + ) + expectFailure("top.b.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested array") { + val schema = new StructType() + .add("top", new StructType() + .add("b", new ArrayType( + new ArrayType(new StructType() + .add("dupColName", new StringType()) + .add("c", new IntegerType()) + .add("dupColName", new StringType()), + true), + true)) + .add("d", new IntegerType()) + ) + expectFailure("top.b.element.element.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested map") { + val keyType = new StructType() + .add("dupColName", new IntegerType()) + .add("d", new StringType()) + expectFailure("top.b.key.dupColName") { + val schema = new StructType() + .add("top", new StructType() + .add("b", new MapType(keyType.add("dupColName", new StringType()), keyType, true)) + ) + checkColumnNameDuplication(schema, "") + } + expectFailure("top.b.value.dupColName") { + val schema = new StructType() + .add("top", new StructType() + .add("b", new MapType(keyType, keyType.add("dupColName", new StringType()), true)) + ) + checkColumnNameDuplication(schema, "") + } + // This is okay + val schema = new StructType() + .add("top", new StructType() + .add("b", new MapType(keyType, keyType, true)) + ) + checkColumnNameDuplication(schema, "") + } + + test("duplicate column name in nested array") { + val schema = new StructType() + .add("top", new ArrayType(new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupColName", new StringType()), true) + ) + expectFailure("top.element.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested array - case sensitivity") { + val schema = new StructType() + .add("top", new ArrayType(new StructType() + .add("dupColName", new IntegerType()) + .add("b", new IntegerType()) + .add("dupCOLNAME", new StringType()), true) + ) + expectFailure("top.element.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("non duplicate column because of back tick") { + val schema = new StructType() + .add("top", new StructType() + .add("a", new IntegerType()) + .add("b", new IntegerType())) + .add("top.a", new IntegerType()) + checkColumnNameDuplication(schema, "") + } + + test("non duplicate column because of back tick - nested") { + val schema = new StructType() + .add("first", new StructType() + .add("top", new StructType() + .add("a", new IntegerType()) + .add("b", new IntegerType())) + .add("top.a", new IntegerType())) + checkColumnNameDuplication(schema, "") + } + + test("duplicate column with back ticks - nested") { + val schema = new StructType() + .add("first", new StructType() + .add("top.a", new StringType()) + .add("b", new IntegerType()) + .add("top.a", new IntegerType())) + expectFailure("first.`top.a`") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column with back ticks - nested and case sensitivity") { + val schema = new StructType() + .add("first", new StructType() + .add("TOP.a", new StringType()) + .add("b", new IntegerType()) + .add("top.a", new IntegerType())) + expectFailure("first.`top.a`") { checkColumnNameDuplication(schema, "") } + } + + /////////////////////////////////////////////////////////////////////////// + // checkFieldNames + /////////////////////////////////////////////////////////////////////////// + + test("check non alphanumeric column characters") { + val badCharacters = " ,;{}()\n\t=" + val goodCharacters = "#.`!@$%^&*~_<>?/:" + + badCharacters.foreach { char => + Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => + val e = intercept[DeltaStandaloneException] { + SchemaUtils.checkFieldNames(Seq(name)) + } + assert(e.getMessage.contains("invalid character")) + } + } + + goodCharacters.foreach { char => + // no issues here + SchemaUtils.checkFieldNames(Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString)) + } + } + +} From c569297fd5c4d12ab8b9e29a380c4535b9d7e8dd Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 22 Oct 2021 10:29:20 -0700 Subject: [PATCH 107/291] [DSW] [26] Misc Fixes 4 (#180) --- .../io/delta/standalone/types/StructType.java | 9 +- .../internal/OptimisticTransactionImpl.scala | 32 ++++- .../internal/exception/DeltaErrors.scala | 10 ++ .../internal/util/CaseInsensitiveMap.scala | 63 +++++++++ .../internal/util/SchemaUtils.scala | 124 ++++++++++++++---- .../OptimisticTransactionLegacySuite.scala | 7 +- .../internal/OptimisticTransactionSuite.scala | 81 +++++++++++- .../internal/SchemaUtilsSuite.scala | 6 + 8 files changed, 296 insertions(+), 36 deletions(-) create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/CaseInsensitiveMap.scala diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index 60e9dc910bd..eed5072bca5 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -94,7 +94,7 @@ public StructType add(StructField field) { * @return The new {@link StructType}. */ public StructType add(String fieldName, DataType dataType) { - StructField newField = new StructField(fieldName, dataType); + StructField newField = new StructField(fieldName, dataType, true); return add(newField); } @@ -112,6 +112,13 @@ public String[] getFieldNames() { return Arrays.stream(fields).map(StructField::getName).toArray(String[]::new); } + /** + * @return the number of fields + */ + public int length() { + return fields.length; + } + /** * @param fieldName the name of the desired {@link StructField}, not null * @return the {@code link} with the given name, not null diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index 2d613433d8f..a15e0a6c67f 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -22,10 +22,11 @@ import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import io.delta.standalone.{CommitResult, DeltaScan, Operation, OptimisticTransaction, NAME, VERSION} +import io.delta.standalone.{CommitResult, DeltaScan, NAME, Operation, OptimisticTransaction, VERSION} import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ} import io.delta.standalone.exceptions.DeltaStandaloneException import io.delta.standalone.expressions.{Expression, Literal} +import io.delta.standalone.types.StructType import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile} import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.logging.Logging @@ -221,7 +222,10 @@ private[internal] class OptimisticTransactionImpl( assert(metadataChanges.length <= 1, "Cannot change the metadata more than once in a transaction.") - metadataChanges.foreach(m => verifyNewMetadata(m)) + metadataChanges.foreach { m => + verifyNewMetadata(m) + verifySchemaCompatibility(snapshot.metadataScala.schema, m.schema, actions) + } finalActions = newProtocol.toSeq ++ finalActions @@ -431,6 +435,30 @@ private[internal] class OptimisticTransactionImpl( Protocol.checkMetadataProtocolProperties(metadata, protocol) } + /** + * We want to check that the [[newSchema]] is compatible with the [[existingSchema]]. + * + * If the table is empty, or if the current commit is removing all the files in the table, + * then we do not need to perform this compatibility check. + */ + private def verifySchemaCompatibility( + existingSchema: StructType, + newSchema: StructType, + actions: Seq[Action]): Unit = { + val tableEmpty = snapshot.numOfFiles == 0 + + lazy val allCurrentFilesRemoved = { + val removeFiles = actions.collect { case r: RemoveFile => r } + removeFiles.map(_.path).toSet == snapshot.allFilesScala.map(_.path).toSet + } + + if (tableEmpty || allCurrentFilesRemoved) return + + if (!SchemaUtils.isWriteCompatible(existingSchema, newSchema)) { + throw DeltaErrors.schemaChangedException(existingSchema, newSchema) + } + } + /** * Returns true if we should checkpoint the version that has just been committed. */ diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index 201f9f2ae7a..cd3be32dc25 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -290,6 +290,16 @@ private[internal] object DeltaErrors { new DeltaStandaloneException(s"Unknown configuration was specified: $confKey") } + def schemaChangedException(oldSchema: StructType, newSchema: StructType): Throwable = { + val msg = + s"""Detected incompatible schema change: + |old schema: ${oldSchema.getTreeString} + | + |new schema: ${newSchema.getTreeString} + """.stripMargin + new IllegalStateException(msg) + } + /////////////////////////////////////////////////////////////////////////// // Helper Methods /////////////////////////////////////////////////////////////////////////// diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/CaseInsensitiveMap.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/CaseInsensitiveMap.scala new file mode 100644 index 00000000000..9ee04dad2fd --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/CaseInsensitiveMap.scala @@ -0,0 +1,63 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import java.util.Locale + +/** + * Builds a map in which keys are case insensitive. Input map can be accessed for cases where + * case-sensitive information is required. The primary constructor is marked private to avoid + * nested case-insensitive map creation, otherwise the keys in the original map will become + * case-insensitive in this scenario. + * Note: CaseInsensitiveMap is serializable. However, after transformation, e.g. `filterKeys()`, + * it may become not serializable. + */ +private[internal] class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) + extends Map[String, T] with Serializable { + + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + + val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) + + override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT)) + + override def contains(k: String): Boolean = + keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) + + override def +[B1 >: T](kv: (String, B1)): CaseInsensitiveMap[B1] = { + new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(kv._1)) + kv) + } + + def ++(xs: TraversableOnce[(String, T)]): CaseInsensitiveMap[T] = { + xs.foldLeft(this)(_ + _) + } + + override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator + + override def -(key: String): Map[String, T] = { + new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) + } + + def toMap: Map[String, T] = originalMap +} + +object CaseInsensitiveMap { + def apply[T](params: Map[String, T]): CaseInsensitiveMap[T] = params match { + case caseSensitiveMap: CaseInsensitiveMap[T] => caseSensitiveMap + case _ => new CaseInsensitiveMap(params) + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala index f4aaac20768..ce1329c5179 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala @@ -35,35 +35,6 @@ private[internal] object SchemaUtils { names.find(_.contains("\n")).foreach(col => throw DeltaErrors.invalidColumnName(col)) } - /** - * This is a simpler version of Delta OSS SchemaUtils::typeAsNullable. Instead of returning the - * nullable DataType, returns true if the input `dt` matches the nullable DataType. - */ - private def matchesNullableType(dt: DataType): Boolean = dt match { - case s: StructType => s.getFields.forall { field => - field.isNullable && matchesNullableType(field.getDataType) - } - - case a: ArrayType => a.getElementType match { - case s: StructType => - a.containsNull() && matchesNullableType(s) - case _ => - a.containsNull() - } - - case m: MapType => (m.getKeyType, m.getValueType) match { - case (s1: StructType, s2: StructType) => - m.valueContainsNull() && matchesNullableType(s1) && matchesNullableType(s2) - case (s1: StructType, _) => - m.valueContainsNull() && matchesNullableType(s1) - case (_, s2: StructType) => - m.valueContainsNull() && matchesNullableType(s2) - case _ => true - } - - case _ => true - } - /** * Go through the schema to look for unenforceable NOT NULL constraints and throw when they're * encountered. @@ -106,6 +77,101 @@ private[internal] object SchemaUtils { traverseColumns(Seq.empty, schema) } + /** + * As the Delta table updates, the schema may change as well. This method defines whether a new + * schema can replace a pre-existing schema of a Delta table. Our rules are to return false if + * the new schema: + * - Drops any column that is present in the current schema + * - Converts nullable=true to nullable=false for any column + * - Changes any datatype + */ + def isWriteCompatible(existingSchema: StructType, newSchema: StructType): Boolean = { + + def isDatatypeWriteCompatible(_existingType: DataType, _newType: DataType): Boolean = { + (_existingType, _newType) match { + case (e: StructType, n: StructType) => + isWriteCompatible(e, n) + case (e: ArrayType, n: ArrayType) => + // if existing elements are nullable, so should be the new element + (!e.containsNull() || n.containsNull()) && + isDatatypeWriteCompatible(e.getElementType, n.getElementType) + case (e: MapType, n: MapType) => + // if existing value is nullable, so should be the new value + (!e.valueContainsNull || n.valueContainsNull) && + isDatatypeWriteCompatible(e.getKeyType, n.getKeyType) && + isDatatypeWriteCompatible(e.getValueType, n.getValueType) + case (a, b) => a == b + } + } + + def isStructWriteCompatible(_existingSchema: StructType, _newSchema: StructType): Boolean = { + val existing = toFieldMap(_existingSchema.getFields) + // scalastyle:off caselocale + val existingFieldNames = _existingSchema.getFieldNames.map(_.toLowerCase).toSet + assert(existingFieldNames.size == _existingSchema.length, + "Delta tables don't allow field names that only differ by case") + val newFields = _newSchema.getFieldNames.map(_.toLowerCase).toSet + assert(newFields.size == _newSchema.length, + "Delta tables don't allow field names that only differ by case") + // scalastyle:on caselocale + + if (!existingFieldNames.subsetOf(newFields)) { + // Dropped a column that was present in the DataFrame schema + return false + } + _newSchema.getFields.forall { newField => + // new fields are fine, they just won't be returned + existing.get(newField.getName).forall { existingField => + // we know the name matches modulo case - now verify exact match + (existingField.getName == newField.getName + // if existing value is nullable, so should be the new value + && (!existingField.isNullable || newField.isNullable) + // and the type of the field must be compatible, too + && isDatatypeWriteCompatible(existingField.getDataType, newField.getDataType)) + } + } + } + + isStructWriteCompatible(existingSchema, newSchema) + } + + /////////////////////////////////////////////////////////////////////////// + // Helper Methods + /////////////////////////////////////////////////////////////////////////// + + private def toFieldMap(fields: Seq[StructField]): Map[String, StructField] = { + CaseInsensitiveMap(fields.map(field => field.getName -> field).toMap) + } + + /** + * This is a simpler version of Delta OSS SchemaUtils::typeAsNullable. Instead of returning the + * nullable DataType, returns true if the input `dt` matches the nullable DataType. + */ + private def matchesNullableType(dt: DataType): Boolean = dt match { + case s: StructType => s.getFields.forall { field => + field.isNullable && matchesNullableType(field.getDataType) + } + + case a: ArrayType => a.getElementType match { + case s: StructType => + a.containsNull() && matchesNullableType(s) + case _ => + a.containsNull() + } + + case m: MapType => (m.getKeyType, m.getValueType) match { + case (s1: StructType, s2: StructType) => + m.valueContainsNull() && matchesNullableType(s1) && matchesNullableType(s2) + case (s1: StructType, _) => + m.valueContainsNull() && matchesNullableType(s1) + case (_, s2: StructType) => + m.valueContainsNull() && matchesNullableType(s2) + case _ => true + } + + case _ => true + } + private def prettyFieldName(columnPath: Seq[String]): String = columnPath.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 3a4dea62410..7c9c5239c19 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -579,14 +579,15 @@ class OptimisticTransactionLegacySuite extends FunSuite { ////////////////////////////////// test("concurrent metadata update should fail") { - withLog(Nil) { log => + withLog(actions = Nil, partitionCols = Nil) { log => val tx1 = log.startTransaction() + val tx2 = log.startTransaction() - tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "foo"))) + tx2.updateMetadata(ConversionUtils.convertMetadata(Metadata())) tx2.commit(Iterable().asJava, manualUpdate, engineInfo) assertThrows[MetadataChangedException] { - tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata(name = "bar"))) + tx1.updateMetadata(ConversionUtils.convertMetadata(Metadata())) tx1.commit(Iterable().asJava, manualUpdate, engineInfo) } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala index 3c9ebdcdcee..54884828156 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuite.scala @@ -16,11 +16,14 @@ package io.delta.standalone.internal +import java.util.Collections + import scala.collection.JavaConverters._ -import io.delta.standalone.actions.{CommitInfo, Protocol, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} +import io.delta.standalone.actions.{CommitInfo, Protocol, Action => ActionJ, AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ} import io.delta.standalone.internal.util.TestUtils._ import io.delta.standalone.DeltaLog +import io.delta.standalone.types.{IntegerType, StringType, StructField, StructType} import org.apache.hadoop.conf.Configuration @@ -183,4 +186,80 @@ class OptimisticTransactionSuite assert(getIsolationLevel(1) == "Serializable") } } + + private def testSchemaChange( + schema1: StructType, + schema2: StructType, + shouldThrow: Boolean, + initialActions: Seq[ActionJ] = addA :: Nil, + commitActions: Seq[ActionJ] = Nil): Unit = { + withTempDir { dir => + val metadata1 = MetadataJ.builder().schema(schema1).build() + val metadata2 = MetadataJ.builder().schema(schema2).build() + + val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) + + log.startTransaction().commit((initialActions :+ metadata1).asJava, op, engineInfo) + + if (shouldThrow) { + intercept[IllegalStateException] { + log.startTransaction().commit((commitActions :+ metadata2).asJava, op, engineInfo) + } + } else { + log.startTransaction().commit((commitActions :+ metadata2).asJava, op, engineInfo) + } + } + } + + // Note: See SchemaUtilsSuite for thorough isWriteCompatible(existingSchema, newSchema) unit tests + test("can change schema to valid schema") { + // col a is non-nullable + val schema1 = new StructType(Array(new StructField("a", new IntegerType(), false))) + + // add nullable field + val schema2 = schema1.add(new StructField("b", new IntegerType(), true)) + testSchemaChange(schema1, schema2, shouldThrow = false) + + // add non-nullable field + val schema3 = schema1.add(new StructField("b", new IntegerType(), false)) + testSchemaChange(schema1, schema3, shouldThrow = false) + + // relaxed nullability (from non-nullable to nullable) + val schema4 = new StructType(Array(new StructField("a", new IntegerType(), true))) + testSchemaChange(schema1, schema4, shouldThrow = false) + } + + // Note: See SchemaUtilsSuite for thorough isWriteCompatible(existingSchema, newSchema) unit tests + test("can't change schema to invalid schema - table non empty, files not removed") { + // col a is nullable + val schema1 = new StructType(Array(new StructField("a", new IntegerType(), true))) + + // drop a field + val schema2 = new StructType(Array()) + testSchemaChange(schema1, schema2, shouldThrow = true) + + // restricted nullability (from nullable to non-nullable) + val schema3 = new StructType(Array(new StructField("a", new IntegerType(), false))) + testSchemaChange(schema1, schema3, shouldThrow = true) + + // change of datatype + val schema4 = new StructType(Array(new StructField("a", new StringType(), true))) + testSchemaChange(schema1, schema4, shouldThrow = true) + } + + test("can change schema to 'invalid' schema - table empty or all files removed") { + val schema1 = new StructType(Array(new StructField("a", new IntegerType()))) + val schema2 = new StructType(Array(new StructField("a", new StringType()))) + val addC = new AddFileJ("c", Collections.emptyMap(), 1, 1, true, null, null) + + // change of datatype - table is empty + testSchemaChange(schema1, schema2, shouldThrow = false, initialActions = Nil) + + // change of datatype - all files are removed and new file added + testSchemaChange(schema1, schema2, shouldThrow = false, commitActions = removeA :: addC :: Nil) + + // change of datatype - not all files are removed (should throw) + testSchemaChange(schema1, schema2, shouldThrow = true, initialActions = addA :: addB :: Nil, + commitActions = removeA :: Nil) + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala index e7be1c005b5..2ba5680ab4a 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala @@ -227,4 +227,10 @@ class SchemaUtilsSuite extends FunSuite { } } + /////////////////////////////////////////////////////////////////////////// + // Write Compatibility Checks + /////////////////////////////////////////////////////////////////////////// + + // TODO copy from Delta OSS + } From 54a28a99f6cc7a69004c6b68322d9c319842eeeb Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 22 Oct 2021 11:50:15 -0700 Subject: [PATCH 108/291] [DSW] [27] Misc fixes 5 - schema utils suite (#181) --- .../io/delta/standalone/types/StructType.java | 17 ++ .../internal/SchemaUtilsSuite.scala | 172 +++++++++++++++++- 2 files changed, 184 insertions(+), 5 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index eed5072bca5..aa13d53c54c 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -98,6 +98,23 @@ public StructType add(String fieldName, DataType dataType) { return add(newField); } + /** + * Creates a new {@link StructType} by adding a new field with no metadata. + * + *
{@code
+     * StructType schema = new StructType()
+     *     .add("a", new IntegerType(), true)
+     *     .add("b", new LongType(), false)
+     *     .add("c", new StringType(), true)
+     * }
+ * @param fieldName The name of the new field to add. + * @return The new {@link StructType}. + */ + public StructType add(String fieldName, DataType dataType, boolean nullable) { + StructField newField = new StructField(fieldName, dataType, nullable); + return add(newField); + } + /** * @return array of fields */ diff --git a/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala index 2ba5680ab4a..677184ba53d 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/SchemaUtilsSuite.scala @@ -19,8 +19,8 @@ package io.delta.standalone.internal import java.util.Locale import io.delta.standalone.exceptions.DeltaStandaloneException -import io.delta.standalone.types.{ArrayType, IntegerType, MapType, StringType, StructType} -import io.delta.standalone.internal.util.SchemaUtils +import io.delta.standalone.types._ +import io.delta.standalone.internal.util.SchemaUtils._ import io.delta.standalone.internal.util.SchemaMergingUtils.checkColumnNameDuplication import org.scalatest.FunSuite @@ -215,7 +215,7 @@ class SchemaUtilsSuite extends FunSuite { badCharacters.foreach { char => Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => val e = intercept[DeltaStandaloneException] { - SchemaUtils.checkFieldNames(Seq(name)) + checkFieldNames(Seq(name)) } assert(e.getMessage.contains("invalid character")) } @@ -223,7 +223,7 @@ class SchemaUtilsSuite extends FunSuite { goodCharacters.foreach { char => // no issues here - SchemaUtils.checkFieldNames(Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString)) + checkFieldNames(Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString)) } } @@ -231,6 +231,168 @@ class SchemaUtilsSuite extends FunSuite { // Write Compatibility Checks /////////////////////////////////////////////////////////////////////////// - // TODO copy from Delta OSS + /** + * Tests change of datatype within a schema. + * - the make() function is a "factory" function to create schemas that vary only by the + * given datatype in a specific position in the schema. + * - other tests will call this method with different make() functions to test datatype + * incompatibility in all the different places within a schema (in a top-level struct, + * in a nested struct, as the element type of an array, etc.) + */ + def testDatatypeChange(scenario: String)(make: DataType => StructType): Unit = { + val schemas = Map( + ("int", make(new IntegerType())), + ("string", make(new StringType())), + ("struct", make(new StructType().add("a", new StringType()))), + ("array", make(new ArrayType(new IntegerType(), true))), // containsNull + ("map", make(new MapType(new StringType(), new FloatType(), true))) // valueContainsNull + ) + test(s"change of datatype should fail write compatibility - $scenario") { + for (a <- schemas.keys; b <- schemas.keys if a != b) { + assert(!isWriteCompatible(schemas(a), schemas(b)), + s"isWriteCompatible should have failed for: ${schemas(a)}, ${schemas(b)}") + } + } + } + + /** + * Tests change of nullability within a schema. + * - ALLOWED: making a non-nullable field nullable + * - NOT ALLOWED: making a nullable field non-nullable + * + * Implementation details: + * - the make() function is a "factory" function to create schemas that vary only by the + * nullability (of a field, array element, or map values) in a specific position in the schema. + * - other tests will call this method with different make() functions to test nullability + * incompatibility in all the different places within a schema (in a top-level struct, + * in a nested struct, for the element type of an array, etc.) + */ + def testNullability (scenario: String)(make: Boolean => StructType): Unit = { + val nullable = make(true) + val nonNullable = make(false) + + // restricted: nullable=true ==> nullable=false + test(s"restricted nullability should fail write compatibility - $scenario") { + assert(!isWriteCompatible(nullable, nonNullable)) + } + + // relaxed: nullable=false ==> nullable=true + test(s"relaxed nullability should not fail write compatibility - $scenario") { + assert(isWriteCompatible(nonNullable, nullable)) + } + } + + /** + * Tests for fields of a struct: adding/dropping fields, changing nullability, case variation + * - The make() function is a "factory" method to produce schemas. It takes a function that + * mutates a struct (for example, but adding a column, or it could just not make any change). + * - Following tests will call this method with different factory methods, to mutate the + * various places where a struct can appear (at the top-level, nested in another struct, + * within an array, etc.) + * - This allows us to have one shared code to test compatibility of a struct field in all the + * different places where it may occur. + */ + def testColumnVariations(scenario: String) + (make: (StructType => StructType) => StructType): Unit = { + + // generate one schema without extra column, one with, one nullable, and one with mixed case + val withoutExtra = make(struct => struct) // produce struct WITHOUT extra field + val withExtraNullable = make(struct => struct.add("extra", new StringType())) + val withExtraMixedCase = make(struct => struct.add("eXtRa", new StringType())) + val withExtraNonNullable = + make(struct => struct.add("extra", new StringType(), false)) // nullable = false + + test(s"dropping a field should fail write compatibility - $scenario") { + assert(!isWriteCompatible(withExtraNullable, withoutExtra)) + } + test(s"adding a nullable field should not fail write compatibility - $scenario") { + assert(isWriteCompatible(withoutExtra, withExtraNullable)) + } + test(s"adding a non-nullable field should not fail write compatibility - $scenario") { + assert(isWriteCompatible(withoutExtra, withExtraNonNullable)) + } + test(s"case variation of field name should fail write compatibility - $scenario") { + assert(!isWriteCompatible(withExtraNullable, withExtraMixedCase)) + } + + testNullability(scenario) { nullable => + make(struct => struct.add("extra", new StringType(), nullable)) + } + testDatatypeChange(scenario) { datatype => + make(struct => struct.add("extra", datatype)) + } + } + + // -------------------------------------------------------------------- + // tests for all kinds of places where a field can appear in a struct + // -------------------------------------------------------------------- + + testColumnVariations("top level")( + f => f(new StructType().add("a", new IntegerType()))) + + testColumnVariations("nested struct")( + f => new StructType() + .add("a", f(new StructType().add("b", new IntegerType())))) + + testColumnVariations("nested in array")( + f => new StructType() + .add("array", new ArrayType( + f(new StructType().add("b", new IntegerType())), true) // containsNull + ) + ) + + testColumnVariations("nested in map key")( + f => new StructType() + .add("map", new MapType( + f(new StructType().add("b", new IntegerType())), + new StringType(), true) // valueContainsNull + ) + ) + + testColumnVariations("nested in map value")( + f => new StructType() + .add("map", new MapType( + new StringType(), + f(new StructType().add("b", new IntegerType())), true) // valueContainsNull + ) + ) + + // -------------------------------------------------------------------- + // tests for data type change in places other than struct + // -------------------------------------------------------------------- + + testDatatypeChange("array element")( + datatype => new StructType() + .add("array", new ArrayType(datatype, true))) // containsNull + + testDatatypeChange("map key")( + datatype => new StructType() + .add("map", new MapType(datatype, new StringType(), true))) // valueContainsNull + + testDatatypeChange("map value")( + datatype => new StructType() + .add("map", new MapType(new StringType(), datatype, true))) // valueContainsNull + + // -------------------------------------------------------------------- + // tests for nullability change in places other than struct + // -------------------------------------------------------------------- + + testNullability("array contains null")( + containsNull => new StructType() + .add("array", new ArrayType(new StringType(), containsNull))) + + testNullability("map contains null values")( + valueContainsNull => new StructType() + .add("map", new MapType(new IntegerType(), new StringType(), valueContainsNull))) + + testNullability("map nested in array")( + valueContainsNull => new StructType() + .add("map", new ArrayType( + new MapType(new IntegerType(), new StringType(), valueContainsNull), true))) // containsNull + testNullability("array nested in map")( + containsNull => new StructType() + .add("map", new MapType( + new IntegerType(), + new ArrayType(new StringType(), containsNull), true))) // valueContainsNull } From 75558ad1999b507e95640457c953e19770e8ac65 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Fri, 22 Oct 2021 11:57:22 -0700 Subject: [PATCH 109/291] [DSW] [29] Operation Parameters fix; document Operation metrics (#185) --- .../java/io/delta/standalone/Operation.java | 197 ++++++++++++++++-- .../internal/OptimisticTransactionImpl.scala | 14 +- .../OptimisticTransactionLegacySuite.scala | 14 +- 3 files changed, 195 insertions(+), 30 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index f66efcb4872..1a62a7b1d9f 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -1,18 +1,26 @@ package io.delta.standalone; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; import java.util.Optional; -import java.util.Set; /** * An operation that can be performed on a Delta table. * * An operation is tracked as the first line in delta logs, and powers `DESCRIBE HISTORY` for Delta * tables. + * + * Operations must be constructed using one of the {@link Operation.Name} types below. + * As well, optional {@link Metrics} values are given below. */ public final class Operation { + /////////////////////////////////////////////////////////////////////////// + // Operation Names + /////////////////////////////////////////////////////////////////////////// + /** * Supported operation types. */ @@ -80,34 +88,192 @@ public String toString() { } } + /////////////////////////////////////////////////////////////////////////// + // Operation Metrics + /////////////////////////////////////////////////////////////////////////// + + /** + * Some possible operation metrics and their suggested corresponding operation types. + * These are purely exemplary, and users may user whichever metrics best fit their application. + */ + public static class Metrics { + /** + * Number of files written. + * + * Usually used with WRITE operation. + */ + public static final String numFiles = "numFiles"; + + /** + * Size in bytes of the written contents. + * + * Usually used with WRITE, STREAMING_UPDATE operations. + */ + public static final String numOutputBytes = "numOutputBytes"; + + /** + * Number of rows written. + * + * Usually used with WRITE, STREAMING_UPDATE, MERGE operations. + */ + public static final String numOutputRows = "numOutputRows"; + + /** + * Number of files added. + * + * Usually used with STREAMING_UPDATE, DELETE, UPDATE operations. + */ + public static final String numAddedFiles = "numAddedFiles"; + + /** + * Number of files removed. + * + * Usually used with STREAMING_UPDATE, DELETE, DELETE_PARTITIONS, TRUNCATE, + * UPDATE operations. + */ + public static final String numRemovedFiles = "numRemovedFiles"; + + /** + * Number of rows removed. + * + * Usually used with DELETE operation. + */ + public static final String numDeletedRows = "numDeletedRows"; + + /** + * Number of rows copied in the process of deleting files. + * + * Usually used with DELETE, UPDATE operations. + */ + public static final String numCopiedRows = "numCopiedRows"; + + /** + * Time taken to execute the entire operation. + * + * Usually used with DELETE, DELETE_PARTITIONS, TRUNCATE, MERGE, UPDATE operations. + */ + public static final String executionTimeMs = "executionTimeMs"; + + /** + * Time taken to scan the files for matches. + * + * Usually used with DELETE, DELETE_PARTITIONS, MERGE, UPDATE operations. + */ + public static final String scanTimeMs = "scanTimeMs"; + + /** + * Time taken to rewrite the matched files. + * + * Usually used with DELETE, DELETE_PARTITIONS, MERGE, UPDATE operations. + */ + public static final String rewriteTimeMs = "rewriteTimeMs"; + + /** + * Number of parquet files that have been converted. + * + * Usually used with the CONVERT operation. + */ + public static final String numConvertedFiles = "numConvertedFiles"; + + /** + * Number of rows in the source table. + * + * Usually used with the MERGE operation. + */ + public static final String numSourceRows = "numSourceRows"; + + /** + * Number of rows inserted into the target table. + * + * Usually used with the MERGE operation. + */ + public static final String numTargetRowsInserted = "numTargetRowsInserted"; + + /** + * Number of rows updated in the target table. + * + * Usually used with the MERGE operation. + */ + public static final String numTargetRowsUpdated = "numTargetRowsUpdated"; + + /** + * Number of rows deleted in the target table. + * + * Usually used with the MERGE operation. + */ + public static final String numTargetRowsDeleted = "numTargetRowsDeleted"; + + /** + * Number of target rows copied. + * + * Usually used with the MERGE operation. + */ + public static final String numTargetRowsCopied = "numTargetRowsCopied"; + + /** + * Number files added to the sink(target). + * + * Usually used with the MERGE operation. + */ + public static final String numTargetFilesAdded = "numTargetFilesAdded"; + + /** + * Number of files removed from the sink(target). + * + * Usually used with the MERGE operation. + */ + public static final String numTargetFilesRemoved = "numTargetFilesRemoved"; + + /** + * Number of rows updated. + * + * Usually used with the UPDATE operation. + */ + public static final String numUpdatedRows = "numUpdatedRows"; + } + + /////////////////////////////////////////////////////////////////////////// + // Operation internals, constructors, and external APIs + /////////////////////////////////////////////////////////////////////////// + + @Nonnull private final Name name; - private final Map parameters; - private final Map operationMetrics; + + @Nullable + private final Map parameters; + + @Nullable + private final Map metrics; + + @Nonnull private final Optional userMetadata; - public Operation(Name name) { + public Operation(@Nonnull Name name) { this(name, Collections.emptyMap(), Collections.emptyMap(), Optional.empty()); } - public Operation(Name name, Map parameters) { + public Operation(@Nonnull Name name, @Nullable Map parameters) { this(name, parameters, Collections.emptyMap(), Optional.empty()); } - public Operation(Name name, Map parameters, Map operationMetrics) { - this(name, parameters, operationMetrics, Optional.empty()); + public Operation(@Nonnull Name name, @Nullable Map parameters, + @Nullable Map metrics) { + this(name, parameters, metrics, Optional.empty()); } - public Operation(Name name, Map parameters, Map operationMetrics, - Optional userMetadata) { + public Operation(@Nonnull Name name, @Nullable Map parameters, + @Nullable Map metrics, + @Nonnull Optional userMetadata) { this.name = name; this.parameters = parameters; - this.operationMetrics = operationMetrics; + this.metrics = metrics; this.userMetadata = userMetadata; } /** * @return operation name */ + @Nonnull public Name getName() { return name; } @@ -115,21 +281,24 @@ public Name getName() { /** * @return operation parameters */ - public Map getParameters() { + @Nullable + public Map getParameters() { return parameters != null ? Collections.unmodifiableMap(parameters) : null; } /** * @return operation metrics */ - public Map getOperationMetrics() { - return operationMetrics != null ? Collections.unmodifiableMap(operationMetrics) : null; + @Nullable + public Map getMetrics() { + return metrics != null ? Collections.unmodifiableMap(metrics) : null; } /** * @return user metadata for this operation */ + @Nonnull public Optional getUserMetadata() { - return null == userMetadata ? Optional.empty() : userMetadata; + return userMetadata; } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala index a15e0a6c67f..16063ac5e92 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala @@ -35,7 +35,7 @@ import io.delta.standalone.internal.util.{ConversionUtils, FileNames, SchemaMerg private[internal] class OptimisticTransactionImpl( deltaLog: DeltaLogImpl, snapshot: SnapshotImpl) extends OptimisticTransaction with Logging { - import OptimisticTransactionImpl._ + val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 /** Used for logging */ private val txnId = UUID.randomUUID().toString @@ -121,12 +121,12 @@ private[internal] class OptimisticTransactionImpl( val commitInfo = CommitInfo( deltaLog.clock.getTimeMillis(), op.getName.toString, - null, // TODO: use operation jsonEncodedValues + if (op.getParameters == null) null else op.getParameters.asScala.toMap, Map.empty, Some(readVersion).filter(_ >= 0), Option(isolationLevelToUse.toString), Some(isBlindAppend), - Some(op.getOperationMetrics.asScala.toMap), + Some(op.getMetrics.asScala.toMap), if (op.getUserMetadata.isPresent) Some(op.getUserMetadata.get()) else None, Some(s"$engineInfo $NAME/$VERSION") ) @@ -508,11 +508,3 @@ private[internal] class OptimisticTransactionImpl( } } - -private[internal] object OptimisticTransactionImpl { - val DELTA_MAX_RETRY_COMMIT_ATTEMPTS = 10000000 // TODO: DeltaConfig this - -// def getOperationJsonEncodedParameters(op: Operation): Map[String, String] = { -// op.getParameters.asScala.mapValues(JsonUtils.toJson(_)).toMap -// } -} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala index 7c9c5239c19..183942bf0c9 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionLegacySuite.scala @@ -16,10 +16,12 @@ package io.delta.standalone.internal +import java.util.Collections + import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import io.delta.standalone.{DeltaLog, Operation, NAME, VERSION} +import io.delta.standalone.{DeltaLog, NAME, Operation, VERSION} import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ} import io.delta.standalone.exceptions.{ConcurrentAppendException, ConcurrentDeleteDeleteException, ConcurrentDeleteReadException, ConcurrentTransactionException, MetadataChangedException, ProtocolChangedException} import io.delta.standalone.expressions.{EqualTo, Literal} @@ -28,6 +30,7 @@ import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.util.{ConversionUtils, SchemaUtils} import io.delta.standalone.types._ import io.delta.standalone.internal.util.TestUtils._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -508,16 +511,17 @@ class OptimisticTransactionLegacySuite extends FunSuite { test("CommitInfo operation and engineInfo is persisted to the delta log") { withTempDir { dir => + val opParams = Collections.singletonMap(Operation.Metrics.numAddedFiles, "0") + val op = new Operation(Operation.Name.MANUAL_UPDATE, opParams) val log = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) - log.startTransaction().commit(Metadata() :: Nil, manualUpdate, engineInfo) + log.startTransaction().commit(Metadata() :: Nil, op, engineInfo) val log2 = DeltaLog.forTable(new Configuration(), dir.getCanonicalPath) val commitInfo = log2.getCommitInfoAt(0) assert(commitInfo.getEngineInfo.isPresent) assert(commitInfo.getEngineInfo.get() == s"$engineInfo $NAME/$VERSION") - assert(commitInfo.getOperation == manualUpdate.getName.toString) - - // TODO: test commitInfo.operationParameters + assert(commitInfo.getOperation == op.getName.toString) + assert(commitInfo.getOperationParameters.asScala == Map("numAddedFiles" -> "0")) } } From 8c5df277e536092274f456eea71142cde36a90b8 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Fri, 22 Oct 2021 15:16:43 -0700 Subject: [PATCH 110/291] structTypeToParquet converter (#171) This PR introduces a schema converter for our Java StructType to Parquet MessageType. The converter is implemented internally in Scala, and exposed through ParquetSchemaConverter.java in io.delta.standalone.util. Tests are copied over from Apache Spark. --- .../util/ParquetSchemaConverter.java | 110 ++++++ .../util/ParquetSchemaConverter.scala | 318 ++++++++++++++++++ .../ParquetSchemaConverterSuite.scala | 305 +++++++++++++++++ 3 files changed, 733 insertions(+) create mode 100644 standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/util/ParquetSchemaConverter.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ParquetSchemaConverterSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java new file mode 100644 index 00000000000..335cdba389b --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java @@ -0,0 +1,110 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.util; + +import org.apache.parquet.schema.MessageType; + +import io.delta.standalone.internal.util.SparkToParquetSchemaConverter; +import io.delta.standalone.types.StructType; + +/** + * Converter class to convert {@link StructType} to Parquet {@link MessageType}. + */ +public class ParquetSchemaConverter { + + /** + * Represents Parquet timestamp types. + * - INT96 is a non-standard but commonly used timestamp type in Parquet. + * - TIMESTAMP_MICROS is a standard timestamp type in Parquet, which stores number of microseconds + * from the Unix epoch. + * - TIMESTAMP_MILLIS is also standard, but with millisecond precision, which means the microsecond + * portion of the timestamp value is truncated. + */ + public enum ParquetOutputTimestampType { + INT96, + TIMESTAMP_MICROS, + TIMESTAMP_MILLIS + } + + private static final Boolean writeLegacyParquetFormatDefault = false; + private static final ParquetOutputTimestampType outputTimestampTypeDefault = + ParquetOutputTimestampType.INT96; + + /** + * Convert a {@link StructType} to Parquet {@link MessageType}. + * + * @param schema the schema to convert + * @return {@code schema} as a Parquet {@link MessageType} + * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s) + */ + public static MessageType sparkToParquet(StructType schema) { + return new SparkToParquetSchemaConverter( + writeLegacyParquetFormatDefault, + outputTimestampTypeDefault).convert(schema); + } + + /** + * Convert a {@link StructType} to Parquet {@link MessageType}. + * + * @param schema the schema to convert + * @param writeLegacyParquetFormat Whether to use legacy Parquet format compatible with Spark + * 1.4 and prior versions when converting a {@link StructType} to a Parquet + * {@link MessageType}. When set to false, use standard format defined in parquet-format + * spec. + * @return {@code schema} as a Parquet {@link MessageType} + * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s) + */ + public static MessageType sparkToParquet(StructType schema, Boolean writeLegacyParquetFormat) { + return new SparkToParquetSchemaConverter( + writeLegacyParquetFormat, + outputTimestampTypeDefault).convert(schema); + } + + /** + * Convert a {@link StructType} to Parquet {@link MessageType}. + * + * @param schema the schema to convert + * @param outputTimestampType which parquet timestamp type to use when writing + * @return {@code schema} as a Parquet {@link MessageType} + * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s) + */ + public static MessageType sparkToParquet(StructType schema, ParquetOutputTimestampType outputTimestampType) { + return new SparkToParquetSchemaConverter( + writeLegacyParquetFormatDefault, + outputTimestampType).convert(schema); + } + + /** + * Convert a {@link StructType} to Parquet {@link MessageType}. + * + * @param schema the schema to convert + * @param writeLegacyParquetFormat Whether to use legacy Parquet format compatible with Spark + * 1.4 and prior versions when converting a {@link StructType} to a Parquet + * {@link MessageType}. When set to false, use standard format defined in parquet-format + * spec. + * @param outputTimestampType which parquet timestamp type to use when writing + * @return {@code schema} as a Parquet {@link MessageType} + * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s) + */ public static MessageType sparkToParquet( + StructType schema, + Boolean writeLegacyParquetFormat, + ParquetOutputTimestampType outputTimestampType) { + return new SparkToParquetSchemaConverter( + writeLegacyParquetFormat, + outputTimestampType).convert(schema); + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ParquetSchemaConverter.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ParquetSchemaConverter.scala new file mode 100644 index 00000000000..ab0adc9901b --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ParquetSchemaConverter.scala @@ -0,0 +1,318 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.util + +import org.apache.parquet.schema.{ConversionPatterns, MessageType, Type, Types} +import org.apache.parquet.schema.OriginalType._ +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ +import org.apache.parquet.schema.Type.Repetition._ + +import io.delta.standalone.types._ +import io.delta.standalone.util.ParquetSchemaConverter.ParquetOutputTimestampType + +/** + * This converter class is used to convert Spark SQL [[StructType]] to Parquet [[MessageType]]. + * + * @param writeLegacyParquetFormat Whether to use legacy Parquet format compatible with Spark 1.4 + * and prior versions when converting a [[StructType]] to a Parquet [[MessageType]]. + * When set to false, use standard format defined in parquet-format spec. This argument only + * affects Parquet write path. + * @param outputTimestampType which parquet timestamp type to use when writing. + */ +private[standalone] class SparkToParquetSchemaConverter( + writeLegacyParquetFormat: Boolean, + outputTimestampType: ParquetOutputTimestampType) { + + /** + * Converts a Spark SQL [[StructType]] to a Parquet [[MessageType]]. + */ + def convert(schema: StructType): MessageType = { + Types + .buildMessage() + .addFields(schema.getFields.map(convertField): _*) + .named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + } + + /** + * Converts a Spark SQL [[StructField]] to a Parquet [[Type]]. + */ + private def convertField(field: StructField): Type = { + convertField(field, if (field.isNullable) OPTIONAL else REQUIRED) + } + + private def convertField(field: StructField, repetition: Type.Repetition): Type = { + ParquetSchemaConverter.checkFieldName(field.getName) + + field.getDataType match { + // =================== + // Simple atomic types + // =================== + + case _: BooleanType => + Types.primitive(BOOLEAN, repetition).named(field.getName) + + case _: ByteType => + Types.primitive(INT32, repetition).as(INT_8).named(field.getName) + + case _: ShortType => + Types.primitive(INT32, repetition).as(INT_16).named(field.getName) + + case _: IntegerType => + Types.primitive(INT32, repetition).named(field.getName) + + case _: LongType => + Types.primitive(INT64, repetition).named(field.getName) + + case _: FloatType => + Types.primitive(FLOAT, repetition).named(field.getName) + + case _: DoubleType => + Types.primitive(DOUBLE, repetition).named(field.getName) + + case _: StringType => + Types.primitive(BINARY, repetition).as(UTF8).named(field.getName) + + case _: DateType => + Types.primitive(INT32, repetition).as(DATE).named(field.getName) + + // NOTE: Spark SQL can write timestamp values to Parquet using INT96, TIMESTAMP_MICROS or + // TIMESTAMP_MILLIS. TIMESTAMP_MICROS is recommended but INT96 is the default to keep the + // behavior same as before. + // + // As stated in PARQUET-323, Parquet `INT96` was originally introduced to represent nanosecond + // timestamp in Impala for some historical reasons. It's not recommended to be used for any + // other types and will probably be deprecated in some future version of parquet-format spec. + // That's the reason why parquet-format spec only defines `TIMESTAMP_MILLIS` and + // `TIMESTAMP_MICROS` which are both logical types annotating `INT64`. + // + // Originally, Spark SQL uses the same nanosecond timestamp type as Impala and Hive. Starting + // from Spark 1.5.0, we resort to a timestamp type with microsecond precision so that we can + // store a timestamp into a `Long`. This design decision is subject to change though, for + // example, we may resort to nanosecond precision in the future. + case _: TimestampType => + outputTimestampType match { + case ParquetOutputTimestampType.INT96 => + Types.primitive(INT96, repetition).named(field.getName) + case ParquetOutputTimestampType.TIMESTAMP_MICROS => + Types.primitive(INT64, repetition).as(TIMESTAMP_MICROS).named(field.getName) + case ParquetOutputTimestampType.TIMESTAMP_MILLIS => + Types.primitive(INT64, repetition).as(TIMESTAMP_MILLIS).named(field.getName) + } + + case _: BinaryType => + Types.primitive(BINARY, repetition).named(field.getName) + + // ====================== + // Decimals (legacy mode) + // ====================== + + // Spark 1.4.x and prior versions only support decimals with a maximum precision of 18 and + // always store decimals in fixed-length byte arrays. To keep compatibility with these older + // versions, here we convert decimals with all precisions to `FIXED_LEN_BYTE_ARRAY` annotated + // by `DECIMAL`. + case decimalType: DecimalType if writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(DECIMAL) + .precision(decimalType.getPrecision) + .scale(decimalType.getScale) + .length(computeMinBytesForPrecision(decimalType.getPrecision)) + .named(field.getName) + + // ======================== + // Decimals (standard mode) + // ======================== + + // Uses INT32 for 1 <= precision <= 9 + case decimalType: DecimalType + if decimalType.getPrecision <= 9 && !writeLegacyParquetFormat => + Types + .primitive(INT32, repetition) + .as(DECIMAL) + .precision(decimalType.getPrecision) + .scale(decimalType.getScale) + .named(field.getName) + + // Uses INT64 for 1 <= precision <= 18 + case decimalType: DecimalType + if decimalType.getPrecision <= 18 && !writeLegacyParquetFormat => + Types + .primitive(INT64, repetition) + .as(DECIMAL) + .precision(decimalType.getPrecision) + .scale(decimalType.getScale) + .named(field.getName) + + // Uses FIXED_LEN_BYTE_ARRAY for all other precisions + case decimalType: DecimalType if !writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(DECIMAL) + .precision(decimalType.getPrecision) + .scale(decimalType.getScale) + .length(computeMinBytesForPrecision(decimalType.getPrecision)) + .named(field.getName) + + // =================================== + // ArrayType and MapType (legacy mode) + // =================================== + + // Spark 1.4.x and prior versions convert `ArrayType` with nullable elements into a 3-level + // `LIST` structure. This behavior is somewhat a hybrid of parquet-hive and parquet-avro + // (1.6.0rc3): the 3-level structure is similar to parquet-hive while the 3rd level element + // field name "array" is borrowed from parquet-avro. + case arrayType: ArrayType if arrayType.containsNull && writeLegacyParquetFormat => + // group (LIST) { + // optional group bag { + // repeated array; + // } + // } + + // This should not use `listOfElements` here because this new method checks if the + // element name is `element` in the `GroupType` and throws an exception if not. + // As mentioned above, Spark prior to 1.4.x writes `ArrayType` as `LIST` but with + // `array` as its element name as below. Therefore, we build manually + // the correct group type here via the builder. (See SPARK-16777) + Types + .buildGroup(repetition).as(LIST) + .addField(Types + .buildGroup(REPEATED) + // "array" is the name chosen by parquet-hive (1.7.0 and prior version) + .addField(convertField( + new StructField("array", arrayType.getElementType, arrayType.containsNull))) + .named("bag")) + .named(field.getName) + + // Spark 1.4.x and prior versions convert ArrayType with non-nullable elements into a 2-level + // LIST structure. This behavior mimics parquet-avro (1.6.0rc3). Note that this case is + // covered by the backwards-compatibility rules implemented in `isElementType()`. + case arrayType: ArrayType if !arrayType.containsNull && writeLegacyParquetFormat => + // group (LIST) { + // repeated element; + // } + + // Here too, we should not use `listOfElements`. (See SPARK-16777) + Types + .buildGroup(repetition).as(LIST) + // "array" is the name chosen by parquet-avro (1.7.0 and prior version) + .addField(convertField( + new StructField("array", arrayType.getElementType, arrayType.containsNull), REPEATED)) + .named(field.getName) + + // Spark 1.4.x and prior versions convert MapType into a 3-level group annotated by + // MAP_KEY_VALUE. This is covered by `convertGroupField(field: GroupType): DataType`. + case mapType: MapType if writeLegacyParquetFormat => + // group (MAP) { + // repeated group map (MAP_KEY_VALUE) { + // required key; + // value; + // } + // } + ConversionPatterns.mapType( + repetition, + field.getName, + "key_value", + convertField(new StructField("key", mapType.getKeyType, false)), + convertField(new StructField("value", mapType.getValueType, mapType.valueContainsNull))) + + // ===================================== + // ArrayType and MapType (standard mode) + // ===================================== + + case arrayType: ArrayType if !writeLegacyParquetFormat => + // group (LIST) { + // repeated group list { + // element; + // } + // } + Types + .buildGroup(repetition).as(LIST) + .addField( + Types.repeatedGroup() + .addField(convertField( + new StructField("element", arrayType.getElementType, arrayType.containsNull))) + .named("list")) + .named(field.getName) + + case mapType: MapType if !writeLegacyParquetFormat => + // group (MAP) { + // repeated group key_value { + // required key; + // value; + // } + // } + Types + .buildGroup(repetition).as(MAP) + .addField( + Types + .repeatedGroup() + .addField(convertField(new StructField("key", mapType.getKeyType, false))) + .addField(convertField( + new StructField("value", mapType.getValueType, mapType.valueContainsNull()))) + .named("key_value")) + .named(field.getName) + + // =========== + // Other types + // =========== + + case structType: StructType => + structType.getFields.foldLeft(Types.buildGroup(repetition)) { (builder, field) => + builder.addField(convertField(field)) + }.named(field.getName) + + case _ => + throw new UnsupportedOperationException( + s"Unsupported data type ${field.getDataType.getTypeName}") + } + } + + // Returns the minimum number of bytes needed to store a decimal with a given `precision`. + private def computeMinBytesForPrecision(precision: Int) : Int = { + var numBytes = 1 + while (math.pow(2.0, 8 * numBytes - 1) < math.pow(10.0, precision)) { + numBytes += 1 + } + numBytes + } +} + +private[internal] object ParquetSchemaConverter { + val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" + + val EMPTY_MESSAGE: MessageType = + Types.buildMessage().named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + + def checkFieldName(name: String): Unit = { + // ,;{}()\n\t= and space are special characters in Parquet schema + checkConversionRequirement( + !name.matches(".*[ ,;{}()\n\t=].*"), + s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\\n\\t=". + |Please use alias to rename it. + """.stripMargin.split("\n").mkString(" ").trim) + } + + def checkFieldNames(names: Seq[String]): Unit = { + names.foreach(checkFieldName) + } + + def checkConversionRequirement(f: => Boolean, message: String): Unit = { + if (!f) { + throw new IllegalArgumentException(message) + } + } +} diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ParquetSchemaConverterSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ParquetSchemaConverterSuite.scala new file mode 100644 index 00000000000..211e7537f54 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/ParquetSchemaConverterSuite.scala @@ -0,0 +1,305 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import org.apache.parquet.schema.MessageTypeParser +import org.scalatest.FunSuite + +import io.delta.standalone.types._ +import io.delta.standalone.util.ParquetSchemaConverter + +class ParquetSchemaConverterSuite extends FunSuite { + + private def testCatalystToParquet( + testName: String, + sqlSchema: StructType, + parquetSchema: String, + writeLegacyParquetFormat: Boolean, + outputTimestampType: ParquetSchemaConverter.ParquetOutputTimestampType = + ParquetSchemaConverter.ParquetOutputTimestampType.INT96): Unit = { + + test(s"sql => parquet: $testName") { + val actual = ParquetSchemaConverter.sparkToParquet( + sqlSchema, + writeLegacyParquetFormat, + outputTimestampType) + val expected = MessageTypeParser.parseMessageType(parquetSchema) + actual.checkContains(expected) + expected.checkContains(actual) + } + } + + // ======================================================= + // Tests for converting Catalyst ArrayType to Parquet LIST + // ======================================================= + + testCatalystToParquet( + "Backwards-compatibility: LIST with nullable element type - 1 - standard", + new StructType(Array( + new StructField( + "f1", + new ArrayType(new IntegerType(), true), + true))), + """message root { + | optional group f1 (LIST) { + | repeated group list { + | optional int32 element; + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "Backwards-compatibility: LIST with nullable element type - 2 - prior to 1.4.x", + new StructType(Array( + new StructField( + "f1", + new ArrayType(new IntegerType(), true), + true))), + """message root { + | optional group f1 (LIST) { + | repeated group bag { + | optional int32 array; + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "Backwards-compatibility: LIST with non-nullable element type - 1 - standard", + new StructType(Array( + new StructField( + "f1", + new ArrayType(new IntegerType(), false), + true))), + """message root { + | optional group f1 (LIST) { + | repeated group list { + | required int32 element; + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "Backwards-compatibility: LIST with non-nullable element type - 2 - prior to 1.4.x", + new StructType(Array( + new StructField( + "f1", + new ArrayType(new IntegerType(), false), + true))), + """message root { + | optional group f1 (LIST) { + | repeated int32 array; + | } + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + // ==================================================== + // Tests for converting Catalyst MapType to Parquet Map + // ==================================================== + + testCatalystToParquet( + "Backwards-compatibility: MAP with non-nullable value type - 1 - standard", + new StructType(Array( + new StructField( + "f1", + new MapType(new IntegerType(), new StringType(), false), + true))), + """message root { + | optional group f1 (MAP) { + | repeated group key_value { + | required int32 key; + | required binary value (UTF8); + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "Backwards-compatibility: MAP with non-nullable value type - 2 - prior to 1.4.x", + new StructType(Array( + new StructField( + "f1", + new MapType(new IntegerType(), new StringType(), false), + true))), + """message root { + | optional group f1 (MAP) { + | repeated group key_value (MAP_KEY_VALUE) { + | required int32 key; + | required binary value (UTF8); + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "Backwards-compatibility: MAP with nullable value type - 1 - standard", + new StructType(Array( + new StructField( + "f1", + new MapType(new IntegerType(), new StringType(), true), + true))), + """message root { + | optional group f1 (MAP) { + | repeated group key_value { + | required int32 key; + | optional binary value (UTF8); + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "Backwards-compatibility: MAP with nullable value type - 3 - prior to 1.4.x", + new StructType(Array( + new StructField( + "f1", + new MapType(new IntegerType(), new StringType(), true), + true))), + """message root { + | optional group f1 (MAP) { + | repeated group key_value (MAP_KEY_VALUE) { + | required int32 key; + | optional binary value (UTF8); + | } + | } + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + // ================================= + // Tests for conversion for decimals + // ================================= + + testCatalystToParquet( + "DECIMAL(1, 0) - standard", + new StructType(Array(new StructField("f1", new DecimalType(1, 0)))), + """message root { + | optional int32 f1 (DECIMAL(1, 0)); + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "DECIMAL(8, 3) - standard", + new StructType(Array(new StructField("f1", new DecimalType(8, 3)))), + """message root { + | optional int32 f1 (DECIMAL(8, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "DECIMAL(9, 3) - standard", + new StructType(Array(new StructField("f1", new DecimalType(9, 3)))), + """message root { + | optional int32 f1 (DECIMAL(9, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "DECIMAL(18, 3) - standard", + new StructType(Array(new StructField("f1", new DecimalType(18, 3)))), + """message root { + | optional int64 f1 (DECIMAL(18, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "DECIMAL(19, 3) - standard", + new StructType(Array(new StructField("f1", new DecimalType(19, 3)))), + """message root { + | optional fixed_len_byte_array(9) f1 (DECIMAL(19, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = false) + + testCatalystToParquet( + "DECIMAL(1, 0) - prior to 1.4.x", + new StructType(Array(new StructField("f1", new DecimalType(1, 0)))), + """message root { + | optional fixed_len_byte_array(1) f1 (DECIMAL(1, 0)); + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "DECIMAL(8, 3) - prior to 1.4.x", + new StructType(Array(new StructField("f1", new DecimalType(8, 3)))), + """message root { + | optional fixed_len_byte_array(4) f1 (DECIMAL(8, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "DECIMAL(9, 3) - prior to 1.4.x", + new StructType(Array(new StructField("f1", new DecimalType(9, 3)))), + """message root { + | optional fixed_len_byte_array(5) f1 (DECIMAL(9, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "DECIMAL(18, 3) - prior to 1.4.x", + new StructType(Array(new StructField("f1", new DecimalType(18, 3)))), + """message root { + | optional fixed_len_byte_array(8) f1 (DECIMAL(18, 3)); + |} + """.stripMargin, + writeLegacyParquetFormat = true) + + testCatalystToParquet( + "Timestamp written and read as INT64 with TIMESTAMP_MILLIS", + new StructType(Array(new StructField("f1", new TimestampType()))), + """message root { + | optional INT64 f1 (TIMESTAMP_MILLIS); + |} + """.stripMargin, + writeLegacyParquetFormat = true, + outputTimestampType = ParquetSchemaConverter.ParquetOutputTimestampType.TIMESTAMP_MILLIS) + + testCatalystToParquet( + "Timestamp written and read as INT64 with TIMESTAMP_MICROS", + new StructType(Array(new StructField("f1", new TimestampType()))), + """message root { + | optional INT64 f1 (TIMESTAMP_MICROS); + |} + """.stripMargin, + writeLegacyParquetFormat = true, + outputTimestampType = ParquetSchemaConverter.ParquetOutputTimestampType.TIMESTAMP_MICROS) + + testCatalystToParquet( + "SPARK-36825: Year-month interval written and read as INT32", + new StructType(Array(new StructField("f1", new DateType()))), + """message root { + | optional INT32 f1; + |} + """.stripMargin, + writeLegacyParquetFormat = false) +} From 9c8f76eafefdb3ffbb2d2e5b7f08cfb0b71df5c9 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Mon, 25 Oct 2021 10:25:20 -0700 Subject: [PATCH 111/291] Add more Expression functionality (#162) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR extends the Expression functionality for the Delta standalone reader and writer. This includes: • Column and Literal support for partition data types • CastingComparator and Util support for valid comparison data types • PartitionRowRecord support for partition data types • In.eval() updated to follow Databrick's SQL semantics guide • ExpressionSuite testing • misc. changes & fixes --- .../io/delta/standalone/expressions/And.java | 26 + .../expressions/BinaryComparison.java | 24 + .../expressions/BinaryExpression.java | 38 ++ .../expressions/BinaryOperator.java | 28 + .../delta/standalone/expressions/Column.java | 76 +++ .../delta/standalone/expressions/EqualTo.java | 16 + .../standalone/expressions/GreaterThan.java | 15 + .../expressions/GreaterThanOrEqual.java | 16 + .../io/delta/standalone/expressions/In.java | 81 +++ .../standalone/expressions/IsNotNull.java | 24 + .../delta/standalone/expressions/IsNull.java | 22 + .../expressions/LeafExpression.java | 11 + .../standalone/expressions/LessThan.java | 15 + .../expressions/LessThanOrEqual.java | 16 + .../delta/standalone/expressions/Literal.java | 114 ++++ .../io/delta/standalone/expressions/Not.java | 29 + .../io/delta/standalone/expressions/Or.java | 26 + .../standalone/expressions/Predicate.java | 11 + .../expressions/UnaryExpression.java | 36 ++ .../expressions/CastingComparator.java | 16 + .../standalone/internal/expressions/Util.java | 74 +++ .../io/delta/standalone/types/StructType.java | 7 + .../standalone/internal/DeltaLogImpl.scala | 24 +- .../internal/data/PartitionRowRecord.scala | 175 ++++++ .../internal/exception/DeltaErrors.scala | 24 +- .../standalone/internal/ExpressionSuite.scala | 505 ++++++++++++++++++ 26 files changed, 1445 insertions(+), 4 deletions(-) create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/And.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Column.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/GreaterThanOrEqual.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/In.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/IsNull.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/LessThan.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/LessThanOrEqual.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Literal.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Not.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Or.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/Predicate.java create mode 100644 standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java create mode 100644 standalone/src/main/java/io/delta/standalone/internal/expressions/CastingComparator.java create mode 100644 standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java create mode 100644 standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala create mode 100644 standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala diff --git a/standalone/src/main/java/io/delta/standalone/expressions/And.java b/standalone/src/main/java/io/delta/standalone/expressions/And.java new file mode 100644 index 00000000000..dd2248648f7 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/And.java @@ -0,0 +1,26 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.internal.exception.DeltaErrors; + +/** + * Usage: {@code new And(expr1, expr2)} - Logical AND + */ +public final class And extends BinaryOperator implements Predicate { + + public And(Expression left, Expression right) { + super(left, right, "&&"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + if (!(leftResult instanceof Boolean) || !(rightResult instanceof Boolean)) { + throw DeltaErrors.illegalExpressionValueType( + "AND", + "Boolean", + leftResult.getClass().getName(), + rightResult.getClass().getName()); + } + + return (boolean) leftResult && (boolean) rightResult; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java new file mode 100644 index 00000000000..2f029233b27 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java @@ -0,0 +1,24 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.internal.expressions.Util; + +import java.util.Comparator; + +/** + * A {@link BinaryOperator} that compares the left and right {@link Expression}s and returns a boolean value. + */ +public abstract class BinaryComparison extends BinaryOperator implements Predicate { + private final Comparator comparator; + + public BinaryComparison(Expression left, Expression right, String symbol) { + super(left, right, symbol); + + // super asserted that left and right DataTypes were the same + + comparator = Util.createComparator(left.dataType()); + } + + protected int compare(Object leftResult, Object rightResult) { + return comparator.compare(leftResult, rightResult); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java new file mode 100644 index 00000000000..2d02d6328f2 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java @@ -0,0 +1,38 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; + +import java.util.Arrays; +import java.util.List; + +/** + * An expression with two inputs and one output. The output is by default evaluated to null + * if any input is evaluated to null. + */ +public abstract class BinaryExpression implements Expression { + protected final Expression left; + protected final Expression right; + + public BinaryExpression(Expression left, Expression right) { + this.left = left; + this.right = right; + } + + @Override + public final Object eval(RowRecord record) { + Object leftResult = left.eval(record); + if (null == leftResult) return null; + + Object rightResult = right.eval(record); + if (null == rightResult) return null; + + return nullSafeEval(leftResult, rightResult); + } + + protected abstract Object nullSafeEval(Object leftResult, Object rightResult); + + @Override + public List children() { + return Arrays.asList(left, right); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java new file mode 100644 index 00000000000..2e9cba7e9d2 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryOperator.java @@ -0,0 +1,28 @@ +package io.delta.standalone.expressions; + +/** + * A {@link BinaryExpression} that is an operator, with two properties: + * + * 1. The string representation is "x symbol y", rather than "funcName(x, y)". + * 2. Two inputs are expected to be of the same type. If the two inputs have different types, a + * {@link IllegalArgumentException} will be thrown. + */ +public abstract class BinaryOperator extends BinaryExpression { + protected final String symbol; + + public BinaryOperator(Expression left, Expression right, String symbol) { + super(left, right); + this.symbol = symbol; + + if (!left.dataType().equals(right.dataType())) { + throw new IllegalArgumentException("BinaryOperator left and right DataTypes must be the" + + " same, found: " + left.dataType().getTypeName() + " " + symbol + " " + + right.dataType().getTypeName()); + } + } + + @Override + public String toString() { + return "(" + left.toString() + " " + symbol + " " + right.toString() + ")"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Column.java b/standalone/src/main/java/io/delta/standalone/expressions/Column.java new file mode 100644 index 00000000000..eb7c9b3b730 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Column.java @@ -0,0 +1,76 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.expressions.LeafExpression; +import io.delta.standalone.types.*; + +/** + * A column whose row-value will be computed based on the data in a {@link RowRecord}. + * + * Usage: {@code schema.column(columnName)} + * + * It is recommended that you instantiate using a table schema ({@link StructType}). + */ +public final class Column extends LeafExpression { + private final String name; + private final DataType dataType; + private final RowRecordEvaluator evaluator; + + public Column(String name, DataType dataType) { + this.name = name; + this.dataType = dataType; + + if (dataType instanceof IntegerType) { + evaluator = (record -> record.getInt(name)); + } else if (dataType instanceof LongType) { + evaluator = (record -> record.getLong(name)); + } else if (dataType instanceof ByteType) { + evaluator = (record -> record.getByte(name)); + } else if (dataType instanceof ShortType) { + evaluator = (record -> record.getShort(name)); + } else if (dataType instanceof BooleanType) { + evaluator = (record -> record.getBoolean(name)); + } else if (dataType instanceof FloatType) { + evaluator = (record -> record.getFloat(name)); + } else if (dataType instanceof DoubleType) { + evaluator = (record -> record.getDouble(name)); + } else if (dataType instanceof StringType) { + evaluator = (record -> record.getString(name)); + } else if (dataType instanceof BinaryType) { + evaluator = (record -> record.getBinary(name)); + } else if (dataType instanceof DecimalType) { + evaluator = (record -> record.getBigDecimal(name)); + } else if (dataType instanceof TimestampType) { + evaluator = (record -> record.getTimestamp(name)); + } else if (dataType instanceof DateType) { + evaluator = (record -> record.getDate(name)); + } else { + throw new UnsupportedOperationException("The data type of column " + name + + " is " + dataType.getTypeName() + ". This is not supported yet."); + } + } + + public String name() { + return name; + } + + @Override + public Object eval(RowRecord record) { + return record.isNullAt(name) ? null : evaluator.nullSafeEval(record); + } + + @Override + public DataType dataType() { + return dataType; + } + + @Override + public String toString() { + return "Column(" + name + ")"; + } + + @FunctionalInterface + private interface RowRecordEvaluator { + Object nullSafeEval(RowRecord record); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java b/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java new file mode 100644 index 00000000000..9eae2e9da3e --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/EqualTo.java @@ -0,0 +1,16 @@ +package io.delta.standalone.expressions; + +/** + * Usage: {@code new EqualTo(expr1, expr2)} - Returns true if `expr1` equals `expr2`, else false. + */ +public final class EqualTo extends BinaryComparison implements Predicate { + + public EqualTo(Expression left, Expression right) { + super(left, right, "="); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) == 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java new file mode 100644 index 00000000000..8fbcdcb3358 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java @@ -0,0 +1,15 @@ +package io.delta.standalone.expressions; + +/** + * Usage: {@code new GreaterThan(expr1, expr2)} - Returns true if `expr1` is greater than `expr2`, else false. + */ +public final class GreaterThan extends BinaryComparison implements Predicate { + public GreaterThan(Expression left, Expression right) { + super(left, right, ">"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) > 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/GreaterThanOrEqual.java b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThanOrEqual.java new file mode 100644 index 00000000000..1df58e17147 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThanOrEqual.java @@ -0,0 +1,16 @@ +package io.delta.standalone.expressions; + +/** + * Usage: {@code new GreaterThanOrEqual(expr1, expr2)} - Returns true if `expr1` is greater than or + * equal to `expr2`, else false. + */ +public final class GreaterThanOrEqual extends BinaryComparison implements Predicate { + public GreaterThanOrEqual(Expression left, Expression right) { + super(left, right, ">="); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) >= 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/In.java b/standalone/src/main/java/io/delta/standalone/expressions/In.java new file mode 100644 index 00000000000..6047e3d9c55 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/In.java @@ -0,0 +1,81 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.internal.expressions.Util; + +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Usage: {@code new In(expr, exprList)} - Returns true if `expr` is equal to any in `exprList`, else false. + */ +public final class In implements Predicate { + private final Expression value; + private final List elems; + private final Comparator comparator; + + public In(Expression value, List elems) { + if (null == value) { + throw new IllegalArgumentException("'In' expression 'value' cannot be null"); + } + if (null == elems) { + throw new IllegalArgumentException("'In' expression 'elems' cannot be null"); + } + if (elems.isEmpty()) { + throw new IllegalArgumentException("'In' expression 'elems' cannot be empty"); + } + + boolean allSameDataType = elems.stream().allMatch(x -> x.dataType().equals(value.dataType())); + + if (!allSameDataType) { + throw new IllegalArgumentException("In expression 'elems' and 'value' must all be of the same DataType"); + } + + this.value = value; + this.elems = elems; + this.comparator = Util.createComparator(value.dataType()); + } + + /** + * This implements the {@code IN} expression functionality outlined by the Databricks SQL Null + * semantics reference guide. The logic is as follows: + * - TRUE if the non-NULL value is found in the list + * - FALSE if the non-NULL value is not found in the list and the list does not contain NULL + * values + * - NULL if the value is NULL, or the non-NULL value is not found in the list and the list + * contains at least one NULL value + * + * @see NULL Semantics + */ + @Override + public Boolean eval(RowRecord record) { + Object origValue = value.eval(record); + if (null == origValue) return null; + + // null if a null value has been found in list, otherwise false + Boolean falseOrNullresult = false; + for (Expression setElem : elems) { + Object setElemValue = setElem.eval(record); + if (setElemValue == null) + // null value found but element may still be in list + falseOrNullresult = null; + else if (comparator.compare(origValue, setElemValue) == 0) + // short circuit and return true; we have found the element in the list + return true; + } + return falseOrNullresult; + } + + @Override + public String toString() { + String elemsStr = elems.stream().map(Expression::toString).collect(Collectors.joining(", ")); + return value + " IN (" + elemsStr + ")"; + } + + @Override + public List children() { + return Stream.concat(Stream.of(value), elems.stream()).collect(Collectors.toList()); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java b/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java new file mode 100644 index 00000000000..738fae932b4 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/IsNotNull.java @@ -0,0 +1,24 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +/** + * Usage: {@code new IsNotNull(expr)} - Returns true if `expr` is not null, else false. + */ +public class IsNotNull extends UnaryExpression implements Predicate { + public IsNotNull(Expression child) { + super(child); + } + + @Override + public Object eval(RowRecord record) { + return child.eval(record) != null; + } + + @Override + public String toString() { + return "(" + child.toString() + ") IS NOT NULL"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/IsNull.java b/standalone/src/main/java/io/delta/standalone/expressions/IsNull.java new file mode 100644 index 00000000000..cde2bd4ddc7 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/IsNull.java @@ -0,0 +1,22 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +/** + * Usage: {@code new IsNull(expr)} - Returns true if `expr` is null, else false. + */ +public class IsNull extends UnaryExpression implements Predicate { + public IsNull(Expression child) { + super(child); + } + + @Override + public Object eval(RowRecord record) { return child.eval(record) == null; } + + @Override + public String toString() { + return "(" + child.toString() + ") IS NULL"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java new file mode 100644 index 00000000000..0df332e0e04 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/LeafExpression.java @@ -0,0 +1,11 @@ +package io.delta.standalone.expressions; + +import java.util.Collections; +import java.util.List; + +public abstract class LeafExpression implements Expression { + @Override + public List children() { + return Collections.emptyList(); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java new file mode 100644 index 00000000000..fba78183d0c --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java @@ -0,0 +1,15 @@ +package io.delta.standalone.expressions; + +/** + * Usage: {@code new LessThan(expr1, expr2)} - Returns true if `expr1` is less than `expr2`, else false. + */ +public final class LessThan extends BinaryComparison implements Predicate { + public LessThan(Expression left, Expression right) { + super(left, right, "<"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) < 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LessThanOrEqual.java b/standalone/src/main/java/io/delta/standalone/expressions/LessThanOrEqual.java new file mode 100644 index 00000000000..e64a8cb8348 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/LessThanOrEqual.java @@ -0,0 +1,16 @@ +package io.delta.standalone.expressions; + +/** + * Usage: {@code new LessThanOrEqual(expr1, expr2)} - Returns true if `expr1` is less than or + * equal to `expr2`, else false. + */ +public final class LessThanOrEqual extends BinaryComparison implements Predicate { + public LessThanOrEqual(Expression left, Expression right) { + super(left, right, "<="); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + return compare(leftResult, rightResult) <= 0; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java new file mode 100644 index 00000000000..6c585335785 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java @@ -0,0 +1,114 @@ +package io.delta.standalone.expressions; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Objects; + +import io.delta.standalone.data.RowRecord; +import io.delta.standalone.types.*; + +public final class Literal extends LeafExpression { + public static final Literal True = Literal.of(true); + public static final Literal False = Literal.of(false); + + private final Object value; + private final DataType dataType; + + private Literal(Object value, DataType dataType) { + this.value = value; + this.dataType = dataType; + } + + public Object value() { + return value; + } + + @Override + public Object eval(RowRecord record) { + return value; + } + + @Override + public DataType dataType() { + return dataType; + } + + @Override + public String toString() { + return String.valueOf(value); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Literal literal = (Literal) o; + return Objects.equals(value, literal.value) && + Objects.equals(dataType, literal.dataType); + } + + @Override + public int hashCode() { + return Objects.hash(value, dataType); + } + + public static Literal of(int value) { + return new Literal(value, new IntegerType()); + } + + public static Literal of(boolean value) { + return new Literal(value, new BooleanType()); + } + + public static Literal of(byte[] value) { + return new Literal(value, new BinaryType()); + } + + public static Literal of(Date value) { + return new Literal(value, new DateType()); + } + + public static Literal of(BigDecimal value) { + //TODO: get the precision and scale from the value + return new Literal(value, DecimalType.USER_DEFAULT); + } + + public static Literal of(double value) { + return new Literal(value, new DoubleType()); + } + + public static Literal of(float value) { + return new Literal(value, new FloatType()); + } + + public static Literal of(long value) { + return new Literal(value, new LongType()); + } + + public static Literal of(short value) { + return new Literal(value, new ShortType()); + } + + public static Literal of(String value) { + return new Literal(value, new StringType()); + } + + public static Literal of(Timestamp value) { + return new Literal(value, new TimestampType()); + } + + public static Literal of(byte value) { + return new Literal(value, new ByteType()); + } + + public static Literal ofNull(DataType dataType) { + if (dataType instanceof NullType + || dataType instanceof ArrayType + || dataType instanceof MapType + || dataType instanceof StructType) { + throw new IllegalArgumentException( + dataType.getTypeName() + " is an invalid data type for Literal."); + } + return new Literal(null, dataType); } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Not.java b/standalone/src/main/java/io/delta/standalone/expressions/Not.java new file mode 100644 index 00000000000..f2914e3dc50 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Not.java @@ -0,0 +1,29 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.internal.exception.DeltaErrors; + +/** + * Usage: {@code new Not(expr)} - Logical not. + */ +public class Not extends UnaryExpression implements Predicate { + public Not(Expression child) { + super(child); + } + + @Override + public Object nullSafeEval(Object childResult) { + if (!(childResult instanceof Boolean)) { + throw DeltaErrors.illegalExpressionValueType( + "NOT", + "Boolean", + childResult.getClass().getName()); + } + + return !((boolean) childResult); + } + + @Override + public String toString() { + return "(NOT " + child.toString() + ")"; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Or.java b/standalone/src/main/java/io/delta/standalone/expressions/Or.java new file mode 100644 index 00000000000..e88f23718a9 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Or.java @@ -0,0 +1,26 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.internal.exception.DeltaErrors; + +/** + * Usage: {@code new Or(expr1, expr2)} - Logical OR + */ +public final class Or extends BinaryOperator implements Predicate { + + public Or(Expression left, Expression right) { + super(left, right, "||"); + } + + @Override + public Object nullSafeEval(Object leftResult, Object rightResult) { + if (!(leftResult instanceof Boolean) || !(rightResult instanceof Boolean)) { + throw DeltaErrors.illegalExpressionValueType( + "OR", + "Boolean", + leftResult.getClass().getName(), + rightResult.getClass().getName()); + } + + return (boolean) leftResult || (boolean) rightResult; + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java b/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java new file mode 100644 index 00000000000..a839d7f7631 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/Predicate.java @@ -0,0 +1,11 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.types.BooleanType; +import io.delta.standalone.types.DataType; + +public interface Predicate extends Expression { + @Override + default DataType dataType() { + return new BooleanType(); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java new file mode 100644 index 00000000000..f6383ebfb4b --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java @@ -0,0 +1,36 @@ +package io.delta.standalone.expressions; + +import io.delta.standalone.data.RowRecord; + +import java.util.Collections; +import java.util.List; + +/** + * An expression with one input and one output. The output is by default evaluated to null + * if the input is evaluated to null. + */ +public abstract class UnaryExpression implements Expression { + protected final Expression child; + + public UnaryExpression(Expression child) { + this.child = child; + } + + @Override + public Object eval(RowRecord record) { + Object childResult = child.eval(record); + + if (null == childResult) return null; + + return nullSafeEval(childResult); + } + + protected Object nullSafeEval(Object childResult) { + throw new IllegalArgumentException("UnaryExpressions must override either eval or nullSafeEval"); + } + + @Override + public List children() { + return Collections.singletonList(child); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/internal/expressions/CastingComparator.java b/standalone/src/main/java/io/delta/standalone/internal/expressions/CastingComparator.java new file mode 100644 index 00000000000..924ea20ace1 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/internal/expressions/CastingComparator.java @@ -0,0 +1,16 @@ +package io.delta.standalone.internal.expressions; + +import java.util.Comparator; + +public class CastingComparator> implements Comparator { + private final Comparator comparator; + + public CastingComparator() { + comparator = Comparator.naturalOrder(); + } + + @Override + public int compare(Object a, Object b) { + return comparator.compare((T) a, (T) b); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java b/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java new file mode 100644 index 00000000000..5c35090e123 --- /dev/null +++ b/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java @@ -0,0 +1,74 @@ +package io.delta.standalone.internal.expressions; + +import java.math.BigDecimal; +import java.util.Comparator; +import java.util.Date; + +import io.delta.standalone.types.*; + +public final class Util { + + public static Comparator createComparator(DataType dataType) { + if (dataType instanceof IntegerType) { + return new CastingComparator(); + } + + if (dataType instanceof BooleanType) { + return new CastingComparator(); + } + + if (dataType instanceof FloatType) { + return new CastingComparator(); + } + + if (dataType instanceof LongType) { + return new CastingComparator(); + } + + if (dataType instanceof ByteType) { + return new CastingComparator(); + } + + if (dataType instanceof ShortType) { + return new CastingComparator(); + } + + if (dataType instanceof DoubleType) { + return new CastingComparator(); + } + + if (dataType instanceof DecimalType) { + return new CastingComparator(); + } + + if (dataType instanceof TimestampType) { + return new CastingComparator(); + } + + if (dataType instanceof DateType) { + return new CastingComparator(); + } + + if (dataType instanceof StringType) { + return new CastingComparator(); + } + + if (dataType instanceof BinaryType) { + return (o1, o2) -> { + byte[] one = (byte[]) o1; + byte[] two = (byte[]) o2; + int i = 0; + while (i < one.length && i < two.length) { + if (one[i] != two[i]) { + return Byte.compare(one[i], two[i]); + } + i ++; + } + return Integer.compare(one.length, two.length); + }; + } + + // unsupported comparison types: ArrayType, StructType, MapType + throw new IllegalArgumentException("Couldn't find matching comparator for DataType: " + dataType.getTypeName()); + } +} diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index 544c2b77740..ed7ee17b294 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -38,6 +38,8 @@ package io.delta.standalone.types; +import io.delta.standalone.expressions.Column; + import java.util.Arrays; import java.util.HashMap; @@ -89,6 +91,11 @@ public StructField get(String fieldName) { return nameToField.get(fieldName); } + public Column column(String fieldName) { + StructField field = nameToField.get(fieldName); + return new Column(fieldName, field.getDataType()); + } + /** * @return a readable indented tree representation of this {@code StructType} * and all of its nested elements diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala index 1a29aef6e33..3ce1acded21 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala @@ -24,10 +24,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import io.delta.standalone.{DeltaLog, VersionLog} import io.delta.standalone.actions.{CommitInfo => CommitInfoJ} -import io.delta.standalone.internal.actions.Action +import io.delta.standalone.expressions.{And, Expression, Literal} +import io.delta.standalone.internal.actions.{Action, AddFile} +import io.delta.standalone.internal.data.PartitionRowRecord import io.delta.standalone.internal.exception.DeltaErrors import io.delta.standalone.internal.storage.HDFSReadOnlyLogStore import io.delta.standalone.internal.util.{ConversionUtils, FileNames} +import io.delta.standalone.types.StructType /** * Scala implementation of Java interface [[DeltaLog]]. @@ -108,4 +111,23 @@ private[standalone] object DeltaLogImpl { new DeltaLogImpl(hadoopConf, path, path.getParent) } + + /** + * Filters the given [[AddFile]]s by the given `partitionFilters`, returning those that match. + * @param files The active files in the DeltaLog state, which contains the partition value + * information + * @param partitionFilters Filters on the partition columns + */ + def filterFileList( + partitionSchema: StructType, + files: Seq[AddFile], + partitionFilters: Seq[Expression]): Seq[AddFile] = { + val expr = partitionFilters.reduceLeftOption(new And(_, _)).getOrElse(Literal.True) + + files.filter { addFile => + val partitionRowRecord = new PartitionRowRecord(partitionSchema, addFile.partitionValues) + val result = expr.eval(partitionRowRecord) + result == true + } + } } diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala new file mode 100644 index 00000000000..bd9853f5eec --- /dev/null +++ b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala @@ -0,0 +1,175 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal.data + +import java.math.{BigDecimal => BigDecimalJ} +import java.sql.{Date, Timestamp} + +import io.delta.standalone.data.{RowRecord => RowRecordJ} +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.types._ + +/** + * A RowRecord representing a Delta Lake partition of Map(partitionKey -> partitionValue) + */ +private[internal] class PartitionRowRecord( + partitionSchema: StructType, + partitionValues: Map[String, String]) extends RowRecordJ { + + require(partitionSchema.getFieldNames.toSet == partitionValues.keySet, + s""" + |Column mismatch between partitionSchema and partitionValues. + |partitionSchema: ${partitionSchema.getFieldNames.mkString(", ")} + |partitionValues: ${partitionValues.keySet.mkString(", ")} + |""".stripMargin) + + private def getPrimitive(field: StructField): String = { + val partitionValue = partitionValues(field.getName) + if (partitionValue == null) throw DeltaErrors.nullValueFoundForPrimitiveTypes(field.getName) + partitionValue + } + + private def getNonPrimitive(field: StructField): Option[String] = { + val partitionValue = partitionValues(field.getName) + if (partitionValue == null) { + if (!field.isNullable) { + throw DeltaErrors.nullValueFoundForNonNullSchemaField(field.getName, partitionSchema) + } + None + } else Some(partitionValue) + } + + override def getSchema: StructType = partitionSchema + + override def getLength: Int = partitionSchema.getFieldNames.length + + override def isNullAt(fieldName: String): Boolean = { + partitionSchema.get(fieldName) // check that the field exists + partitionValues(fieldName) == null + } + + override def getInt(fieldName: String): Int = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[IntegerType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "integer") + } + getPrimitive(field).toInt + } + + override def getLong(fieldName: String): Long = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[LongType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "long") + } + getPrimitive(field).toLong + } + + override def getByte(fieldName: String): Byte = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[ByteType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "byte") + } + getPrimitive(field).toByte + } + + override def getShort(fieldName: String): Short = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[ShortType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "short") + } + getPrimitive(field).toShort + } + + override def getBoolean(fieldName: String): Boolean = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[BooleanType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "boolean") + } + getPrimitive(field).toBoolean + } + + override def getFloat(fieldName: String): Float = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[FloatType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "float") + } + getPrimitive(field).toFloat + } + + override def getDouble(fieldName: String): Double = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[DoubleType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "double") + } + getPrimitive(field).toDouble + } + + override def getString(fieldName: String): String = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[StringType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "string") + } + getNonPrimitive(field).orNull + } + + override def getBinary(fieldName: String): Array[Byte] = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[BinaryType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "binary") + } + getNonPrimitive(field).map(_.map(_.toByte).toArray).orNull + } + + override def getBigDecimal(fieldName: String): BigDecimalJ = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[DecimalType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "decimal") + } + getNonPrimitive(field).map(new BigDecimalJ(_)).orNull + } + + override def getTimestamp(fieldName: String): Timestamp = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[TimestampType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "timestamp") + } + getNonPrimitive(field).map(Timestamp.valueOf).orNull + } + + override def getDate(fieldName: String): Date = { + val field = partitionSchema.get(fieldName) + if (!field.getDataType.isInstanceOf[DateType]) { + throw DeltaErrors.fieldTypeMismatch(fieldName, field.getDataType, "date") + } + getNonPrimitive(field).map(Date.valueOf).orNull + } + + override def getRecord(fieldName: String): RowRecordJ = { + throw new UnsupportedOperationException( + "Struct is not a supported partition type.") + } + + override def getList[T](fieldName: String): java.util.List[T] = { + throw new UnsupportedOperationException( + "Array is not a supported partition type.") + } + + override def getMap[K, V](fieldName: String): java.util.Map[K, V] = { + throw new UnsupportedOperationException( + "Map is not a supported partition type.") + } +} diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala index cdc5090c408..a319f8872a7 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala @@ -18,9 +18,10 @@ package io.delta.standalone.internal.exception import java.io.FileNotFoundException -import org.apache.hadoop.fs.Path +import scala.annotation.varargs -import io.delta.standalone.types.StructType +import org.apache.hadoop.fs.Path +import io.delta.standalone.types.{DataType, StructType} /** A holder object for Delta errors. */ private[internal] object DeltaErrors { @@ -92,7 +93,7 @@ private[internal] object DeltaErrors { } def nullValueFoundForPrimitiveTypes(fieldName: String): Throwable = { - new NullPointerException(s"Read a null value for field $fieldName which is a primitive type") + new NullPointerException(s"Read a null value for field $fieldName which is a primitive type.") } def nullValueFoundForNonNullSchemaField(fieldName: String, schema: StructType): Throwable = { @@ -100,6 +101,15 @@ private[internal] object DeltaErrors { s"that this field can't be null. Schema: ${schema.getTreeString}") } + def fieldTypeMismatch( + fieldName: String, + expectedType: DataType, + actualType: String): Throwable = { + new ClassCastException( + s"The data type of field $fieldName is ${expectedType.getTypeName}. " + + s"Cannot cast it to $actualType") + } + def failOnDataLossException(expectedVersion: Long, seenVersion: Long): Throwable = { new IllegalStateException( s"""The stream from your Delta table was expecting process data from version $expectedVersion, @@ -112,4 +122,12 @@ private[internal] object DeltaErrors { """.stripMargin ) } + + @varargs def illegalExpressionValueType( + exprName: String, + expectedType: String, + realTypes: String*): RuntimeException = { + new IllegalArgumentException( + s"$exprName expression requires $expectedType type. But found ${realTypes.mkString(", ")}"); + } } diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala new file mode 100644 index 00000000000..0fdd23aa927 --- /dev/null +++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala @@ -0,0 +1,505 @@ +/* + * Copyright (2020) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.standalone.internal + +import java.math.{BigDecimal => BigDecimalJ} +import java.util.{Objects, Arrays => ArraysJ} +import java.sql.{Date => DateJ, Timestamp => TimestampJ} + +import scala.collection.JavaConverters._ + +import io.delta.standalone.data.RowRecord +import io.delta.standalone.expressions.{Column, _} +import io.delta.standalone.internal.actions.AddFile +import io.delta.standalone.internal.data.PartitionRowRecord +import io.delta.standalone.internal.exception.DeltaErrors +import io.delta.standalone.types._ + +// scalastyle:off funsuite +import org.scalatest.FunSuite + +class ExpressionSuite extends FunSuite { + // scalastyle:on funsuite + + private def testPredicate( + predicate: Expression, + expectedResult: Any, + record: RowRecord = null) = { + assert(predicate.eval(record) == expectedResult) + } + + private def testException[T <: Throwable](f: => Any, messageContains: String) + (implicit manifest: Manifest[T]) = { + val e = intercept[T]{ + f; + }.getMessage + assert(e.contains(messageContains)) + } + + test("logical predicates") { + // AND tests + testPredicate( + new And(Literal.ofNull(new BooleanType()), Literal.False), null) + testPredicate( + new And(Literal.False, Literal.ofNull(new BooleanType())), null) + testPredicate( + new And(Literal.True, Literal.ofNull(new BooleanType())), null) + testPredicate( + new And(Literal.ofNull(new BooleanType()), Literal.ofNull(new BooleanType())), null) + testPredicate(new And(Literal.False, Literal.False), false) + testPredicate(new And(Literal.True, Literal.False), false) + testPredicate(new And(Literal.False, Literal.True), false) + testPredicate(new And(Literal.True, Literal.True), true) + testException[IllegalArgumentException]( + new And(Literal.of(1), Literal.of(2)).eval(null), + "AND expression requires Boolean type.") + testException[IllegalArgumentException]( + new And(Literal.False, Literal.ofNull(new IntegerType())), + "BinaryOperator left and right DataTypes must be the same") + + // OR tests + testPredicate( + new Or(Literal.ofNull(new BooleanType()), Literal.False), null) + testPredicate( + new Or(Literal.False, Literal.ofNull(new BooleanType())), null) + testPredicate( + new Or(Literal.ofNull(new BooleanType()), Literal.ofNull(new BooleanType())), null) + testPredicate( + new Or(Literal.ofNull(new BooleanType()), Literal.ofNull(new BooleanType())), null) + testPredicate(new Or(Literal.False, Literal.False), false) + testPredicate(new Or(Literal.True, Literal.False), true) + testPredicate(new Or(Literal.False, Literal.True), true) + testPredicate(new Or(Literal.True, Literal.True), true) + // TODO: fail upon creation instead of eval + testException[IllegalArgumentException]( + new Or(Literal.of(1), Literal.of(2)).eval(null), + "OR expression requires Boolean type.") + testException[IllegalArgumentException]( + new Or(Literal.False, Literal.ofNull(new IntegerType())), + "BinaryOperator left and right DataTypes must be the same") + + // NOT tests + testPredicate(new Not(Literal.False), true) + testPredicate(new Not(Literal.True), false) + testPredicate(new Not(Literal.ofNull(new BooleanType())), null) + testException[IllegalArgumentException]( + new Not(Literal.of(1)).eval(null), + "NOT expression requires Boolean type.") + } + + test("comparison predicates") { + // (small, big, small, null) + val literals = Seq( + (Literal.of(1), Literal.of(2), Literal.of(1), Literal.ofNull(new IntegerType())), + (Literal.of(1.0F), Literal.of(2.0F), Literal.of(1.0F), Literal.ofNull(new FloatType())), + (Literal.of(1L), Literal.of(2L), Literal.of(1L), Literal.ofNull(new LongType())), + (Literal.of(1.toShort), Literal.of(2.toShort), Literal.of(1.toShort), + Literal.ofNull(new ShortType())), + (Literal.of(1.0), Literal.of(2.0), Literal.of(1.0), Literal.ofNull(new DoubleType())), + (Literal.of(1.toByte), Literal.of(2.toByte), Literal.of(1.toByte), + Literal.ofNull(new ByteType())), + (Literal.of(new BigDecimalJ("0.1")), Literal.of(new BigDecimalJ("0.2")), + Literal.of(new BigDecimalJ("0.1")), Literal.ofNull(DecimalType.USER_DEFAULT)), + (Literal.False, Literal.True, Literal.False, Literal.ofNull(new BooleanType())), + (Literal.of(new TimestampJ(0)), Literal.of(new TimestampJ(1000000)), + Literal.of(new TimestampJ(0)), Literal.ofNull(new TimestampType())), + (Literal.of(new DateJ(0)), Literal.of(new DateJ(1000000)), + Literal.of(new DateJ(0)), Literal.ofNull(new DateType())), + (Literal.of("apples"), Literal.of("oranges"), Literal.of("apples"), + Literal.ofNull(new StringType())), + (Literal.of("apples".getBytes()), Literal.of("oranges".getBytes()), + Literal.of("apples".getBytes()), Literal.ofNull(new BinaryType())) + ) + + // Literal creation: (Literal, Literal) -> Expr(a, b) , + // Expected result: (Expr(small, big).eval(), Expr(big, small).eval(), Expr(small, small).eval() + // (Literal creation, Expected result) + val predicates = Seq( + ((a: Literal, b: Literal) => new LessThan(a, b), (true, false, false)), + ((a: Literal, b: Literal) => new LessThanOrEqual(a, b), (true, false, true)), + ((a: Literal, b: Literal) => new GreaterThan(a, b), (false, true, false)), + ((a: Literal, b: Literal) => new GreaterThanOrEqual(a, b), (false, true, true)), + ((a: Literal, b: Literal) => new EqualTo(a, b), (false, false, true)) + ) + + literals.foreach { case (small, big, small2, nullLit) => + predicates.foreach { case (predicateCreator, (smallBig, bigSmall, smallSmall)) => + testPredicate(predicateCreator(small, big), smallBig) + testPredicate(predicateCreator(big, small), bigSmall) + testPredicate(predicateCreator(small, small2), smallSmall) + testPredicate(predicateCreator(small, nullLit), null) + testPredicate(predicateCreator(nullLit, small), null) + } + } + + // more extensive comparison tests for custom-implemented binary comparison + + // in the Databricks SQL guide, BINARY values are initiated from a hexadecimal string, where + // each byte is represented by 2 digits (for a string of odd length, a 0 is prepended) + // A few examples: + // - X'0' == X'00' == [0] + // - X'001' == X'0001' == [0, 1] + // (see: https://docs.databricks.com/sql/language-manual/data-types/binary-type.html) + + // (small, big, small2) + val binaryLiterals = Seq( + (Array.empty[Int], Array(0), Array.empty[Int]), // [] < [0] or X'' < X'0' + (Array.empty[Int], Array(1), Array.empty[Int]), // [] < [1] or X'' < X'1' + (Array(0), Array(1), Array(0)), // [0] < [1] or X'0' < X'1' + (Array(0, 1), Array(1), Array(0, 1)), // [0, 1] < [1] or X'001' < X'1' + (Array(0), Array(0, 0), Array(0)), // [0] < [0, 0] or X'0' < X'000' + (Array(0), Array(0, 1), Array(0)), // [0] < [0, 1] or X'0' < X'001' + (Array(0, 1), Array(1, 0), Array(0, 1)), // [0, 1] < [1, 0] or X'001' < X'100' + (Array(0, 1), Array(0, 2), Array(0, 1)), // [0, 1] < [0, 2] or X'001' < X'002' + // [0, 0, 2] < [0, 1, 0] or X'00002' < X'00100' + (Array(0, 0, 2), Array(0, 1, 0), Array(0, 0, 2)) + ).map{ case (small, big, small2) => + (small.map(_.toByte), big.map(_.toByte), small2.map(_.toByte)) + } + + binaryLiterals.foreach { case (small, big, small2) => + predicates.foreach { case (predicateCreator, (smallBig, bigSmall, smallSmall)) => + testPredicate(predicateCreator(Literal.of(small), Literal.of(big)), smallBig) + testPredicate(predicateCreator(Literal.of(big), Literal.of(small)), bigSmall) + testPredicate(predicateCreator(Literal.of(small), Literal.of(small2)), smallSmall) + } + } + } + + test("null predicates") { + // ISNOTNULL tests + testPredicate(new IsNotNull(Literal.ofNull(new BooleanType())), false) + testPredicate(new IsNotNull(Literal.False), true) + + // ISNULL tests + testPredicate(new IsNull(Literal.ofNull(new BooleanType())), true) + testPredicate(new IsNull(Literal.False), false) + } + + test("In predicate") { + // IN TESTS + testException[IllegalArgumentException]( + new In(null, List(Literal.True, Literal.True).asJava), + "'In' expression 'value' cannot be null") + testException[IllegalArgumentException]( + new In(Literal.True, null), + "'In' expression 'elems' cannot be null") + testException[IllegalArgumentException]( + new In(Literal.True, List().asJava), + "'In' expression 'elems' cannot be empty") + // mismatched DataTypes throws exception + testException[IllegalArgumentException]( + new In(Literal.of(1), List(Literal.True, Literal.True).asJava), + "In expression 'elems' and 'value' must all be of the same DataType") + testException[IllegalArgumentException]( + new In(Literal.True, List(Literal.of(1), Literal.True).asJava), + "In expression 'elems' and 'value' must all be of the same DataType") + + // value.eval() null -> null + testPredicate(new In(Literal.ofNull(new BooleanType()), List(Literal.True).asJava), null) + // value in list (w/ null in list) + testPredicate(new In(Literal.True, List(Literal.True, + Literal.ofNull(new BooleanType())).asJava), true) + // value not in list (w/ null in list) + testPredicate(new In(Literal.False, List(Literal.True, + Literal.ofNull(new BooleanType())).asJava), null) + + // test correct output + // TODO: test all types? uses comparator same as the other comparison expressions + testPredicate( new In(Literal.of(1), + (0 to 10).map{Literal.of}.asJava), true) + testPredicate( new In(Literal.of(100), + (0 to 10).map{Literal.of}.asJava), false) + testPredicate( new In(Literal.of(10), + (0 to 10).map{Literal.of}.asJava), true) + } + + private def testLiteral(literal: Literal, expectedResult: Any) = { + assert(Objects.equals(literal.eval(null), expectedResult)) + } + + test("Literal tests") { + // LITERAL tests + testLiteral(Literal.True, true) + testLiteral(Literal.False, false) + testLiteral(Literal.of(8.toByte), 8.toByte) + testLiteral(Literal.of(1.0), 1.0) + testLiteral(Literal.of(2.0F), 2.0F) + testLiteral(Literal.of(5), 5) + testLiteral(Literal.of(10L), 10L) + testLiteral(Literal.ofNull(new BooleanType()), null) + testLiteral(Literal.ofNull(new IntegerType()), null) + testLiteral(Literal.of(5.toShort), 5.toShort) + testLiteral(Literal.of("test"), "test") + val curr_time = System.currentTimeMillis() + testLiteral( + Literal.of(new TimestampJ(curr_time)), new TimestampJ(curr_time)) + testLiteral(Literal.of(new DateJ(curr_time)), new DateJ(curr_time)) + testLiteral(Literal.of(new BigDecimalJ("0.1")), + new BigDecimalJ("0.1")) + assert(ArraysJ.equals( + Literal.of("test".getBytes()).eval(null).asInstanceOf[Array[Byte]], + "test".getBytes())) + + // Literal.ofNull(NullType) is prohibited + testException[IllegalArgumentException]( + Literal.ofNull(new NullType()), + "null is an invalid data type for Literal" + ) + // Literal.ofNull(ArrayType) is prohibited + testException[IllegalArgumentException]( + Literal.ofNull(new ArrayType(new IntegerType(), true)), + "array is an invalid data type for Literal" + ) + // Literal.ofNull(MapType) is prohibited + testException[IllegalArgumentException]( + Literal.ofNull(new MapType(new IntegerType(), new IntegerType(), true)), + "map is an invalid data type for Literal" + ) + // Literal.ofNull(StructType) is prohibited + testException[IllegalArgumentException]( + Literal.ofNull(new StructType(Array())), + "struct is an invalid data type for Literal" + ) + } + + private def testColumn(fieldName: String, dataType: DataType, record: RowRecord, + expectedResult: Any) = { + assert(Objects.equals(new Column(fieldName, dataType).eval(record), + expectedResult)) + } + + test("Column tests") { + // COLUMN tests + val schema = new StructType(Array( + new StructField("testInt", new IntegerType(), true), + new StructField("testLong", new LongType(), true), + new StructField("testByte", new ByteType(), true), + new StructField("testShort", new ShortType(), true), + new StructField("testBoolean", new BooleanType(), true), + new StructField("testFloat", new FloatType(), true), + new StructField("testDouble", new DoubleType(), true), + new StructField("testString", new StringType(), true), + new StructField("testBinary", new BinaryType(), true), + new StructField("testDecimal", DecimalType.USER_DEFAULT, true), + new StructField("testTimestamp", new TimestampType(), true), + new StructField("testDate", new DateType(), true))) + val partRowRecord = new PartitionRowRecord(schema, + Map("testInt"->"1", + "testLong"->"10", + "testByte" ->"8", + "testShort" -> "100", + "testBoolean" -> "true", + "testFloat" -> "20.0", + "testDouble" -> "22.0", + "testString" -> "onetwothree", + "testBinary" -> "\u0001\u0005\u0008", + "testDecimal" -> "0.123", + "testTimestamp" -> (new TimestampJ(12345678)).toString, + "testDate" -> "1970-01-01")) + + testColumn("testInt", new IntegerType(), partRowRecord, 1) + testColumn("testLong", new LongType(), partRowRecord, 10L) + testColumn("testByte", new ByteType(), partRowRecord, 8.toByte) + testColumn("testShort", new ShortType(), partRowRecord, 100.toShort) + testColumn("testBoolean", new BooleanType(), partRowRecord, true) + testColumn("testFloat", new FloatType(), partRowRecord, 20.0F) + testColumn("testDouble", new DoubleType(), partRowRecord, 22.0) + testColumn("testString", new StringType(), partRowRecord, "onetwothree") + assert(Array(1.toByte, 5.toByte, 8.toByte) sameElements + (new Column("testBinary", new BinaryType())).eval(partRowRecord).asInstanceOf[Array[Byte]]) + testColumn("testDecimal", DecimalType.USER_DEFAULT, partRowRecord, new BigDecimalJ("0.123")) + testColumn("testTimestamp", new TimestampType(), partRowRecord, new TimestampJ(12345678)) + testColumn("testDate", new DateType(), partRowRecord, new DateJ(70, 0, 1)) + + testException[UnsupportedOperationException]( + new Column("testArray", new ArrayType(new BooleanType(), true)), + "The data type of column testArray is array. This is not supported yet") + testException[UnsupportedOperationException]( + new Column("testMap", new MapType(new StringType(), new StringType(), true)), + "The data type of column testMap is map. This is not supported yet") + testException[UnsupportedOperationException]( + new Column("testStruct", new StructType(Array(new StructField("test", new BooleanType())))), + "The data type of column testStruct is struct. This is not supported yet") + } + + private def buildPartitionRowRecord( + dataType: DataType, + nullable: Boolean, + value: String, + name: String = "test") = { + new PartitionRowRecord(new StructType(Array(new StructField(name, dataType, nullable))), + Map(name -> value)) + } + + test("PartitionRowRecord tests") { + val testPartitionRowRecord = buildPartitionRowRecord(new IntegerType(), nullable = true, "5") + assert(buildPartitionRowRecord(new IntegerType(), nullable = true, null).isNullAt("test")) + assert(!buildPartitionRowRecord(new IntegerType(), nullable = true, "5").isNullAt("test")) + // non-nullable field + assert(buildPartitionRowRecord(new IntegerType(), nullable = false, null).isNullAt("test")) + + assert(!testPartitionRowRecord.isNullAt("test")) + testException[IllegalArgumentException]( + testPartitionRowRecord.isNullAt("foo"), + "Field \"foo\" does not exist.") + + // primitive types can't be null + // for primitive type T: (DataType, getter: partitionRowRecord => T, value: String, value: T) + val primTypes = Seq( + (new IntegerType(), (x: PartitionRowRecord) => x.getInt("test"), "0", 0), + (new LongType(), (x: PartitionRowRecord) => x.getLong("test"), "0", 0L), + (new ByteType(), (x: PartitionRowRecord) => x.getByte("test"), "0", 0.toByte), + (new ShortType(), (x: PartitionRowRecord) => x.getShort("test"), "0", 0.toShort), + (new BooleanType(), (x: PartitionRowRecord) => x.getBoolean("test"), "true", true), + (new FloatType(), (x: PartitionRowRecord) => x.getFloat("test"), "0", 0.0F), + (new DoubleType(), (x: PartitionRowRecord) => x.getDouble("test"), "0.0", 0.0) + ) + + primTypes.foreach { case (dataType: DataType, f: (PartitionRowRecord => Any), s: String, v) => + assert(f(buildPartitionRowRecord(dataType, nullable = true, s)) == v) + testException[NullPointerException]( + f(buildPartitionRowRecord(dataType, nullable = true, null)), + s"Read a null value for field test which is a primitive type") + testException[ClassCastException]( + f(buildPartitionRowRecord(new StringType(), nullable = true, "test")), + s"The data type of field test is string. Cannot cast it to ${dataType.getTypeName}") + testException[IllegalArgumentException]( + f(buildPartitionRowRecord(dataType, nullable = true, s, "foo")), + "Field \"test\" does not exist.") + } + + val curr_time = System.currentTimeMillis() + // non primitive types can be null ONLY when nullable (test both) + // for non-primitive type T: + // (DataType, getter: partitionRowRecord => T, value: String, value: T) + val nonPrimTypes = Seq( + (new StringType(), (x: PartitionRowRecord) => x.getString("test"), "foo", "foo"), + (DecimalType.USER_DEFAULT, (x: PartitionRowRecord) => x.getBigDecimal("test"), "0.01", + new BigDecimalJ("0.01")), + (new TimestampType(), (x: PartitionRowRecord) => x.getTimestamp("test"), + (new TimestampJ(curr_time)).toString, new TimestampJ(curr_time)), + (new DateType(), (x: PartitionRowRecord) => x.getDate("test"), "1970-01-01", + DateJ.valueOf("1970-01-01")) + ) + nonPrimTypes.foreach { + case (dataType: DataType, f: (PartitionRowRecord => Any), s: String, v) => + assert(Objects.equals(f(buildPartitionRowRecord(dataType, nullable = true, s)), v)) + assert(f(buildPartitionRowRecord(dataType, nullable = true, null)) == null) + testException[NullPointerException]( + f(buildPartitionRowRecord(dataType, nullable = false, null)), + "Read a null value for field test, yet schema indicates that this field can't be null.") + testException[ClassCastException]( + f(buildPartitionRowRecord(new IntegerType(), nullable = true, "test")), + s"The data type of field test is integer. Cannot cast it to ${dataType.getTypeName}") + testException[IllegalArgumentException]( + f(buildPartitionRowRecord(dataType, nullable = true, s, "foo")), + "Field \"test\" does not exist.") + } + + assert(buildPartitionRowRecord(new BinaryType(), nullable = true, "") + .getBinary("test").isEmpty) + assert(buildPartitionRowRecord(new BinaryType(), nullable = true, "\u0001\u0002") + .getBinary("test") sameElements Array(1.toByte, 2.toByte)) + testException[NullPointerException]( + buildPartitionRowRecord(new BinaryType(), nullable = false, null).getBinary("test"), + "Read a null value for field test, yet schema indicates that this field can't be null.") + testException[ClassCastException]( + buildPartitionRowRecord(new IntegerType(), nullable = true, "test").getBinary("test"), + s"The data type of field test is integer. Cannot cast it to binary") + testException[IllegalArgumentException]( + buildPartitionRowRecord(new BinaryType, nullable = true, "", "foo").getBinary("test"), + "Field \"test\" does not exist.") + + testException[UnsupportedOperationException]( + testPartitionRowRecord.getRecord("test"), + "Struct is not a supported partition type.") + testException[UnsupportedOperationException]( + testPartitionRowRecord.getList("test"), + "Array is not a supported partition type.") + intercept[UnsupportedOperationException]( + testPartitionRowRecord.getMap("test"), + "Map is not a supported partition type.") + } + + // TODO: add nested expression tree tests? + + private def testPartitionFilter(inputSchema: StructType, inputFiles: Seq[AddFile], + filters: Seq[Expression], expectedMatchedFiles: Seq[AddFile]) = { + val matchedFiles = DeltaLogImpl.filterFileList(inputSchema, inputFiles, filters) + assert(matchedFiles.length == expectedMatchedFiles.length) + assert(matchedFiles.forall(expectedMatchedFiles.contains(_))) + } + + test("basic partition filter") { + val schema = new StructType(Array( + new StructField("col1", new IntegerType()), + new StructField("col2", new IntegerType()))) + + val add00 = AddFile("1", Map("col1" -> "0", "col2" -> "0"), 0, 0, dataChange = true) + val add01 = AddFile("2", Map("col1" -> "0", "col2" -> "1"), 0, 0, dataChange = true) + val add02 = AddFile("2", Map("col1" -> "0", "col2" -> "2"), 0, 0, dataChange = true) + val add10 = AddFile("3", Map("col1" -> "1", "col2" -> "0"), 0, 0, dataChange = true) + val add11 = AddFile("4", Map("col1" -> "1", "col2" -> "1"), 0, 0, dataChange = true) + val add12 = AddFile("4", Map("col1" -> "1", "col2" -> "2"), 0, 0, dataChange = true) + val add20 = AddFile("4", Map("col1" -> "2", "col2" -> "0"), 0, 0, dataChange = true) + val add21 = AddFile("4", Map("col1" -> "2", "col2" -> "1"), 0, 0, dataChange = true) + val add22 = AddFile("4", Map("col1" -> "2", "col2" -> "2"), 0, 0, dataChange = true) + val inputFiles = Seq(add00, add01, add02, add10, add11, add12, add20, add21, add22) + + val f1Expr1 = new EqualTo(schema.column("col1"), Literal.of(0)) + val f1Expr2 = new EqualTo(schema.column("col2"), Literal.of(1)) + val f1 = new And(f1Expr1, f1Expr2) + + testPartitionFilter(schema, inputFiles, f1 :: Nil, add01 :: Nil) + testPartitionFilter(schema, inputFiles, f1Expr1 :: f1Expr2 :: Nil, add01 :: Nil) + + val f2Expr1 = new LessThan(schema.column("col1"), Literal.of(1)) + val f2Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f2 = new And(f2Expr1, f2Expr2) + testPartitionFilter(schema, inputFiles, f2 :: Nil, add00 :: Nil) + testPartitionFilter(schema, inputFiles, f2Expr1 :: f2Expr2 :: Nil, add00 :: Nil) + + val f3Expr1 = new EqualTo(schema.column("col1"), Literal.of(2)) + val f3Expr2 = new LessThan(schema.column("col2"), Literal.of(1)) + val f3 = new Or(f3Expr1, f3Expr2) + testPartitionFilter(schema, inputFiles, f3 :: Nil, Seq(add20, add21, add22, add00, add10)) + + val inSet4 = (2 to 10).map(Literal.of).asJava + val f4 = new In(schema.column("col1"), inSet4) + testPartitionFilter(schema, inputFiles, f4 :: Nil, add20 :: add21 :: add22 :: Nil) + + val inSet5 = (100 to 110).map(Literal.of).asJava + val f5 = new In(schema.column("col1"), inSet5) +// testPartitionFilter(schema, inputFiles, f5 :: Nil, Nil) + } + + // TODO: add additional partition filter tests + + test("not null partition filter") { + val schema = new StructType(Array( + new StructField("col1", new IntegerType(), true), + new StructField("col2", new IntegerType(), true))) + + val add0Null = AddFile("1", Map("col1" -> "0", "col2" -> null), 0, 0, dataChange = true) + val addNull1 = AddFile("1", Map("col1" -> null, "col2" -> "1"), 0, 0, dataChange = true) + val inputFiles = Seq(add0Null, addNull1) + + val f1 = new IsNotNull(schema.column("col1")) + testPartitionFilter(schema, inputFiles, f1 :: Nil, add0Null :: Nil) + } +} From 12a89a96657a2482b5b526979c675270b303f248 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Mon, 25 Oct 2021 16:40:52 -0700 Subject: [PATCH 112/291] [DSW] [28] API Documentation (#184) --- .../internal/util/ComparisonUtil.scala | 2 +- .../io/delta/standalone/CommitResult.java | 4 +- .../java/io/delta/standalone/DeltaLog.java | 11 ++-- .../java/io/delta/standalone/DeltaScan.java | 14 ++++- .../java/io/delta/standalone/Operation.java | 2 + .../standalone/OptimisticTransaction.java | 21 +++++-- .../java/io/delta/standalone/Snapshot.java | 4 +- .../java/io/delta/standalone/VersionLog.java | 12 +++- .../io/delta/standalone/actions/Action.java | 2 + .../delta/standalone/actions/AddCDCFile.java | 43 +++++++++++-- .../io/delta/standalone/actions/AddFile.java | 24 ++++++-- .../delta/standalone/actions/FileAction.java | 13 ++++ .../io/delta/standalone/actions/Metadata.java | 6 +- .../delta/standalone/actions/RemoveFile.java | 61 ++++++++++++++++++- .../standalone/actions/SetTransaction.java | 34 +++++++++-- .../exceptions/ConcurrentAppendException.java | 3 + .../ConcurrentDeleteDeleteException.java | 3 + .../ConcurrentDeleteReadException.java | 3 + .../ConcurrentTransactionException.java | 3 + .../DeltaConcurrentModificationException.java | 3 + .../exceptions/DeltaStandaloneException.java | 3 + .../exceptions/MetadataChangedException.java | 4 ++ .../exceptions/ProtocolChangedException.java | 3 + .../io/delta/standalone/types/StructType.java | 1 + .../util/ParquetSchemaConverter.java | 7 ++- .../internal/scan/DeltaScanImpl.scala | 2 + .../internal/scan/FilteredDeltaScanImpl.scala | 2 + .../internal/util/ConversionUtils.scala | 2 +- 28 files changed, 250 insertions(+), 42 deletions(-) diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala index 8c540c343fd..db06b21e58a 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala @@ -155,7 +155,7 @@ trait ComparisonUtil { standalone: io.delta.standalone.actions.SetTransaction, oss: org.apache.spark.sql.delta.actions.SetTransaction): Unit = { assert(standalone.getAppId == oss.appId) - assert(standalone.getVerion == oss.version) + assert(standalone.getVersion == oss.version) compareOptions(standalone.getLastUpdated, oss.lastUpdated) } diff --git a/standalone/src/main/java/io/delta/standalone/CommitResult.java b/standalone/src/main/java/io/delta/standalone/CommitResult.java index 49f7543f7b2..3ff4de448cb 100644 --- a/standalone/src/main/java/io/delta/standalone/CommitResult.java +++ b/standalone/src/main/java/io/delta/standalone/CommitResult.java @@ -1,7 +1,9 @@ +// TODO: copyright + package io.delta.standalone; /** - * Wrapper around the result of {@link OptimisticTransaction#commit(Iterable, Operation, String)}.. + * Wrapper around the result of {@link OptimisticTransaction#commit}. */ public class CommitResult { private final long version; diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index d03c5791fbe..9c9106cadf9 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -13,15 +13,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.delta.standalone; -import io.delta.standalone.actions.CommitInfo; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; +import java.util.Iterator; +import io.delta.standalone.actions.CommitInfo; import io.delta.standalone.internal.DeltaLogImpl; -import java.util.Iterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; /** * {@link DeltaLog} is the representation of the transaction logs of a Delta table. It provides APIs @@ -76,7 +77,7 @@ public interface DeltaLog { * Note that all reads in a transaction must go through the returned transaction object, and not * directly to the {@link DeltaLog} otherwise they will not be checked for conflicts. * - * @return a new OptimisticTransaction + * @return a new {@link OptimisticTransaction}. */ OptimisticTransaction startTransaction(); diff --git a/standalone/src/main/java/io/delta/standalone/DeltaScan.java b/standalone/src/main/java/io/delta/standalone/DeltaScan.java index 0c14d7b7be8..a2fa70372e4 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaScan.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaScan.java @@ -16,12 +16,17 @@ package io.delta.standalone; +import java.util.Optional; + import io.delta.standalone.actions.AddFile; import io.delta.standalone.data.CloseableIterator; import io.delta.standalone.expressions.Expression; -import java.util.Optional; - +/** + * Provides access to an iterator over the files in this snapshot. + * + * Typically created with a read predicate {@link Expression} to let users filter files. + */ public interface DeltaScan { /** @@ -36,6 +41,11 @@ public interface DeltaScan { */ CloseableIterator getFiles(); + /** + * @return the input predicate used to filter files. + */ + Optional getInputPredicate(); + /** * @return portion of the input predicate that can be evaluated by Delta Standalone using only * metadata. Files returned by {@link #getFiles()} are guaranteed to satisfy the pushed diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index 1a62a7b1d9f..f2cba617c6f 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -1,3 +1,5 @@ +// TODO: copyright + package io.delta.standalone; import javax.annotation.Nonnull; diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 6362a59083e..89f4f45f3a7 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -1,12 +1,19 @@ +// TODO: copyright + package io.delta.standalone; import io.delta.standalone.actions.Action; -import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.Metadata; import io.delta.standalone.expressions.Expression; -import java.util.List; - +/** + * Used to perform a set of reads in a transaction and then commit a set of updates to the + * state of the log. All reads from the {@link DeltaLog}, MUST go through this instance rather + * than directly to the {@link DeltaLog} otherwise they will not be checked for logical conflicts + * with concurrent updates. + * + * This class is not thread-safe. + */ public interface OptimisticTransaction { /** @@ -14,6 +21,8 @@ public interface OptimisticTransaction { * the given `lastVersion`. In the case of a conflict with a concurrent writer this * method will throw an exception. * + * @param A derived class of {@link Action}. This allows, for example, both a + * {@code List} and a {@code List} to be accepted. * @param actions Set of actions to commit. * @param op Details of operation that is performing this transactional commit. * @param engineInfo String used to identify the writer engine. It should resemble @@ -35,11 +44,11 @@ public interface OptimisticTransaction { * - After TXN1 starts, another transaction TXN2 reads partition 'date=2021-09-07' and commits * first at table version N (with no other metadata changes). * - TXN1 sees that another commit won, and needs to know whether to commit at version N+1 or - * fail. Using the `readPredicates` and resultant `readFiles`, TXN1 can see that none of its - * readFiles were changed by TXN2. Thus there are no logical conflicts and TXN1 can commit at + * fail. Using the `readPredicate` and resultant `readFiles`, TXN1 can see that none of its + * read files were changed by TXN2. Thus there are no logical conflicts and TXN1 can commit at * table version N+1. * - * @param readPredicate Predicates used to determine which files were read. + * @param readPredicate Predicate used to determine which files were read. * @return a {@link DeltaScan} containing the list of files matching the push portion of the * readPredicate. */ diff --git a/standalone/src/main/java/io/delta/standalone/Snapshot.java b/standalone/src/main/java/io/delta/standalone/Snapshot.java index 83715d24c39..cec74733741 100644 --- a/standalone/src/main/java/io/delta/standalone/Snapshot.java +++ b/standalone/src/main/java/io/delta/standalone/Snapshot.java @@ -16,14 +16,14 @@ package io.delta.standalone; +import java.util.List; + import io.delta.standalone.actions.AddFile; import io.delta.standalone.actions.Metadata; import io.delta.standalone.data.CloseableIterator; import io.delta.standalone.data.RowRecord; import io.delta.standalone.expressions.Expression; -import java.util.List; - /** * {@link Snapshot} provides APIs to access the Delta table state (such as table metadata, active * files) at some version. diff --git a/standalone/src/main/java/io/delta/standalone/VersionLog.java b/standalone/src/main/java/io/delta/standalone/VersionLog.java index 3234616d33e..03e0c1ce994 100644 --- a/standalone/src/main/java/io/delta/standalone/VersionLog.java +++ b/standalone/src/main/java/io/delta/standalone/VersionLog.java @@ -1,19 +1,24 @@ -package io.delta.standalone; +// TODO: copyright -import io.delta.standalone.actions.Action; +package io.delta.standalone; +import javax.annotation.Nonnull; import java.util.Collections; import java.util.List; +import io.delta.standalone.actions.Action; + /** * {@link VersionLog} is the representation of all actions (changes) to the Delta Table * at a specific table version. */ public class VersionLog { private final long version; + + @Nonnull private final List actions; - public VersionLog(long version, List actions) { + public VersionLog(long version, @Nonnull List actions) { this.version = version; this.actions = actions; } @@ -28,6 +33,7 @@ public long getVersion() { /** * @return an unmodifiable {@code List} of the actions for this table version */ + @Nonnull public List getActions() { return Collections.unmodifiableList(actions); } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Action.java b/standalone/src/main/java/io/delta/standalone/actions/Action.java index e7378d944e7..30c3d5e5dd7 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Action.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Action.java @@ -1,3 +1,5 @@ +// TODO: copyright + package io.delta.standalone.actions; /** diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java index 9d83e494d13..e2e21d32f9f 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java @@ -1,35 +1,70 @@ +// TODO: copyright + package io.delta.standalone.actions; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; -public class AddCDCFile implements FileAction { +/** + * A change file containing CDC data for the Delta version it's within. Non-CDC readers should + * ignore this, CDC readers should scan all ChangeFiles in a version rather than computing + * changes from AddFile and RemoveFile actions. + */ +public final class AddCDCFile implements FileAction { + @Nonnull private final String path; + + @Nonnull private final Map partitionValues; + private final long size; + + @Nullable private final Map tags; - public AddCDCFile(String path, Map partitionValues, long size, - Map tags) { + public AddCDCFile(@Nonnull String path, @Nonnull Map partitionValues, long size, + @Nullable Map tags) { this.path = path; this.partitionValues = partitionValues; this.size = size; this.tags = tags; } + /** + * @return the relative path or the absolute path that should be added to the table. If it's a + * relative path, it's relative to the root of the table. Note: the path is encoded and + * should be decoded by {@code new java.net.URI(path)} when using it. + */ @Override + @Nonnull public String getPath() { return path; } + /** + * @return an unmodifiable {@code Map} from partition column to value for + * this file. Partition values are stored as strings, using the following formats. + * An empty string for any type translates to a null partition value. + * @see Delta Protocol Partition Value Serialization + */ + @Nonnull public Map getPartitionValues() { - return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; + return Collections.unmodifiableMap(partitionValues); } + /** + * @return the size of this file in bytes + */ public long getSize() { return size; } + /** + * @return an unmodifiable {@code Map} containing metadata about this file + */ + @Nullable public Map getTags() { return tags != null ? Collections.unmodifiableMap(tags) : null; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java index 95c936ef2ab..0cf32802a59 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java @@ -16,6 +16,8 @@ package io.delta.standalone.actions; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; import java.util.Objects; @@ -30,17 +32,27 @@ * @see Delta Transaction Log Protocol */ public final class AddFile implements FileAction { + @Nonnull private final String path; + + @Nonnull private final Map partitionValues; + private final long size; + private final long modificationTime; + private final boolean dataChange; + + @Nullable private final String stats; + + @Nullable private final Map tags; - public AddFile(String path, Map partitionValues, long size, - long modificationTime, boolean dataChange, String stats, - Map tags) { + public AddFile(@Nonnull String path, @Nonnull Map partitionValues, long size, + long modificationTime, boolean dataChange, @Nullable String stats, + @Nullable Map tags) { this.path = path; this.partitionValues = partitionValues; this.size = size; @@ -56,6 +68,7 @@ public AddFile(String path, Map partitionValues, long size, * should be decoded by {@code new java.net.URI(path)} when using it. */ @Override + @Nonnull public String getPath() { return path; } @@ -66,8 +79,9 @@ public String getPath() { * An empty string for any type translates to a null partition value. * @see Delta Protocol Partition Value Serialization */ + @Nonnull public Map getPartitionValues() { - return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; + return Collections.unmodifiableMap(partitionValues); } /** @@ -99,6 +113,7 @@ public boolean isDataChange() { * @return statistics (for example: count, min/max values for columns) * about the data in this file as serialized JSON */ + @Nullable public String getStats() { return stats; } @@ -106,6 +121,7 @@ public String getStats() { /** * @return an unmodifiable {@code Map} containing metadata about this file */ + @Nullable public Map getTags() { return tags != null ? Collections.unmodifiableMap(tags) : null; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/FileAction.java b/standalone/src/main/java/io/delta/standalone/actions/FileAction.java index 221384e6d04..f59ca355594 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/FileAction.java +++ b/standalone/src/main/java/io/delta/standalone/actions/FileAction.java @@ -1,8 +1,21 @@ +// TODO: copyright + package io.delta.standalone.actions; +/** + * Generic interface for {@link Action}s pertaining to the addition and removal of files. + */ public interface FileAction extends Action { + /** + @return the relative path or the absolute path of the file being added or removed by this + * action. If it's a relative path, it's relative to the root of the table. Note: the path + * is encoded and should be decoded by {@code new java.net.URI(path)} when using it. + */ String getPath(); + /** + * @return whether any data was changed as a result of this file being added or removed. + */ boolean isDataChange(); } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index 52fabe01cfd..76ba5bfba90 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -16,7 +16,11 @@ package io.delta.standalone.actions; -import java.util.*; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; import io.delta.standalone.types.StructType; diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java index 27f8bd64e65..efdc2c9ac8f 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java @@ -1,22 +1,46 @@ +// TODO: copyright + package io.delta.standalone.actions; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Optional; +/** + * Logical removal of a given file from the reservoir. Acts as a tombstone before a file is + * deleted permanently. + * + * Note that for protocol compatibility reasons, the fields {@code partitionValues}, {@code size}, + * and {@code tags} are only present when the extendedFileMetadata flag is true. New writers should + * generally be setting this flag, but old writers (and FSCK) won't, so readers must check this flag + * before attempting to consume those values. + */ public class RemoveFile implements FileAction { + @Nonnull private final String path; + + @Nonnull private final Optional deletionTimestamp; + private final boolean dataChange; + private final boolean extendedFileMetadata; + + @Nullable private final Map partitionValues; + private final long size; + + @Nullable private final Map tags; - public RemoveFile(String path, Optional deletionTimestamp, boolean dataChange, - boolean extendedFileMetadata, Map partitionValues, long size, - Map tags) { + public RemoveFile(@Nonnull String path, @Nonnull Optional deletionTimestamp, + boolean dataChange, boolean extendedFileMetadata, + @Nullable Map partitionValues, long size, + @Nullable Map tags) { this.path = path; this.deletionTimestamp = deletionTimestamp; this.dataChange = dataChange; @@ -26,32 +50,63 @@ public RemoveFile(String path, Optional deletionTimestamp, boolean dataCha this.tags = tags; } + /** + * @return the relative path or the absolute path that should be removed from the table. If it's + * a relative path, it's relative to the root of the table. Note: the path is encoded + * and should be decoded by {@code new java.net.URI(path)} when using it. + */ @Override public String getPath() { return path; } + /** + * @return the time that this file was deleted as milliseconds since the epoch + */ public Optional getDeletionTimestamp() { return deletionTimestamp; } + /** + * @return whether any data was changed as a result of this file being created. When + * {@code false} the file must already be present in the table or the records in the + * added file must be contained in one or more remove actions in the same version + */ @Override public boolean isDataChange() { return dataChange; } + /** + * @return true if the fields {@code partitionValues}, {@code size}, and {@code tags} are + * present + */ public boolean isExtendedFileMetadata() { return extendedFileMetadata; } + /** + * @return an unmodifiable {@code Map} from partition column to value for + * this file. Partition values are stored as strings, using the following formats. + * An empty string for any type translates to a null partition value. + * @see Delta Protocol Partition Value Serialization + */ + @Nullable public Map getPartitionValues() { return partitionValues != null ? Collections.unmodifiableMap(partitionValues) : null; } + /** + * @return the size of this file in bytes + */ public long getSize() { return size; } + /** + * @return an unmodifiable {@code Map} containing metadata about this file + */ + @Nullable public Map getTags() { return tags != null ? Collections.unmodifiableMap(tags) : null; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java index 64ef81a9c8d..69171a11f86 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java @@ -1,26 +1,48 @@ +// TODO: copyright + package io.delta.standalone.actions; +import javax.annotation.Nonnull; import java.util.Optional; -public class SetTransaction implements Action { +/** + * Sets the committed version for a given application. Used to make operations like streaming append + * idempotent. + */ +public final class SetTransaction implements Action { + @Nonnull private final String appId; - private final long verion; + + private final long version; + + @Nonnull private final Optional lastUpdated; - public SetTransaction(String appId, long verion, Optional lastUpdated) { + public SetTransaction(@Nonnull String appId, long version, @Nonnull Optional lastUpdated) { this.appId = appId; - this.verion = verion; + this.version = version; this.lastUpdated = lastUpdated; } + /** + * @return the application ID + */ + @Nonnull public String getAppId() { return appId; } - public long getVerion() { - return verion; + /** + * @return the committed version for the application ID + */ + public long getVersion() { + return version; } + /** + * @return the last updated timestamp of this transaction (milliseconds since the epoch) + */ + @Nonnull public Optional getLastUpdated() { return lastUpdated; } diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java index 603024e8f01..b9755a7ab07 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentAppendException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when files are added that would have been read by the current transaction. + */ public class ConcurrentAppendException extends DeltaConcurrentModificationException { public ConcurrentAppendException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java index 0f2b529b187..6593f32083a 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when the current transaction deletes data that was deleted by a concurrent transaction. + */ public class ConcurrentDeleteDeleteException extends DeltaConcurrentModificationException { public ConcurrentDeleteDeleteException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java index f5163c7bb6d..e950d2c03c3 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when the current transaction reads data that was deleted by a concurrent transaction. + */ public class ConcurrentDeleteReadException extends DeltaConcurrentModificationException { public ConcurrentDeleteReadException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java index 1ab4ec95df0..0960d3204c4 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ConcurrentTransactionException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when concurrent transaction both attempt to update the same idempotent transaction. + */ public class ConcurrentTransactionException extends DeltaConcurrentModificationException { public ConcurrentTransactionException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java index 5100a6ed202..82b34ad0de8 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.java @@ -18,6 +18,9 @@ import java.util.ConcurrentModificationException; +/** + * The basic class for all Delta Standalone commit conflict exceptions. + */ public class DeltaConcurrentModificationException extends ConcurrentModificationException { public DeltaConcurrentModificationException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java index ee516a19273..8448f03b617 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/DeltaStandaloneException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when a query fails, usually because the query itself is invalid. + */ public class DeltaStandaloneException extends RuntimeException { public DeltaStandaloneException() { super(); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java b/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java index e0193e49b57..7d749751673 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/MetadataChangedException.java @@ -16,6 +16,10 @@ package io.delta.standalone.exceptions; +/** + * Thrown when the metadata of the Delta table has changed between the time of read + * and the time of commit. + */ public class MetadataChangedException extends DeltaConcurrentModificationException { public MetadataChangedException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java b/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java index 9986e09e751..14eb3453f0e 100644 --- a/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java +++ b/standalone/src/main/java/io/delta/standalone/exceptions/ProtocolChangedException.java @@ -16,6 +16,9 @@ package io.delta.standalone.exceptions; +/** + * Thrown when the protocol version has changed between the time of read and the time of commit. + */ public class ProtocolChangedException extends DeltaConcurrentModificationException { public ProtocolChangedException(String message) { super(message); diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java index aa13d53c54c..7c41292c59d 100644 --- a/standalone/src/main/java/io/delta/standalone/types/StructType.java +++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java @@ -91,6 +91,7 @@ public StructType add(StructField field) { * .add("c", new StringType()) * } * @param fieldName The name of the new field to add. + * @param dataType The datatype for the new field. * @return The new {@link StructType}. */ public StructType add(String fieldName, DataType dataType) { diff --git a/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java index 335cdba389b..39346ea72c7 100644 --- a/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java +++ b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java @@ -16,11 +16,11 @@ package io.delta.standalone.util; -import org.apache.parquet.schema.MessageType; - import io.delta.standalone.internal.util.SparkToParquetSchemaConverter; import io.delta.standalone.types.StructType; +import org.apache.parquet.schema.MessageType; + /** * Converter class to convert {@link StructType} to Parquet {@link MessageType}. */ @@ -99,7 +99,8 @@ public static MessageType sparkToParquet(StructType schema, ParquetOutputTimesta * @param outputTimestampType which parquet timestamp type to use when writing * @return {@code schema} as a Parquet {@link MessageType} * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s) - */ public static MessageType sparkToParquet( + */ + public static MessageType sparkToParquet( StructType schema, Boolean writeLegacyParquetFormat, ParquetOutputTimestampType outputTimestampType) { diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala index 955163617a1..e90a1a7e184 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala @@ -83,6 +83,8 @@ private[internal] class DeltaScanImpl(files: Seq[AddFile]) extends DeltaScan { override def close(): Unit = { } } + override def getInputPredicate: Optional[Expression] = Optional.empty() + override def getPushedPredicate: Optional[Expression] = Optional.empty() override def getResidualPredicate: Optional[Expression] = Optional.empty() diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala index 317427205d4..d3d88b5bbab 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala @@ -48,6 +48,8 @@ final private[internal] class FilteredDeltaScanImpl( result.asInstanceOf[Boolean] } + override def getInputPredicate: Optional[Expression] = Optional.of(expr) + override def getPushedPredicate: Optional[Expression] = Optional.ofNullable(metadataConjunction.orNull) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala index 54985ef5e1e..aedf8071dae 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala @@ -307,7 +307,7 @@ private[internal] object ConversionUtils { def convertSetTransactionJ(external: SetTransactionJ): SetTransaction = { SetTransaction( external.getAppId, - external.getVerion, + external.getVersion, external.getLastUpdated ) } From 78a3c9eef5b308cb304b3f48d47f45be2a6151f1 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Thu, 28 Oct 2021 09:20:55 -0700 Subject: [PATCH 113/291] [#137] Shade jackson and json4s jars (#187) Closes #137 --- .github/workflows/test.yaml | 2 + build.sbt | 164 ++++++++++++++++-- .../java/io/delta/standalone/DeltaLog.java | 2 +- .../src/test/scala/ShadedJarSuite.scala | 39 +++++ 4 files changed, 195 insertions(+), 12 deletions(-) create mode 100644 testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 7066a69ed26..f75e558f9fd 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -27,7 +27,9 @@ jobs: build/sbt "++ 2.12.8 hiveMR/test" build/sbt "++ 2.12.8 hiveTez/test" build/sbt "++ 2.12.8 sqlDeltaImport/test" + build/sbt "++ 2.12.8 testStandaloneCosmetic/test" build/sbt "++ 2.11.12 standalone/test" build/sbt "++ 2.11.12 hive/test" build/sbt "++ 2.11.12 hiveMR/test" build/sbt "++ 2.11.12 hiveTez/test" + build/sbt "++ 2.11.12 testStandaloneCosmetic/test" diff --git a/build.sbt b/build.sbt index 62db714c201..ba4be3affb2 100644 --- a/build.sbt +++ b/build.sbt @@ -14,7 +14,11 @@ * limitations under the License. */ +// scalastyle:off line.size.limit + import ReleaseTransformations._ +import scala.xml.{Node => XmlNode, NodeSeq => XmlNodeSeq, _} +import scala.xml.transform._ parallelExecution in ThisBuild := false scalastyleConfig in ThisBuild := baseDirectory.value / "scalastyle-config.xml" @@ -28,6 +32,9 @@ val hadoopVersion = "3.1.0" val hiveVersion = "3.1.2" val tezVersion = "0.9.2" val hiveDeltaVersion = "0.5.0" +val parquet4sVersion = "1.2.1" +val parquetHadoopVersion = "1.10.1" +val scalaTestVersion = "3.0.5" lazy val commonSettings = Seq( organization := "io.delta", @@ -257,17 +264,109 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") ) ) -lazy val standalone = (project in file("standalone")) +/** + * We want to publish the `standalone` project's shaded JAR (created from the + * build/sbt standalone/assembly command). + * + * However, build/sbt standalone/publish and build/sbt standalone/publishLocal will use the + * non-shaded JAR from the build/sbt standalone/package command. + * + * So, we create an impostor, cosmetic project used only for publishing. + * + * build/sbt standaloneCosmetic/package + * - creates connectors/standalone/target/scala-2.12/delta-standalone-original-shaded_2.12-0.2.1-SNAPSHOT.jar + * (this is the shaded JAR we want) + * + * build/sbt standaloneCosmetic/publishLocal + * - packages the shaded JAR (above) and then produces: + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/poms/delta-standalone_2.12.pom + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/jars/delta-standalone_2.12.jar + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/srcs/delta-standalone_2.12-sources.jar + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/docs/delta-standalone_2.12-javadoc.jar + */ +lazy val standaloneCosmetic = project .settings( name := "delta-standalone", commonSettings, releaseSettings, + pomPostProcess := { (node: XmlNode) => + val hardcodeDeps = new RewriteRule { + override def transform(n: XmlNode): XmlNodeSeq = n match { + case e: Elem if e != null && e.label == "dependencies" => + + + org.scala-lang + scala-library + {scalaVersion.value} + + + org.apache.hadoop + hadoop-client + {hadoopVersion} + provided + + + org.apache.parquet + parquet-hadoop + {parquetHadoopVersion} + provided + + + com.github.mjakubowski84 + parquet4s-core_{scalaBinaryVersion.value} + {parquet4sVersion} + + + org.slf4j + slf4j-api + + + org.apache.parquet + parquet-hadoop + + + + + org.scalatest + scalatest_{scalaBinaryVersion.value} + {scalaTestVersion} + test + + + case _ => n + } + } + new RuleTransformer(hardcodeDeps).transform(node).head + }, + exportJars := true, + packageBin in Compile := (assembly in standalone).value + ) + +lazy val testStandaloneCosmetic = project.dependsOn(standaloneCosmetic) + .settings( + name := "test-standalone-cosmetic", + commonSettings, + skipReleaseSettings, + libraryDependencies ++= Seq( + "org.scalatest" %% "scalatest" % "3.0.5" % "test" + ) + ) + +lazy val standalone = (project in file("standalone")) + .enablePlugins(GenJavadocPlugin, JavaUnidocPlugin) + .settings( + name := "delta-standalone-original", + skip in publish := true, + commonSettings, + skipReleaseSettings, mimaSettings, unmanagedResourceDirectories in Test += file("golden-tables/src/test/resources"), + // When updating any dependency here, we should also review `pomPostProcess` in project + // `standaloneCosmetic` and update it accordingly. libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", - "org.apache.parquet" % "parquet-hadoop" % "1.10.1" % "provided", - "com.github.mjakubowski84" %% "parquet4s-core" % "1.2.1" excludeAll ( + "org.apache.parquet" % "parquet-hadoop" % parquetHadoopVersion % "provided", + "com.github.mjakubowski84" %% "parquet4s-core" % parquet4sVersion excludeAll ( ExclusionRule("org.slf4j", "slf4j-api"), ExclusionRule("org.apache.parquet", "parquet-hadoop") ), @@ -276,15 +375,58 @@ lazy val standalone = (project in file("standalone")) ExclusionRule("com.fasterxml.jackson.core"), ExclusionRule("com.fasterxml.jackson.module") ), - "org.scalatest" %% "scalatest" % "3.0.5" % "test" - )) + "org.scalatest" %% "scalatest" % scalaTestVersion % "test" + ), - /** - * Unidoc settings - * Generate javadoc with `unidoc` command, outputs to `standalone/target/javaunidoc` - */ - .enablePlugins(GenJavadocPlugin, JavaUnidocPlugin) - .settings( + /** + * Standalone packaged (unshaded) jar. + * + * Build with `build/sbt standalone/package` command. + * e.g. connectors/standalone/target/scala-2.12/delta-standalone-original-unshaded_2.12-0.2.1-SNAPSHOT.jar + */ + artifactName := { (sv: ScalaVersion, module: ModuleID, artifact: Artifact) => + artifact.name + "-unshaded" + "_" + sv.binary + "-" + module.revision + "." + artifact.extension + }, + + /** + * Standalone assembly (shaded) jar. This is what we want to release. + * + * Build with `build/sbt standalone/assembly` command. + * e.g. connectors/standalone/target/scala-2.12/delta-standalone-original-shaded_2.12-0.2.1-SNAPSHOT.jar + */ + logLevel in assembly := Level.Info, + test in assembly := {}, + assemblyJarName in assembly := s"${name.value}-shaded_${scalaBinaryVersion.value}-${version.value}.jar", + // we exclude jars first, and then we shade what is remaining + assemblyExcludedJars in assembly := { + val cp = (fullClasspath in assembly).value + val allowedPrefixes = Set("META_INF", "io", "json4s", "jackson", "paranamer") + cp.filter { f => + !allowedPrefixes.exists(prefix => f.data.getName.startsWith(prefix)) + } + }, + assemblyShadeRules in assembly := Seq( + ShadeRule.rename("com.fasterxml.jackson.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("com.thoughtworks.paranamer.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("org.json4s.**" -> "shadedelta.@0").inAll + ), + assemblyMergeStrategy in assembly := { + // Discard `module-info.class` to fix the `different file contents found` error. + // TODO Upgrade SBT to 1.5 which will do this automatically + case "module-info.class" => MergeStrategy.discard + case x => + val oldStrategy = (assemblyMergeStrategy in assembly).value + oldStrategy(x) + }, + artifact in assembly := { + val art = (artifact in assembly).value + art.withClassifier(Some("assembly")) + }, + addArtifact(artifact in assembly, assembly), + /** + * Unidoc settings + * Generate javadoc with `unidoc` command, outputs to `standalone/target/javaunidoc` + */ javacOptions in (JavaUnidoc, unidoc) := Seq( "-public", "-windowtitle", "Delta Standalone Reader " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index 6c15846dadb..378687eb3d4 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -37,7 +37,7 @@ public interface DeltaLog { /** * @return the current {@link Snapshot} of the Delta table. You may need to call - * {@link #update()} to access the latest snapshot if the current snapshot is stale. + * {@link #update()} to access the latest snapshot if the current snapshot is stale. */ Snapshot snapshot(); diff --git a/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala b/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala new file mode 100644 index 00000000000..5c38ae234bf --- /dev/null +++ b/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala @@ -0,0 +1,39 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.scalatest.FunSuite + +class ShadedJarSuite extends FunSuite { + test("properly shaded jars") { + // scalastyle:off classforname + Class.forName("io.delta.standalone.DeltaLog") + + Class.forName("shadedelta.com.fasterxml.jackson.databind.JavaType") + Class.forName("shadedelta.com.thoughtworks.paranamer.AdaptiveParanamer") + Class.forName("shadedelta.org.json4s.JsonDSL") + + intercept[ClassNotFoundException] { + Class.forName("com.fasterxml.jackson.databind.JavaType") + } + intercept[ClassNotFoundException] { + Class.forName("com.thoughtworks.paranamer.AdaptiveParanamer") + } + intercept[ClassNotFoundException] { + Class.forName("org.json4s.JsonDSL") + } + // scalastyle:on classforname + } +} From a57ed24db49165ba54c0365a3c036772ff19e60f Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 28 Oct 2021 11:48:03 -0700 Subject: [PATCH 114/291] merged with master --- .github/workflows/test.yaml | 2 + build.sbt | 164 ++++++++++++++++-- .../java/io/delta/standalone/DeltaLog.java | 2 +- .../src/test/scala/ShadedJarSuite.scala | 39 +++++ 4 files changed, 195 insertions(+), 12 deletions(-) create mode 100644 testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 7066a69ed26..f75e558f9fd 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -27,7 +27,9 @@ jobs: build/sbt "++ 2.12.8 hiveMR/test" build/sbt "++ 2.12.8 hiveTez/test" build/sbt "++ 2.12.8 sqlDeltaImport/test" + build/sbt "++ 2.12.8 testStandaloneCosmetic/test" build/sbt "++ 2.11.12 standalone/test" build/sbt "++ 2.11.12 hive/test" build/sbt "++ 2.11.12 hiveMR/test" build/sbt "++ 2.11.12 hiveTez/test" + build/sbt "++ 2.11.12 testStandaloneCosmetic/test" diff --git a/build.sbt b/build.sbt index 20193d8aa94..b0a44d3ca09 100644 --- a/build.sbt +++ b/build.sbt @@ -14,7 +14,11 @@ * limitations under the License. */ +// scalastyle:off line.size.limit + import ReleaseTransformations._ +import scala.xml.{Node => XmlNode, NodeSeq => XmlNodeSeq, _} +import scala.xml.transform._ parallelExecution in ThisBuild := false scalastyleConfig in ThisBuild := baseDirectory.value / "scalastyle-config.xml" @@ -26,6 +30,10 @@ lazy val testScalastyle = taskKey[Unit]("testScalastyle") val hadoopVersion = "3.1.0" val hiveVersion = "3.1.2" val tezVersion = "0.9.2" +val hiveDeltaVersion = "0.5.0" +val parquet4sVersion = "1.2.1" +val parquetHadoopVersion = "1.10.1" +val scalaTestVersion = "3.0.5" lazy val commonSettings = Seq( organization := "io.delta", @@ -255,17 +263,109 @@ lazy val hiveTez = (project in file("hive-tez")) dependsOn(hive % "test->test") ) ) -lazy val standalone = (project in file("standalone")) +/** + * We want to publish the `standalone` project's shaded JAR (created from the + * build/sbt standalone/assembly command). + * + * However, build/sbt standalone/publish and build/sbt standalone/publishLocal will use the + * non-shaded JAR from the build/sbt standalone/package command. + * + * So, we create an impostor, cosmetic project used only for publishing. + * + * build/sbt standaloneCosmetic/package + * - creates connectors/standalone/target/scala-2.12/delta-standalone-original-shaded_2.12-0.2.1-SNAPSHOT.jar + * (this is the shaded JAR we want) + * + * build/sbt standaloneCosmetic/publishLocal + * - packages the shaded JAR (above) and then produces: + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/poms/delta-standalone_2.12.pom + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/jars/delta-standalone_2.12.jar + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/srcs/delta-standalone_2.12-sources.jar + * -- .ivy2/local/io.delta/delta-standalone_2.12/0.2.1-SNAPSHOT/docs/delta-standalone_2.12-javadoc.jar + */ +lazy val standaloneCosmetic = project .settings( name := "delta-standalone", commonSettings, releaseSettings, + pomPostProcess := { (node: XmlNode) => + val hardcodeDeps = new RewriteRule { + override def transform(n: XmlNode): XmlNodeSeq = n match { + case e: Elem if e != null && e.label == "dependencies" => + + + org.scala-lang + scala-library + {scalaVersion.value} + + + org.apache.hadoop + hadoop-client + {hadoopVersion} + provided + + + org.apache.parquet + parquet-hadoop + {parquetHadoopVersion} + provided + + + com.github.mjakubowski84 + parquet4s-core_{scalaBinaryVersion.value} + {parquet4sVersion} + + + org.slf4j + slf4j-api + + + org.apache.parquet + parquet-hadoop + + + + + org.scalatest + scalatest_{scalaBinaryVersion.value} + {scalaTestVersion} + test + + + case _ => n + } + } + new RuleTransformer(hardcodeDeps).transform(node).head + }, + exportJars := true, + packageBin in Compile := (assembly in standalone).value + ) + +lazy val testStandaloneCosmetic = project.dependsOn(standaloneCosmetic) + .settings( + name := "test-standalone-cosmetic", + commonSettings, + skipReleaseSettings, + libraryDependencies ++= Seq( + "org.scalatest" %% "scalatest" % "3.0.5" % "test" + ) + ) + +lazy val standalone = (project in file("standalone")) + .enablePlugins(GenJavadocPlugin, JavaUnidocPlugin) + .settings( + name := "delta-standalone-original", + skip in publish := true, + commonSettings, + skipReleaseSettings, mimaSettings, unmanagedResourceDirectories in Test += file("golden-tables/src/test/resources"), + // When updating any dependency here, we should also review `pomPostProcess` in project + // `standaloneCosmetic` and update it accordingly. libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", - "org.apache.parquet" % "parquet-hadoop" % "1.10.1" % "provided", - "com.github.mjakubowski84" %% "parquet4s-core" % "1.2.1" excludeAll ( + "org.apache.parquet" % "parquet-hadoop" % parquetHadoopVersion % "provided", + "com.github.mjakubowski84" %% "parquet4s-core" % parquet4sVersion excludeAll ( ExclusionRule("org.slf4j", "slf4j-api"), ExclusionRule("org.apache.parquet", "parquet-hadoop") ), @@ -289,15 +389,57 @@ lazy val standalone = (project in file("standalone")) |} |""".stripMargin) Seq(file) - } - ) + }, - /** - * Unidoc settings - * Generate javadoc with `unidoc` command, outputs to `standalone/target/javaunidoc` - */ - .enablePlugins(GenJavadocPlugin, JavaUnidocPlugin) - .settings( + /** + * Standalone packaged (unshaded) jar. + * + * Build with `build/sbt standalone/package` command. + * e.g. connectors/standalone/target/scala-2.12/delta-standalone-original-unshaded_2.12-0.2.1-SNAPSHOT.jar + */ + artifactName := { (sv: ScalaVersion, module: ModuleID, artifact: Artifact) => + artifact.name + "-unshaded" + "_" + sv.binary + "-" + module.revision + "." + artifact.extension + }, + + /** + * Standalone assembly (shaded) jar. This is what we want to release. + * + * Build with `build/sbt standalone/assembly` command. + * e.g. connectors/standalone/target/scala-2.12/delta-standalone-original-shaded_2.12-0.2.1-SNAPSHOT.jar + */ + logLevel in assembly := Level.Info, + test in assembly := {}, + assemblyJarName in assembly := s"${name.value}-shaded_${scalaBinaryVersion.value}-${version.value}.jar", + // we exclude jars first, and then we shade what is remaining + assemblyExcludedJars in assembly := { + val cp = (fullClasspath in assembly).value + val allowedPrefixes = Set("META_INF", "io", "json4s", "jackson", "paranamer") + cp.filter { f => + !allowedPrefixes.exists(prefix => f.data.getName.startsWith(prefix)) + } + }, + assemblyShadeRules in assembly := Seq( + ShadeRule.rename("com.fasterxml.jackson.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("com.thoughtworks.paranamer.**" -> "shadedelta.@0").inAll, + ShadeRule.rename("org.json4s.**" -> "shadedelta.@0").inAll + ), + assemblyMergeStrategy in assembly := { + // Discard `module-info.class` to fix the `different file contents found` error. + // TODO Upgrade SBT to 1.5 which will do this automatically + case "module-info.class" => MergeStrategy.discard + case x => + val oldStrategy = (assemblyMergeStrategy in assembly).value + oldStrategy(x) + }, + artifact in assembly := { + val art = (artifact in assembly).value + art.withClassifier(Some("assembly")) + }, + addArtifact(artifact in assembly, assembly), + /** + * Unidoc settings + * Generate javadoc with `unidoc` command, outputs to `standalone/target/javaunidoc` + */ javacOptions in (JavaUnidoc, unidoc) := Seq( "-public", "-windowtitle", "Delta Standalone Reader " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index 9c9106cadf9..ae2281a820f 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -38,7 +38,7 @@ public interface DeltaLog { /** * @return the current {@link Snapshot} of the Delta table. You may need to call - * {@link #update()} to access the latest snapshot if the current snapshot is stale. + * {@link #update()} to access the latest snapshot if the current snapshot is stale. */ Snapshot snapshot(); diff --git a/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala b/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala new file mode 100644 index 00000000000..5c38ae234bf --- /dev/null +++ b/testStandaloneCosmetic/src/test/scala/ShadedJarSuite.scala @@ -0,0 +1,39 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.scalatest.FunSuite + +class ShadedJarSuite extends FunSuite { + test("properly shaded jars") { + // scalastyle:off classforname + Class.forName("io.delta.standalone.DeltaLog") + + Class.forName("shadedelta.com.fasterxml.jackson.databind.JavaType") + Class.forName("shadedelta.com.thoughtworks.paranamer.AdaptiveParanamer") + Class.forName("shadedelta.org.json4s.JsonDSL") + + intercept[ClassNotFoundException] { + Class.forName("com.fasterxml.jackson.databind.JavaType") + } + intercept[ClassNotFoundException] { + Class.forName("com.thoughtworks.paranamer.AdaptiveParanamer") + } + intercept[ClassNotFoundException] { + Class.forName("org.json4s.JsonDSL") + } + // scalastyle:on classforname + } +} From 1693e795a529a9648ede3288234f9ecb1b143208 Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Thu, 28 Oct 2021 11:51:16 -0700 Subject: [PATCH 115/291] [DSW] [25] Add Java checkstyle tests (#179) --- .github/workflows/test.yaml | 2 + build.sbt | 10 +- dev/README.md | 38 +++ dev/checkstyle-suppressions.xml | 31 +++ dev/checkstyle.xml | 218 ++++++++++++++++++ .../java/io/delta/hive/DeltaInputSplit.java | 7 +- .../io/delta/hive/IndexPredicateAnalyzer.java | 18 +- .../java/io/delta/hive/util/JavaUtils.java | 6 +- .../internal/util/StandaloneUtil.scala | 8 +- project/plugins.sbt | 6 + .../java/io/delta/standalone/DeltaLog.java | 21 +- .../java/io/delta/standalone/Operation.java | 4 +- .../standalone/OptimisticTransaction.java | 6 +- .../java/io/delta/standalone/VersionLog.java | 2 +- .../io/delta/standalone/actions/Action.java | 3 +- .../delta/standalone/actions/AddCDCFile.java | 4 +- .../io/delta/standalone/actions/AddFile.java | 4 +- .../delta/standalone/actions/CommitInfo.java | 3 +- .../io/delta/standalone/actions/JobInfo.java | 3 +- .../io/delta/standalone/actions/Metadata.java | 3 +- .../delta/standalone/actions/RemoveFile.java | 4 +- .../standalone/actions/SetTransaction.java | 5 +- .../expressions/BinaryComparison.java | 7 +- .../expressions/BinaryExpression.java | 6 +- .../standalone/expressions/Expression.java | 8 +- .../standalone/expressions/GreaterThan.java | 5 +- .../io/delta/standalone/expressions/In.java | 29 ++- .../standalone/expressions/LessThan.java | 3 +- .../expressions/UnaryExpression.java | 9 +- .../io/delta/standalone/expressions/Util.java | 3 +- .../standalone/internal/expressions/Util.java | 3 +- .../io/delta/standalone/storage/LogStore.java | 24 +- .../io/delta/standalone/types/ArrayType.java | 8 +- .../io/delta/standalone/types/DataType.java | 12 +- .../delta/standalone/types/FieldMetadata.java | 4 +- .../io/delta/standalone/types/MapType.java | 8 +- .../delta/standalone/types/StructField.java | 3 - .../io/delta/standalone/types/StructType.java | 4 +- .../util/ParquetSchemaConverter.java | 18 +- .../standalone/internal/actions/actions.scala | 2 +- 40 files changed, 463 insertions(+), 99 deletions(-) create mode 100644 dev/README.md create mode 100644 dev/checkstyle-suppressions.xml create mode 100644 dev/checkstyle.xml diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index f75e558f9fd..6d875c078aa 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -23,12 +23,14 @@ jobs: shell: bash -l {0} run: | build/sbt "++ 2.12.8 standalone/test" + build/sbt "++ 2.12.8 compatibility/test" build/sbt "++ 2.12.8 hive/test" build/sbt "++ 2.12.8 hiveMR/test" build/sbt "++ 2.12.8 hiveTez/test" build/sbt "++ 2.12.8 sqlDeltaImport/test" build/sbt "++ 2.12.8 testStandaloneCosmetic/test" build/sbt "++ 2.11.12 standalone/test" + build/sbt "++ 2.11.12 compatibility/test" build/sbt "++ 2.11.12 hive/test" build/sbt "++ 2.11.12 hiveMR/test" build/sbt "++ 2.11.12 hiveTez/test" diff --git a/build.sbt b/build.sbt index b0a44d3ca09..31eccce278c 100644 --- a/build.sbt +++ b/build.sbt @@ -54,7 +54,15 @@ lazy val commonSettings = Seq( compileScalastyle := scalastyle.in(Compile).toTask("").value, (compile in Compile) := ((compile in Compile) dependsOn compileScalastyle).value, testScalastyle := scalastyle.in(Test).toTask("").value, - (test in Test) := ((test in Test) dependsOn testScalastyle).value + (test in Test) := ((test in Test) dependsOn testScalastyle).value, + + // Can be run explicitly via: build/sbt $module/checkstyle + // Will automatically be run during compilation (e.g. build/sbt compile) + // and during tests (e.g. build/sbt test) + checkstyleConfigLocation := CheckstyleConfigLocation.File("dev/checkstyle.xml"), + checkstyleSeverityLevel := Some(CheckstyleSeverityLevel.Error), + (checkstyle in Compile) := (checkstyle in Compile).triggeredBy(compile in Compile).value, + (checkstyle in Test) := (checkstyle in Test).triggeredBy(compile in Test).value ) lazy val releaseSettings = Seq( diff --git a/dev/README.md b/dev/README.md new file mode 100644 index 00000000000..8a85deb2a6d --- /dev/null +++ b/dev/README.md @@ -0,0 +1,38 @@ +# Dev README +Below are some helpful IntelliJ configurations you can set to match our coding style and standards. + +## Checkstyle +This project uses checkstyle to format Java code. If developing locally, please setup checkstyle using the following steps. + +1. Add the CheckStyle-IDEA plugin to IntelliJ. +- `Settings > Plugins > Marketplace > CheckStyle-IDEA > INSTALL`. +- Restart your IDE if prompted. + +2. Configure IntelliJ to use the `checkstyle.xml` file provided in this directory. +- Go to `Settings > Tools > Checkstyle` (this tool location may differ based on your version of IntelliJ). +- Set the version to 8.29. +- Under the `Configuration File` heading, click the `+` symbol to add our specific configuration file. +- Give our file a useful description, such as `Delta Connectors Java Checks`, and provide the `connectors/dev/checkstyle.xml` path. +- Click `Next` to add the checkstyle file +- Check `Active` next to it once it has been added +- In the top right, set the Scan Scope to `Only Java sources (including tests)` + +3. Now, on the bottom tab bar, there should be a `CheckStyle` tab that lets you run Java style checks against using the `Check Project` button. + +4. You can also run checkstyle using SBT. For example, `build/sbt checkstyle` to run against all modules or `build/sbt standalone/checkstyle` to test only the `standalone` module. + +## Java Import Order +We use the following import order in our Java files. Please update this in `Settings > Editor > Code Style > Java > Imports > Import Layout`: + +``` +import java.* +import javax.* + +import scala.* + +import all other imports + +import io.delta.standalone.* +import io.delta.standalone.internal.* +``` + \ No newline at end of file diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml new file mode 100644 index 00000000000..642a5503a3d --- /dev/null +++ b/dev/checkstyle-suppressions.xml @@ -0,0 +1,31 @@ + + + + + + + + diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml new file mode 100644 index 00000000000..f942462aa3f --- /dev/null +++ b/dev/checkstyle.xml @@ -0,0 +1,218 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java index 38d3492df7b..faaa83ed9ee 100644 --- a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java +++ b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java @@ -19,6 +19,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; @@ -37,12 +38,14 @@ protected DeltaInputSplit() { partitionColumns = new PartitionColumnInfo[0]; } - public DeltaInputSplit(Path file, long start, long length, String[] hosts, PartitionColumnInfo[] partitionColumns) { + public DeltaInputSplit(Path file, long start, long length, String[] hosts, + PartitionColumnInfo[] partitionColumns) { super(file, start, length, hosts); this.partitionColumns = partitionColumns; } - public DeltaInputSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts, PartitionColumnInfo[] partitionColumns) { + public DeltaInputSplit(Path file, long start, long length, String[] hosts, + String[] inMemoryHosts, PartitionColumnInfo[] partitionColumns) { super(file, start, length, hosts, inMemoryHosts); this.partitionColumns = partitionColumns; } diff --git a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java index 387bdd43b9d..f9cca4bbe54 100644 --- a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java +++ b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java @@ -188,7 +188,8 @@ private ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { if ((udf instanceof GenericUDFBridge || udf instanceof GenericUDFToBinary || udf instanceof GenericUDFToChar || udf instanceof GenericUDFToVarchar || udf instanceof GenericUDFToDecimal || udf instanceof GenericUDFToDate - || udf instanceof GenericUDFToUnixTimeStamp || udf instanceof GenericUDFToUtcTimestamp) + || udf instanceof GenericUDFToUnixTimeStamp + || udf instanceof GenericUDFToUtcTimestamp) && funcDesc.getChildren().size() == 1 && funcDesc.getChildren().get(0) instanceof ExprNodeColumnDesc) { return expr.getChildren().get(0); @@ -241,7 +242,8 @@ private ExprNodeDesc analyzeExpr( if (exprNodeColDescs.size() != 1) { if (LOG.isInfoEnabled()) { - LOG.info("Pushed down expr should only have one column, while it is " + StringUtils.join(exprNodeColDescs.toArray())); + LOG.info("Pushed down expr should only have one column, while it is " + + StringUtils.join(exprNodeColDescs.toArray())); } return expr; } @@ -251,7 +253,8 @@ private ExprNodeDesc analyzeExpr( Set allowed = columnToUDFs.get(columnDesc.getColumn()); if (allowed == null) { if (LOG.isInfoEnabled()) { - LOG.info("This column " + columnDesc.getColumn() + " is not allowed to pushed down to delta..."); + LOG.info("This column " + columnDesc.getColumn() + + " is not allowed to pushed down to delta..."); } return expr; } @@ -262,7 +265,8 @@ private ExprNodeDesc analyzeExpr( } if (!allowed.contains(udfClassName)) { if (LOG.isInfoEnabled()) { - LOG.info("This udf " + genericUDF.getUdfName() + " is not allowed to pushed down to delta..."); + LOG.info("This udf " + genericUDF.getUdfName() + + " is not allowed to pushed down to delta..."); } return expr; } @@ -270,7 +274,8 @@ private ExprNodeDesc analyzeExpr( if (!udfClassName.equals("org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn") && exprConstantColDescs.size() > 1) { if (LOG.isInfoEnabled()) { - LOG.info("There should be one constant in this udf(" + udfClassName + ") except UDFIn"); + LOG.info("There should be one constant in this udf(" + udfClassName + + ") except UDFIn"); } return expr; } @@ -278,7 +283,8 @@ private ExprNodeDesc analyzeExpr( // We also need to update the expr so that the index query can be generated. // Note that, hive does not support UDFToDouble etc in the query text. ExprNodeGenericFuncDesc indexExpr = - new ExprNodeGenericFuncDesc(expr.getTypeInfo(), expr.getGenericUDF(), Arrays.asList(peelOffExprs)); + new ExprNodeGenericFuncDesc(expr.getTypeInfo(), expr.getGenericUDF(), + Arrays.asList(peelOffExprs)); searchConditions.add( new IndexSearchCondition( diff --git a/hive/src/test/java/io/delta/hive/util/JavaUtils.java b/hive/src/test/java/io/delta/hive/util/JavaUtils.java index ae02e04d679..5761f16b734 100644 --- a/hive/src/test/java/io/delta/hive/util/JavaUtils.java +++ b/hive/src/test/java/io/delta/hive/util/JavaUtils.java @@ -3,9 +3,7 @@ import java.io.*; import com.google.common.base.Preconditions; - import org.apache.commons.lang3.SystemUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,8 +45,8 @@ public static void deleteRecursively(File file, FilenameFilter filter) throws IO deleteRecursivelyUsingUnixNative(file); return; } catch (IOException e) { - logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + - "Falling back to Java IO way", file.getAbsolutePath(), e); + logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + + "Falling back to Java IO way", file.getAbsolutePath(), e); } } diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala index bb0a65ff488..689f745684c 100644 --- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala +++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala @@ -38,10 +38,10 @@ class StandaloneUtil(now: Long) { val partitionColumns: Seq[String] = schema.getFieldNames.filter(_.contains("part")).toSeq - val op = new Operation(Operation.Name.MANUAL_UPDATE, Map[String, Object]( - "mode" -> "Append", - "partitionBy" -> JsonUtils.toJson(partitionColumns), - "predicate" -> "predicate_str" + val op = new Operation(Operation.Name.MANUAL_UPDATE, Map[String, String]( + "mode" -> "\"Append\"", + "partitionBy" -> "\"[\\\"col1_part\\\",\\\"col2_part\\\"]\"", + "predicate" -> "\"predicate_str\"" ).asJava) val metadata: Metadata = Metadata.builder() diff --git a/project/plugins.sbt b/project/plugins.sbt index f69f9e9fd8b..1c1c106a73f 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -33,3 +33,9 @@ addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.3.0") + +addSbtPlugin("com.etsy" % "sbt-checkstyle-plugin" % "3.1.1") + +// By default, sbt-checkstyle-plugin uses checkstyle version 6.15, but we should set it to use the +// same version as Spark OSS (8.29) +dependencyOverrides += "com.puppycrawl.tools" % "checkstyle" % "8.29" diff --git a/standalone/src/main/java/io/delta/standalone/DeltaLog.java b/standalone/src/main/java/io/delta/standalone/DeltaLog.java index ae2281a820f..8fcaa029340 100644 --- a/standalone/src/main/java/io/delta/standalone/DeltaLog.java +++ b/standalone/src/main/java/io/delta/standalone/DeltaLog.java @@ -18,12 +18,12 @@ import java.util.Iterator; -import io.delta.standalone.actions.CommitInfo; -import io.delta.standalone.internal.DeltaLogImpl; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import io.delta.standalone.actions.CommitInfo; +import io.delta.standalone.internal.DeltaLogImpl; + /** * {@link DeltaLog} is the representation of the transaction logs of a Delta table. It provides APIs * to access the states of a Delta table. @@ -55,17 +55,20 @@ public interface DeltaLog { * * @param version the snapshot version to generate * @return the snapshot at the provided {@code version} - * @throws IllegalArgumentException if the {@code version} is outside the range of available versions + * @throws IllegalArgumentException if the {@code version} is outside the range of available + * versions */ Snapshot getSnapshotForVersionAsOf(long version); /** - * Travel back in time to the latest {@link Snapshot} that was generated at or before {@code timestamp}. + * Travel back in time to the latest {@link Snapshot} that was generated at or before + * {@code timestamp}. * * @param timestamp the number of milliseconds since midnight, January 1, 1970 UTC * @return the snapshot nearest to, but not after, the provided {@code timestamp} * @throws RuntimeException if the snapshot is unable to be recreated - * @throws IllegalArgumentException if the {@code timestamp} is before the earliest possible snapshot or after the latest possible snapshot + * @throws IllegalArgumentException if the {@code timestamp} is before the earliest possible + * snapshot or after the latest possible snapshot */ Snapshot getSnapshotForTimestampAsOf(long timestamp); @@ -103,7 +106,8 @@ public interface DeltaLog { Iterator getChanges(long startVersion, boolean failOnDataLoss); /** - * Create a {@link DeltaLog} instance representing the table located at the provided {@code path}. + * Create a {@link DeltaLog} instance representing the table located at the provided + * {@code path}. * * @param hadoopConf Hadoop {@code Configuration} to use when accessing the Delta table * @param path the path to the Delta table @@ -114,7 +118,8 @@ static DeltaLog forTable(Configuration hadoopConf, String path) { } /** - * Create a {@link DeltaLog} instance representing the table located at the provide {@code path}. + * Create a {@link DeltaLog} instance representing the table located at the provide + * {@code path}. * * @param hadoopConf Hadoop {@code Configuration} to use when accessing the Delta table * @param path the path to the Delta table diff --git a/standalone/src/main/java/io/delta/standalone/Operation.java b/standalone/src/main/java/io/delta/standalone/Operation.java index f2cba617c6f..8b681fb093b 100644 --- a/standalone/src/main/java/io/delta/standalone/Operation.java +++ b/standalone/src/main/java/io/delta/standalone/Operation.java @@ -2,11 +2,11 @@ package io.delta.standalone; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; import java.util.Optional; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; /** * An operation that can be performed on a Delta table. diff --git a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java index 89f4f45f3a7..fbd91b3e010 100644 --- a/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/OptimisticTransaction.java @@ -29,7 +29,7 @@ public interface OptimisticTransaction { * "{engineName}/{engineVersion}". * @return a {@link CommitResult}, wrapping the table version that was committed. */ - CommitResult commit(Iterable actions, Operation op, String engineInfo); + CommitResult commit(Iterable actions, Operation op, String engineInfo); /** * Mark files matched by the `readPredicates` as read by this transaction. @@ -78,8 +78,8 @@ public interface OptimisticTransaction { long txnVersion(String id); /** - * @return the metadata for this transaction. The metadata refers to the metadata of the snapshot - * at the transaction's read version unless updated during the transaction. + * @return the metadata for this transaction. The metadata refers to the metadata of the + * snapshot at the transaction's read version unless updated during the transaction. */ Metadata metadata(); } diff --git a/standalone/src/main/java/io/delta/standalone/VersionLog.java b/standalone/src/main/java/io/delta/standalone/VersionLog.java index 03e0c1ce994..12a3247ec93 100644 --- a/standalone/src/main/java/io/delta/standalone/VersionLog.java +++ b/standalone/src/main/java/io/delta/standalone/VersionLog.java @@ -2,9 +2,9 @@ package io.delta.standalone; -import javax.annotation.Nonnull; import java.util.Collections; import java.util.List; +import javax.annotation.Nonnull; import io.delta.standalone.actions.Action; diff --git a/standalone/src/main/java/io/delta/standalone/actions/Action.java b/standalone/src/main/java/io/delta/standalone/actions/Action.java index 30c3d5e5dd7..52c3a156e75 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Action.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Action.java @@ -8,7 +8,8 @@ * * You can use the following code to extract the concrete type of an {@link Action}. *
{@code
- *   List actions = ... // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
+ *   // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
+ *   List actions = ...
  *   actions.forEach(x -> {
  *       if (x instanceof AddFile) {
  *          AddFile addFile = (AddFile) x;
diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
index e2e21d32f9f..2e8ee0bf5ce 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
@@ -2,10 +2,10 @@
 
 package io.delta.standalone.actions;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 import java.util.Collections;
 import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 /**
  * A change file containing CDC data for the Delta version it's within. Non-CDC readers should
diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
index 0cf32802a59..184efa1829a 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
@@ -16,11 +16,11 @@
 
 package io.delta.standalone.actions;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 /**
  * Represents an action that adds a new file to the table. The path of a file acts as the primary
diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java
index 33aac13b091..15f8f9298dd 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java
@@ -136,7 +136,8 @@ public String getOperation() {
      * @return any relevant operation parameters. e.g. "mode", "partitionBy"
      */
     public Map getOperationParameters() {
-        return operationParameters != null ? Collections.unmodifiableMap(operationParameters) : null;
+        if (operationParameters != null) return Collections.unmodifiableMap(operationParameters);
+        return null;
     }
 
     /**
diff --git a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java
index daa83a65623..f87dd745aed 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java
@@ -26,7 +26,8 @@ public class JobInfo {
     private final String jobOwnerId;
     private final String triggerType;
 
-    public JobInfo(String jobId, String jobName, String runId, String jobOwnerId, String triggerType) {
+    public JobInfo(String jobId, String jobName, String runId, String jobOwnerId,
+                   String triggerType) {
         this.jobId = jobId;
         this.jobName = jobName;
         this.runId = runId;
diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java
index 76ba5bfba90..8d6e66a5ee1 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java
@@ -133,7 +133,8 @@ public boolean equals(Object o) {
 
     @Override
     public int hashCode() {
-        return Objects.hash(id, name, description, format, partitionColumns, configuration, createdTime, schema);
+        return Objects.hash(id, name, description, format, partitionColumns, configuration,
+                createdTime, schema);
     }
 
     /**
diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java
index efdc2c9ac8f..a3209f65277 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java
@@ -2,12 +2,12 @@
 
 package io.delta.standalone.actions;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 /**
  * Logical removal of a given file from the reservoir. Acts as a tombstone before a file is
diff --git a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java
index 69171a11f86..eb92bf907df 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java
@@ -2,8 +2,8 @@
 
 package io.delta.standalone.actions;
 
-import javax.annotation.Nonnull;
 import java.util.Optional;
+import javax.annotation.Nonnull;
 
 /**
  * Sets the committed version for a given application. Used to make operations like streaming append
@@ -18,7 +18,8 @@ public final class SetTransaction implements Action {
     @Nonnull
     private final Optional lastUpdated;
 
-    public SetTransaction(@Nonnull String appId, long version, @Nonnull Optional lastUpdated) {
+    public SetTransaction(@Nonnull String appId, long version,
+                          @Nonnull Optional lastUpdated) {
         this.appId = appId;
         this.version = version;
         this.lastUpdated = lastUpdated;
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java
index 2f029233b27..654b3c99ba8 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryComparison.java
@@ -1,11 +1,12 @@
 package io.delta.standalone.expressions;
 
-import io.delta.standalone.internal.expressions.Util;
-
 import java.util.Comparator;
 
+import io.delta.standalone.internal.expressions.Util;
+
 /**
- * A {@link BinaryOperator} that compares the left and right {@link Expression}s and returns a boolean value.
+ * A {@link BinaryOperator} that compares the left and right {@link Expression}s and returns a
+ * boolean value.
  */
 public abstract class BinaryComparison extends BinaryOperator implements Predicate {
     private final Comparator comparator;
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java
index bc258662eed..fa51515e4ad 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/BinaryExpression.java
@@ -1,11 +1,13 @@
-package io.delta.standalone.expressions;
+// todo: copyright
 
-import io.delta.standalone.data.RowRecord;
+package io.delta.standalone.expressions;
 
 import java.util.Arrays;
 import java.util.List;
 import java.util.Objects;
 
+import io.delta.standalone.data.RowRecord;
+
 /**
  * An expression with two inputs and one output. The output is by default evaluated to null
  * if any input is evaluated to null.
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java
index 1830a1a56b8..67c3a916021 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/Expression.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/Expression.java
@@ -1,12 +1,14 @@
-package io.delta.standalone.expressions;
+// todo: copyright
 
-import io.delta.standalone.data.RowRecord;
-import io.delta.standalone.types.DataType;
+package io.delta.standalone.expressions;
 
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import io.delta.standalone.data.RowRecord;
+import io.delta.standalone.types.DataType;
+
 /**
  * An expression in Delta Standalone.
  */
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java
index 8fbcdcb3358..2c9d9d089a7 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/GreaterThan.java
@@ -1,7 +1,10 @@
+// todo: copyright
+
 package io.delta.standalone.expressions;
 
 /**
- * Usage: {@code new GreaterThan(expr1, expr2)} - Returns true if `expr1` is greater than `expr2`, else false.
+ * Usage: {@code new GreaterThan(expr1, expr2)} - Returns true if `expr1` is greater than `expr2`,
+ * else false.
  */
 public final class GreaterThan extends BinaryComparison implements Predicate {
     public GreaterThan(Expression left, Expression right) {
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/In.java b/standalone/src/main/java/io/delta/standalone/expressions/In.java
index 6047e3d9c55..a8130bfda34 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/In.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/In.java
@@ -1,15 +1,18 @@
-package io.delta.standalone.expressions;
+// todo: copyright
 
-import io.delta.standalone.data.RowRecord;
-import io.delta.standalone.internal.expressions.Util;
+package io.delta.standalone.expressions;
 
 import java.util.Comparator;
 import java.util.List;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import io.delta.standalone.data.RowRecord;
+import io.delta.standalone.internal.expressions.Util;
+
 /**
- * Usage: {@code new In(expr, exprList)} - Returns true if `expr` is equal to any in `exprList`, else false.
+ * Usage: {@code new In(expr, exprList)} - Returns true if `expr` is equal to any in `exprList`,
+ * else false.
  */
 public final class In implements Predicate {
     private final Expression value;
@@ -27,10 +30,13 @@ public In(Expression value, List elems) {
             throw new IllegalArgumentException("'In' expression 'elems' cannot be empty");
         }
 
-        boolean allSameDataType = elems.stream().allMatch(x -> x.dataType().equals(value.dataType()));
+        boolean allSameDataType = elems
+            .stream()
+            .allMatch(x -> x.dataType().equals(value.dataType()));
 
         if (!allSameDataType) {
-            throw new IllegalArgumentException("In expression 'elems' and 'value' must all be of the same DataType");
+            throw new IllegalArgumentException(
+                "In expression 'elems' and 'value' must all be of the same DataType");
         }
 
         this.value = value;
@@ -58,19 +64,24 @@ public Boolean eval(RowRecord record) {
         Boolean falseOrNullresult = false;
         for (Expression setElem : elems) {
             Object setElemValue = setElem.eval(record);
-            if (setElemValue == null)
+            if (setElemValue == null) {
                 // null value found but element may still be in list
                 falseOrNullresult = null;
-            else if (comparator.compare(origValue, setElemValue) == 0)
+            } else if (comparator.compare(origValue, setElemValue) == 0) {
                 // short circuit and return true; we have found the element in the list
                 return true;
+            }
+
         }
         return falseOrNullresult;
     }
 
     @Override
     public String toString() {
-        String elemsStr = elems.stream().map(Expression::toString).collect(Collectors.joining(", "));
+        String elemsStr = elems
+            .stream()
+            .map(Expression::toString)
+            .collect(Collectors.joining(", "));
         return value + " IN (" + elemsStr + ")";
     }
 
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java
index fba78183d0c..8eb9cf4986a 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/LessThan.java
@@ -1,7 +1,8 @@
 package io.delta.standalone.expressions;
 
 /**
- * Usage: {@code new LessThan(expr1, expr2)} - Returns true if `expr1` is less than `expr2`, else false.
+ * Usage: {@code new LessThan(expr1, expr2)} - Returns true if `expr1` is less than `expr2`, else
+ * false.
  */
 public final class LessThan extends BinaryComparison implements Predicate {
     public LessThan(Expression left, Expression right) {
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java
index 6db6fa4dc76..cb94d165d5b 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/UnaryExpression.java
@@ -1,11 +1,13 @@
-package io.delta.standalone.expressions;
+// todo: copyright
 
-import io.delta.standalone.data.RowRecord;
+package io.delta.standalone.expressions;
 
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
+import io.delta.standalone.data.RowRecord;
+
 /**
  * An expression with one input and one output. The output is by default evaluated to null
  * if the input is evaluated to null.
@@ -31,7 +33,8 @@ public Object eval(RowRecord record) {
     }
 
     protected Object nullSafeEval(Object childResult) {
-        throw new IllegalArgumentException("UnaryExpressions must override either eval or nullSafeEval");
+        throw new IllegalArgumentException(
+            "UnaryExpressions must override either eval or nullSafeEval");
     }
 
     @Override
diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Util.java b/standalone/src/main/java/io/delta/standalone/expressions/Util.java
index d30f2fdaabe..3d925ca5f3e 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/Util.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/Util.java
@@ -20,6 +20,7 @@ public static CastingComparator createCastingComparator(DataType dataType) {
             return new CastingComparator();
         }
 
-        throw new RuntimeException("Couldn't find matching comparator for DataType: " + dataType.toString());
+        throw new RuntimeException(
+            "Couldn't find matching comparator for DataType: " + dataType.toString());
     }
 }
diff --git a/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java b/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java
index 5c35090e123..1e493f8097b 100644
--- a/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java
+++ b/standalone/src/main/java/io/delta/standalone/internal/expressions/Util.java
@@ -69,6 +69,7 @@ public static Comparator createComparator(DataType dataType) {
         }
 
         // unsupported comparison types: ArrayType, StructType, MapType
-        throw new IllegalArgumentException("Couldn't find matching comparator for DataType: " + dataType.getTypeName());
+        throw new IllegalArgumentException(
+            "Couldn't find matching comparator for DataType: " + dataType.getTypeName());
     }
 }
diff --git a/standalone/src/main/java/io/delta/standalone/storage/LogStore.java b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java
index 851968b69d7..08f540d8641 100644
--- a/standalone/src/main/java/io/delta/standalone/storage/LogStore.java
+++ b/standalone/src/main/java/io/delta/standalone/storage/LogStore.java
@@ -16,14 +16,15 @@
 
 package io.delta.standalone.storage;
 
-import io.delta.standalone.data.CloseableIterator;
+import java.io.FileNotFoundException;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.Iterator;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-import java.io.FileNotFoundException;
-import java.nio.file.FileAlreadyExistsException;
-import java.util.Iterator;
+import io.delta.standalone.data.CloseableIterator;
 
 /**
  * :: DeveloperApi ::
@@ -48,7 +49,7 @@
  * LogStore and its implementations are not meant for direct access but for configuration based
  * on storage system. See [[https://docs.delta.io/latest/delta-storage.html]] for details.
  *
- * @since 0.3.0 // TODO: double check this will be the new DSW version
+ * @since 0.3.0
  */
 public abstract class LogStore {
 
@@ -72,10 +73,11 @@ public LogStore(Configuration initHadoopConf) {
     /**
      * :: DeveloperApi ::
      *
-     * Load the given file and return an `Iterator` of lines, with line breaks removed from each line.
-     * Callers of this function are responsible to close the iterator if they are done with it.
+     * Load the given file and return an `Iterator` of lines, with line breaks removed from each
+     * line. Callers of this function are responsible to close the iterator if they are done with
+     * it.
      *
-     * @since 0.3.0 // TODO: double check this will be the new DSW version
+     * @since 0.3.0
      *
      * @param path  the path to load
      * @param hadoopConf  the latest hadoopConf
@@ -92,7 +94,7 @@ public LogStore(Configuration initHadoopConf) {
      * implementation must ensure that the entire file is made visible atomically, that is,
      * it should not generate partial files.
      *
-     * @since 0.3.0 // TODO: double check this will be the new DSW version
+     * @since 0.3.0
      *
      * @param path  the path to write to
      * @param actions  actions to be written
@@ -111,7 +113,7 @@ public abstract void write(
      * List the paths in the same directory that are lexicographically greater or equal to
      * (UTF-8 sorting) the given `path`. The result should also be sorted by the file name.
      *
-     * @since 0.3.0 // TODO: double check this will be the new DSW version
+     * @since 0.3.0
      *
      * @param path  the path to load
      * @param hadoopConf  the latest hadoopConf
@@ -140,7 +142,7 @@ public abstract Iterator listFrom(
      *
      * Whether a partial write is visible for the underlying file system of `path`.
      *
-     * @since 0.3.0 // TODO: double check this will be the new DSW version
+     * @since 0.3.0
      *
      * @param path  the path in question
      * @param hadoopConf  the latest hadoopConf
diff --git a/standalone/src/main/java/io/delta/standalone/types/ArrayType.java b/standalone/src/main/java/io/delta/standalone/types/ArrayType.java
index 1be635656d7..4d0c4421133 100644
--- a/standalone/src/main/java/io/delta/standalone/types/ArrayType.java
+++ b/standalone/src/main/java/io/delta/standalone/types/ArrayType.java
@@ -75,7 +75,13 @@ public boolean containsNull() {
      */
     protected void buildFormattedString(String prefix, StringBuilder builder) {
         final String nextPrefix = prefix + "    |";
-        builder.append(String.format("%s-- element: %s (containsNull = %b)\n", prefix, elementType.getTypeName(), containsNull));
+        builder.append(
+            String.format(
+                "%s-- element: %s (containsNull = %b)\n",
+                prefix,
+                elementType.getTypeName(),
+                containsNull)
+        );
         DataType.buildFormattedString(elementType, nextPrefix, builder);
     }
 
diff --git a/standalone/src/main/java/io/delta/standalone/types/DataType.java b/standalone/src/main/java/io/delta/standalone/types/DataType.java
index 4649e55da8f..a5739a968d9 100644
--- a/standalone/src/main/java/io/delta/standalone/types/DataType.java
+++ b/standalone/src/main/java/io/delta/standalone/types/DataType.java
@@ -93,9 +93,15 @@ protected static void buildFormattedString(
             DataType dataType,
             String prefix,
             StringBuilder builder) {
-        if (dataType instanceof ArrayType) ((ArrayType) dataType).buildFormattedString(prefix, builder);
-        if (dataType instanceof StructType) ((StructType) dataType).buildFormattedString(prefix, builder);
-        if (dataType instanceof MapType) ((MapType) dataType).buildFormattedString(prefix, builder);
+        if (dataType instanceof ArrayType) {
+            ((ArrayType) dataType).buildFormattedString(prefix, builder);
+        }
+        if (dataType instanceof StructType) {
+            ((StructType) dataType).buildFormattedString(prefix, builder);
+        }
+        if (dataType instanceof MapType) {
+            ((MapType) dataType).buildFormattedString(prefix, builder);
+        }
     }
 
     @Override
diff --git a/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java b/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java
index 7ea46fe02f8..d357bfdc795 100644
--- a/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java
+++ b/standalone/src/main/java/io/delta/standalone/types/FieldMetadata.java
@@ -40,8 +40,8 @@
 
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Map;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 /**
@@ -113,7 +113,7 @@ public int hashCode() {
                                 Arrays.hashCode((Object[]) entry.getValue()) :
                         entry.hashCode())
                 ).sum();
-        }
+    }
 
     /**
      * @return a new {@code FieldMetadata.Builder}
diff --git a/standalone/src/main/java/io/delta/standalone/types/MapType.java b/standalone/src/main/java/io/delta/standalone/types/MapType.java
index a24d58a370e..54f4e2069ba 100644
--- a/standalone/src/main/java/io/delta/standalone/types/MapType.java
+++ b/standalone/src/main/java/io/delta/standalone/types/MapType.java
@@ -87,7 +87,13 @@ protected void buildFormattedString(String prefix, StringBuilder builder) {
         final String nextPrefix = prefix + "    |";
         builder.append(String.format("%s-- key: %s\n", prefix, keyType.getTypeName()));
         DataType.buildFormattedString(keyType, nextPrefix, builder);
-        builder.append(String.format("%s-- value: %s (valueContainsNull = %b)\n", prefix, valueType.getTypeName(), valueContainsNull));
+        builder.append(
+            String.format(
+                "%s-- value: %s (valueContainsNull = %b)\n",
+                prefix,
+                valueType.getTypeName(),
+                valueContainsNull)
+        );
     }
 
     @Override
diff --git a/standalone/src/main/java/io/delta/standalone/types/StructField.java b/standalone/src/main/java/io/delta/standalone/types/StructField.java
index 8bbee138f7c..932cc44c6fd 100644
--- a/standalone/src/main/java/io/delta/standalone/types/StructField.java
+++ b/standalone/src/main/java/io/delta/standalone/types/StructField.java
@@ -38,10 +38,7 @@
 
 package io.delta.standalone.types;
 
-import java.util.Collections;
-import java.util.Map;
 import java.util.Objects;
-import java.util.stream.Collectors;
 
 /**
  * A field inside a {@link StructType}.
diff --git a/standalone/src/main/java/io/delta/standalone/types/StructType.java b/standalone/src/main/java/io/delta/standalone/types/StructType.java
index 7c41292c59d..6688a829e23 100644
--- a/standalone/src/main/java/io/delta/standalone/types/StructType.java
+++ b/standalone/src/main/java/io/delta/standalone/types/StructType.java
@@ -38,11 +38,11 @@
 
 package io.delta.standalone.types;
 
-import io.delta.standalone.expressions.Column;
-
 import java.util.Arrays;
 import java.util.HashMap;
 
+import io.delta.standalone.expressions.Column;
+
 /**
  * The data type representing a table's schema, consisting of a collection of
  * fields (that is, {@code fieldName} to {@code dataType} pairs).
diff --git a/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java
index 39346ea72c7..d688a7006ba 100644
--- a/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java
+++ b/standalone/src/main/java/io/delta/standalone/util/ParquetSchemaConverter.java
@@ -16,11 +16,11 @@
 
 package io.delta.standalone.util;
 
-import io.delta.standalone.internal.util.SparkToParquetSchemaConverter;
-import io.delta.standalone.types.StructType;
-
 import org.apache.parquet.schema.MessageType;
 
+import io.delta.standalone.types.StructType;
+import io.delta.standalone.internal.util.SparkToParquetSchemaConverter;
+
 /**
  * Converter class to convert {@link StructType} to Parquet {@link MessageType}.
  */
@@ -29,10 +29,10 @@ public class ParquetSchemaConverter {
     /**
      * Represents Parquet timestamp types.
      * - INT96 is a non-standard but commonly used timestamp type in Parquet.
-     * - TIMESTAMP_MICROS is a standard timestamp type in Parquet, which stores number of microseconds
-     *   from the Unix epoch.
-     * - TIMESTAMP_MILLIS is also standard, but with millisecond precision, which means the microsecond
-     *   portion of the timestamp value is truncated.
+     * - TIMESTAMP_MICROS is a standard timestamp type in Parquet, which stores number of
+     *   microseconds from the Unix epoch.
+     * - TIMESTAMP_MILLIS is also standard, but with millisecond precision, which means the
+     *   microsecond portion of the timestamp value is truncated.
      */
     public enum ParquetOutputTimestampType {
         INT96,
@@ -82,7 +82,9 @@ public static MessageType sparkToParquet(StructType schema, Boolean writeLegacyP
      * @return {@code schema} as a Parquet {@link MessageType}
      * @throws IllegalArgumentException if a {@code StructField} name contains invalid character(s)
      */
-    public static MessageType sparkToParquet(StructType schema, ParquetOutputTimestampType outputTimestampType) {
+    public static MessageType sparkToParquet(
+            StructType schema,
+            ParquetOutputTimestampType outputTimestampType) {
         return new SparkToParquetSchemaConverter(
                 writeLegacyParquetFormatDefault,
                 outputTimestampType).convert(schema);
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
index cbd65a322b5..5757fe48e4b 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
@@ -388,7 +388,7 @@ private[internal] class JsonMapSerializer extends JsonSerializer[Map[String, Str
  *
  * With the inclusion of RemoveFile as an exposed Java API, and since it was upgraded to match the
  * latest Delta OSS release, we now had a case class inside of [[SingleAction]] that had "primitive"
- * default paramaters. They are primitive in the sense that Parquet4s would try to decode them using
+ * default parameters. They are primitive in the sense that Parquet4s would try to decode them using
  * the [[PrimitiveValueCodecs]] trait. But since these parameters have default values, there is no
  * guarantee that they will exist in the underlying parquet checkpoint files. Thus (without these
  * classes), parquet4s would throw errors like this:

From ab2b4e23f6965cee6ffd9da2bf6b13aab4a18a22 Mon Sep 17 00:00:00 2001
From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com>
Date: Thu, 28 Oct 2021 11:52:22 -0700
Subject: [PATCH 116/291] Minor fix in Literal Expression (#188)

---
 .../delta/standalone/expressions/Literal.java |  3 +-
 .../internal/exception/DeltaErrors.scala      | 14 ++++--
 .../standalone/internal/ExpressionSuite.scala | 50 +++++++++++--------
 3 files changed, 40 insertions(+), 27 deletions(-)

diff --git a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java
index f4409cb6a78..9595b61cdae 100644
--- a/standalone/src/main/java/io/delta/standalone/expressions/Literal.java
+++ b/standalone/src/main/java/io/delta/standalone/expressions/Literal.java
@@ -70,8 +70,7 @@ public static Literal of(Date value) {
     }
 
     public static Literal of(BigDecimal value) {
-        // TODO: get the precision and scale from the value
-        return new Literal(value, DecimalType.USER_DEFAULT);
+        return new Literal(value, new DecimalType(value.precision(), value.scale()));
     }
 
     public static Literal of(double value) {
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
index 0d96407af51..6b4b27a2f93 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
@@ -20,7 +20,6 @@ import java.io.{FileNotFoundException, IOException}
 
 import scala.annotation.varargs
 
-
 import io.delta.standalone.types.{DataType, StructType}
 import io.delta.standalone.exceptions._
 import io.delta.standalone.internal.actions.{CommitInfo, Protocol}
@@ -117,13 +116,18 @@ private[internal] object DeltaErrors {
       s"that this field can't be null. Schema: ${schema.getTreeString}")
   }
 
+  /**
+   * Thrown when a user tries to get a value of type `desiredType` from a
+   * [[io.delta.standalone.expressions.Column]] with name `fieldName` and dataType `actualType`,
+   * but `actualType` and `desiredType` are not the same.
+   */
   def fieldTypeMismatch(
       fieldName: String,
-      expectedType: DataType,
-      actualType: String): Throwable = {
+      actualType: DataType,
+      desiredType: String): Throwable = {
     new ClassCastException(
-      s"The data type of field $fieldName is ${expectedType.getTypeName}. " +
-        s"Cannot cast it to $actualType")
+      s"The data type of field $fieldName is ${actualType.getTypeName}. " +
+        s"Cannot cast it to $desiredType")
   }
 
   def failOnDataLossException(expectedVersion: Long, seenVersion: Long): Throwable = {
diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala
index 06e70314f8e..93a54628d94 100644
--- a/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala
+++ b/standalone/src/test/scala/io/delta/standalone/internal/ExpressionSuite.scala
@@ -119,8 +119,8 @@ class ExpressionSuite extends FunSuite {
       (Literal.of(1.0), Literal.of(2.0), Literal.of(1.0), Literal.ofNull(new DoubleType())),
       (Literal.of(1.toByte), Literal.of(2.toByte), Literal.of(1.toByte),
         Literal.ofNull(new ByteType())),
-      (Literal.of(new BigDecimalJ("0.1")), Literal.of(new BigDecimalJ("0.2")),
-        Literal.of(new BigDecimalJ("0.1")), Literal.ofNull(DecimalType.USER_DEFAULT)),
+      (Literal.of(new BigDecimalJ("123.45")), Literal.of(new BigDecimalJ("887.62")),
+        Literal.of(new BigDecimalJ("123.45")), Literal.ofNull(new DecimalType(5, 2))),
       (Literal.False, Literal.True, Literal.False, Literal.ofNull(new BooleanType())),
       (Literal.of(new TimestampJ(0)), Literal.of(new TimestampJ(1000000)),
       Literal.of(new TimestampJ(0)), Literal.ofNull(new TimestampType())),
@@ -289,13 +289,15 @@ class ExpressionSuite extends FunSuite {
     )
   }
 
-  private def testColumn(fieldName: String, dataType: DataType, record: RowRecord,
-      expectedResult: Any) = {
-    assert(Objects.equals(new Column(fieldName, dataType).eval(record),
-      expectedResult))
-  }
-
   test("Column tests") {
+    def testColumn(
+        fieldName: String,
+        dataType: DataType,
+        record: RowRecord,
+        expectedResult: Any): Unit = {
+      assert(Objects.equals(new Column(fieldName, dataType).eval(record), expectedResult))
+    }
+
     val schema = new StructType(Array(
       new StructField("testInt", new IntegerType(), true),
       new StructField("testLong", new LongType(), true),
@@ -309,11 +311,12 @@ class ExpressionSuite extends FunSuite {
       new StructField("testDecimal", DecimalType.USER_DEFAULT, true),
       new StructField("testTimestamp", new TimestampType(), true),
       new StructField("testDate", new DateType(), true)))
+
     val partRowRecord = new PartitionRowRecord(schema,
       Map("testInt"->"1",
         "testLong"->"10",
         "testByte" ->"8",
-      "testShort" -> "100",
+        "testShort" -> "100",
         "testBoolean" -> "true",
         "testFloat" -> "20.0",
         "testDouble" -> "22.0",
@@ -333,7 +336,7 @@ class ExpressionSuite extends FunSuite {
     testColumn("testString", new StringType(), partRowRecord, "onetwothree")
     assert(Array(1.toByte, 5.toByte, 8.toByte) sameElements
       (new Column("testBinary", new BinaryType())).eval(partRowRecord).asInstanceOf[Array[Byte]])
-    testColumn("testDecimal", DecimalType.USER_DEFAULT, partRowRecord, new BigDecimalJ("0.123"))
+    testColumn("testDecimal", new DecimalType(4, 3), partRowRecord, new BigDecimalJ("0.123"))
     testColumn("testTimestamp", new TimestampType(), partRowRecord, new TimestampJ(12345678))
     testColumn("testDate", new DateType(), partRowRecord, new DateJ(70, 0, 1))
 
@@ -348,16 +351,17 @@ class ExpressionSuite extends FunSuite {
       "The data type of column testStruct is struct. This is not supported yet")
   }
 
-  private def buildPartitionRowRecord(
-      dataType: DataType,
-      nullable: Boolean,
-      value: String,
-      name: String = "test") = {
-    new PartitionRowRecord(new StructType(Array(new StructField(name, dataType, nullable))),
-      Map(name -> value))
-  }
-
   test("PartitionRowRecord tests") {
+    def buildPartitionRowRecord(
+        dataType: DataType,
+        nullable: Boolean,
+        value: String,
+        name: String = "test"): PartitionRowRecord = {
+      new PartitionRowRecord(
+        new StructType(Array(new StructField(name, dataType, nullable))),
+        Map(name -> value))
+    }
+
     val testPartitionRowRecord = buildPartitionRowRecord(new IntegerType(), nullable = true, "5")
     assert(buildPartitionRowRecord(new IntegerType(), nullable = true, null).isNullAt("test"))
     assert(!buildPartitionRowRecord(new IntegerType(), nullable = true, "5").isNullAt("test"))
@@ -408,7 +412,7 @@ class ExpressionSuite extends FunSuite {
         DateJ.valueOf("1970-01-01"))
     )
     nonPrimTypes.foreach {
-      case (dataType: DataType, f: (PartitionRowRecord => Any), s: String, v) =>
+      case (dataType: DataType, f: (PartitionRowRecord => Any), s: String, v: Any) =>
         assert(Objects.equals(f(buildPartitionRowRecord(dataType, nullable = true, s)), v))
         assert(f(buildPartitionRowRecord(dataType, nullable = true, null)) == null)
         testException[NullPointerException](
@@ -552,4 +556,10 @@ class ExpressionSuite extends FunSuite {
     assert(col1 == col1Copy)
     assert(col1 != col2)
   }
+
+  test("decimal literal creation") {
+    val dec52 = new BigDecimalJ("123.45")
+    val lit52 = Literal.of(dec52)
+    assert(lit52.dataType().equals(new DecimalType(5, 2)))
+  }
 }

From 72c3c00effb18f283cc81997972de48388eb9941 Mon Sep 17 00:00:00 2001
From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com>
Date: Thu, 28 Oct 2021 11:54:41 -0700
Subject: [PATCH 117/291] [DSW] [30] scalastyle fixes (#189)

---
 build.sbt                                           | 11 +++++++++--
 hive/src/main/scala/io/delta/hive/DeltaHelper.scala |  9 +++++----
 .../main/scala/io/delta/hive/DeltaInputFormat.scala |  7 +++----
 scalastyle-config.xml                               | 11 +++++++++++
 .../src/main/scala/DataTransforms.scala             |  2 +-
 sql-delta-import/src/main/scala/JDBCImport.scala    |  4 +---
 .../io/delta/standalone/internal/Checkpoints.scala  | 13 +++++++------
 .../delta/standalone/internal/ConflictChecker.scala |  2 +-
 .../io/delta/standalone/internal/DeltaConfig.scala  |  6 +++---
 .../standalone/internal/DeltaHistoryManager.scala   |  7 ++++---
 .../io/delta/standalone/internal/DeltaLogImpl.scala |  9 +++++----
 .../delta/standalone/internal/MetadataCleanup.scala |  4 ++--
 .../internal/OptimisticTransactionImpl.scala        |  1 +
 .../io/delta/standalone/internal/SnapshotImpl.scala |  9 +++++----
 .../standalone/internal/SnapshotManagement.scala    |  4 ++--
 .../delta/standalone/internal/actions/actions.scala |  2 ++
 .../internal/data/PartitionRowRecord.scala          |  3 ++-
 .../internal/data/RowParquetRecordImpl.scala        |  3 ++-
 .../standalone/internal/exception/DeltaErrors.scala |  7 ++++---
 .../standalone/internal/scan/DeltaScanImpl.scala    |  1 +
 .../internal/scan/FilteredDeltaScanImpl.scala       |  1 +
 .../standalone/internal/storage/HDFSLogStore.scala  |  6 +++---
 .../internal/storage/HadoopFileSystemLogStore.scala |  9 +++++----
 .../internal/storage/LineCloseableIterator.scala    |  3 ++-
 .../internal/storage/LogStoreProvider.scala         |  5 +++--
 .../standalone/internal/util/ConversionUtils.scala  |  3 ++-
 .../standalone/internal/util/DataTypeParser.scala   | 10 ++++++----
 .../standalone/internal/util/PartitionUtils.scala   |  4 +++-
 .../standalone/internal/util/SchemaUtils.scala      |  1 +
 29 files changed, 97 insertions(+), 60 deletions(-)

diff --git a/build.sbt b/build.sbt
index 31eccce278c..1a3fb840a6a 100644
--- a/build.sbt
+++ b/build.sbt
@@ -39,8 +39,15 @@ lazy val commonSettings = Seq(
   organization := "io.delta",
   scalaVersion := "2.12.8",
   fork := true,
-  javacOptions ++= Seq("-source", "1.8", "-target", "1.8"),
-  scalacOptions += "-target:jvm-1.8",
+  javacOptions ++= Seq("-source", "1.8", "-target", "1.8", "-Xlint:unchecked"),
+  scalacOptions ++= Seq(
+    "-target:jvm-1.8",
+    "-Ywarn-unused-import",
+    // "-Xfatal-warnings",
+    // "-feature",
+    // "-deprecation",
+    "-language:implicitConversions"
+  ),
   // Configurations to speed up tests and reduce memory footprint
   javaOptions in Test ++= Seq(
     "-Dspark.ui.enabled=false",
diff --git a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
index bda07d942d4..95d12f3cdf6 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
@@ -25,20 +25,21 @@ import scala.collection.mutable
 import scala.util.control.NonFatal
 
 import com.google.common.cache.{Cache, CacheBuilder}
-import io.delta.standalone.actions.AddFile
-import io.delta.standalone.types._
-import io.delta.standalone.{DeltaLog, Snapshot}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{BlockLocation, FileStatus, FileSystem, LocatedFileStatus, Path}
 import org.apache.hadoop.hive.metastore.api.MetaException
 import org.apache.hadoop.hive.ql.exec.{ExprNodeEvaluatorFactory, SerializationUtilities}
 import org.apache.hadoop.hive.ql.plan.{ExprNodeGenericFuncDesc, TableScanDesc}
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
 import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, ObjectInspectorFactory, PrimitiveObjectInspector}
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
 import org.apache.hadoop.hive.serde2.typeinfo._
 import org.apache.hadoop.mapred.JobConf
 import org.slf4j.LoggerFactory
 
+import io.delta.standalone.{DeltaLog, Snapshot}
+import io.delta.standalone.actions.AddFile
+import io.delta.standalone.types._
+
 object DeltaHelper {
 
   private val LOG = LoggerFactory.getLogger(getClass.getName)
diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
index 72469ba559e..a36476e0fd8 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
@@ -23,9 +23,8 @@ import org.apache.hadoop.fs.FileStatus
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.metastore.api.MetaException
-import org.apache.hadoop.hive.ql.exec.Utilities
 import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport
-import org.apache.hadoop.io.{ArrayWritable, NullWritable, Writable}
+import org.apache.hadoop.io.{ArrayWritable, NullWritable}
 import org.apache.hadoop.mapred._
 import org.apache.hadoop.mapreduce.security.TokenCache
 import org.apache.parquet.hadoop.ParquetInputFormat
@@ -50,8 +49,8 @@ import org.slf4j.LoggerFactory
  *
  * For each reader created from a [[DeltaInputSplit]], we can get all partition column types, the
  * locations of a partition column in the schema, and their string values. The reader can build
- * [[Writable]] for all partition values, and insert them to the raw row returned by
- * [[org.apache.parquet.hadoop.ParquetRecordReader]].
+ * [[org.apache.hadoop.io.Writable]] for all partition values, and insert them to the raw row
+ * returned by [[org.apache.parquet.hadoop.ParquetRecordReader]].
  */
 class DeltaInputFormat(realInput: ParquetInputFormat[ArrayWritable])
   extends FileInputFormat[NullWritable, ArrayWritable] {
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index ec6744923eb..fd78efa4610 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -306,6 +306,17 @@ This file is divided into 3 sections:
     is slower.  
   
 
+  
+    
+      java,scala,3rdParty,standalone,standaloneInternal
+      javax?\..*
+      scala\..*
+      (?!io\.delta\.standalone\.).*
+      io\.delta\.standalone\.(?!internal).*
+      io\.delta\.standalone\.internal\..*
+    
+  
+
   
     
       COMMA
diff --git a/sql-delta-import/src/main/scala/DataTransforms.scala b/sql-delta-import/src/main/scala/DataTransforms.scala
index 404154f0eac..6fc4d5f4ca4 100644
--- a/sql-delta-import/src/main/scala/DataTransforms.scala
+++ b/sql-delta-import/src/main/scala/DataTransforms.scala
@@ -16,7 +16,7 @@
 
 package io.delta.connectors.spark.jdbc
 
-import org.apache.spark.sql.{DataFrame, SparkSession}
+import org.apache.spark.sql.DataFrame
 
 /**
  * Class that applies transformation functions one by one on input DataFrame
diff --git a/sql-delta-import/src/main/scala/JDBCImport.scala b/sql-delta-import/src/main/scala/JDBCImport.scala
index 41a9f9c9ccd..70715708bcf 100644
--- a/sql-delta-import/src/main/scala/JDBCImport.scala
+++ b/sql-delta-import/src/main/scala/JDBCImport.scala
@@ -19,10 +19,8 @@ package io.delta.connectors.spark.jdbc
 import java.util.Properties
 
 import io.delta.tables.DeltaTable
-import org.apache.spark.sql.functions.col
 import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
-
-import scala.collection.JavaConverters._
+import org.apache.spark.sql.functions.col
 
 /**
  * Class that contains JDBC source, read parallelism params and target table name
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala
index 12996cc1433..8d09a518f8b 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/Checkpoints.scala
@@ -22,16 +22,17 @@ import java.util.UUID
 import scala.collection.JavaConverters._
 import scala.util.control.NonFatal
 
+import com.github.mjakubowski84.parquet4s.ParquetWriter
+import org.apache.hadoop.fs.Path
+import org.apache.parquet.hadoop.metadata.CompressionCodecName
+
 import io.delta.standalone.data.CloseableIterator
+
 import io.delta.standalone.internal.actions.SingleAction
-import io.delta.standalone.internal.util.JsonUtils
-import io.delta.standalone.internal.util.FileNames._
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.logging.Logging
-
-import org.apache.hadoop.fs.Path
-import org.apache.parquet.hadoop.metadata.CompressionCodecName
-import com.github.mjakubowski84.parquet4s.ParquetWriter
+import io.delta.standalone.internal.util.FileNames._
+import io.delta.standalone.internal.util.JsonUtils
 
 /**
  * Records information about a checkpoint.
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala
index d5fa1c67690..6e13523274d 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/ConflictChecker.scala
@@ -18,10 +18,10 @@ package io.delta.standalone.internal
 
 import java.util.concurrent.TimeUnit
 
-import scala.collection.JavaConverters._
 import scala.collection.mutable
 
 import io.delta.standalone.expressions.Expression
+
 import io.delta.standalone.internal.actions._
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.logging.Logging
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala
index 06477822d65..81c766bf0ce 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaConfig.scala
@@ -18,13 +18,13 @@ package io.delta.standalone.internal
 
 import java.util.{HashMap, Locale}
 
-import io.delta.standalone.internal.actions.{Action, Metadata, Protocol}
+import org.apache.hadoop.conf.Configuration
+
+import io.delta.standalone.internal.actions.{Metadata, Protocol}
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.logging.Logging
 import io.delta.standalone.internal.util.{CalendarInterval, IntervalUtils}
 
-import org.apache.hadoop.conf.Configuration
-
 private[internal] case class DeltaConfig[T](
     key: String,
     defaultValue: String,
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala
index 1ac2e3e6b80..f533c46a02b 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaHistoryManager.scala
@@ -20,13 +20,14 @@ import java.sql.Timestamp
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.fs.Path
+
 import io.delta.standalone.storage.LogStore
+
 import io.delta.standalone.internal.actions.{Action, CommitInfo, CommitMarker}
 import io.delta.standalone.internal.exception.DeltaErrors
-import io.delta.standalone.internal.util.FileNames
 import io.delta.standalone.internal.logging.Logging
-
-import org.apache.hadoop.fs.Path
+import io.delta.standalone.internal.util.FileNames
 
 /**
  * This class keeps tracks of the version of commits and their timestamps for a Delta table to
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala
index dac49db5bba..d98f623bffd 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/DeltaLogImpl.scala
@@ -17,13 +17,17 @@
 package io.delta.standalone.internal
 
 import java.io.IOException
-import java.util.concurrent.locks.ReentrantLock
 import java.util.TimeZone
+import java.util.concurrent.locks.ReentrantLock
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
 import io.delta.standalone.{DeltaLog, OptimisticTransaction, VersionLog}
 import io.delta.standalone.actions.{CommitInfo => CommitInfoJ}
+
 import io.delta.standalone.internal.actions.{Action, Metadata, Protocol}
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.logging.Logging
@@ -31,9 +35,6 @@ import io.delta.standalone.internal.sources.StandaloneHadoopConf
 import io.delta.standalone.internal.storage.LogStoreProvider
 import io.delta.standalone.internal.util.{Clock, ConversionUtils, FileNames, SystemClock}
 
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
 /**
  * Scala implementation of Java interface [[DeltaLog]].
  */
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala
index bcd708b46b6..1339e7c1204 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/MetadataCleanup.scala
@@ -20,11 +20,11 @@ import java.util.{Calendar, TimeZone}
 
 import scala.collection.JavaConverters._
 
-import io.delta.standalone.internal.util.FileNames.{checkpointPrefix, isCheckpointFile, isDeltaFile, checkpointVersion, deltaVersion}
-
 import org.apache.commons.lang.time.DateUtils
 import org.apache.hadoop.fs.{FileStatus, Path}
 
+import io.delta.standalone.internal.util.FileNames.{checkpointPrefix, checkpointVersion, deltaVersion, isCheckpointFile, isDeltaFile}
+
 private[internal] trait MetadataCleanup {
   self: DeltaLogImpl =>
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
index 16063ac5e92..896d74e4982 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
@@ -27,6 +27,7 @@ import io.delta.standalone.actions.{Action => ActionJ, Metadata => MetadataJ}
 import io.delta.standalone.exceptions.DeltaStandaloneException
 import io.delta.standalone.expressions.{Expression, Literal}
 import io.delta.standalone.types.StructType
+
 import io.delta.standalone.internal.actions.{Action, AddFile, CommitInfo, FileAction, Metadata, Protocol, RemoveFile}
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.logging.Logging
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala
index 97b03da3086..2416af7f231 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala
@@ -20,10 +20,15 @@ import java.net.URI
 
 import scala.collection.JavaConverters._
 
+import com.github.mjakubowski84.parquet4s.ParquetReader
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+
 import io.delta.standalone.{DeltaScan, Snapshot}
 import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ}
 import io.delta.standalone.data.{CloseableIterator, RowRecord => RowParquetRecordJ}
 import io.delta.standalone.expressions.Expression
+
 import io.delta.standalone.internal.actions.{AddFile, InMemoryLogReplay, Metadata, Parquet4sSingleActionWrapper, Protocol, RemoveFile, SetTransaction, SingleAction}
 import io.delta.standalone.internal.data.CloseableParquetDataIterator
 import io.delta.standalone.internal.exception.DeltaErrors
@@ -31,10 +36,6 @@ import io.delta.standalone.internal.logging.Logging
 import io.delta.standalone.internal.scan.{DeltaScanImpl, FilteredDeltaScanImpl}
 import io.delta.standalone.internal.util.{ConversionUtils, FileNames, JsonUtils}
 
-import com.github.mjakubowski84.parquet4s.ParquetReader
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
-
 /**
  * Scala implementation of Java interface [[Snapshot]].
  *
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala
index 12e8907ac4b..5dbd67ef807 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotManagement.scala
@@ -21,11 +21,11 @@ import java.sql.Timestamp
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.fs.{FileStatus, Path}
+
 import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.internal.util.FileNames._
 
-import org.apache.hadoop.fs.{FileStatus, Path}
-
 /**
  * Manages the creation, computation, and access of Snapshot's for Delta tables. Responsibilities
  * include:
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
index 5757fe48e4b..14d2c1cfdf5 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/actions/actions.scala
@@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.{JsonIgnore, JsonInclude, JsonRawValue}
 import com.fasterxml.jackson.core.JsonGenerator
 import com.fasterxml.jackson.databind.{JsonSerializer, SerializerProvider}
 import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize}
+
 import io.delta.standalone.types.StructType
+
 import io.delta.standalone.internal.util.{DataTypeParser, JsonUtils}
 
 private[internal] object Action {
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala
index 3e1c1ba8bdb..013be1774af 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/data/PartitionRowRecord.scala
@@ -20,9 +20,10 @@ import java.math.{BigDecimal => BigDecimalJ}
 import java.sql.{Date, Timestamp}
 
 import io.delta.standalone.data.{RowRecord => RowRecordJ}
-import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.types._
 
+import io.delta.standalone.internal.exception.DeltaErrors
+
 /**
  * A RowRecord representing a Delta Lake partition of Map(partitionKey -> partitionValue)
  */
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala
index 24b09238638..cf604b86ae9 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/data/RowParquetRecordImpl.scala
@@ -55,9 +55,10 @@ import scala.reflect.ClassTag
 import com.github.mjakubowski84.parquet4s._
 
 import io.delta.standalone.data.{RowRecord => RowParquetRecordJ}
-import io.delta.standalone.internal.exception.DeltaErrors
 import io.delta.standalone.types._
 
+import io.delta.standalone.internal.exception.DeltaErrors
+
 /**
  * Scala implementation of Java interface [[RowParquetRecordJ]].
  *
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
index 6b4b27a2f93..f1a88321946 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/exception/DeltaErrors.scala
@@ -20,13 +20,14 @@ import java.io.{FileNotFoundException, IOException}
 
 import scala.annotation.varargs
 
-import io.delta.standalone.types.{DataType, StructType}
+import org.apache.hadoop.fs.Path
+
 import io.delta.standalone.exceptions._
+import io.delta.standalone.types.{DataType, StructType}
+
 import io.delta.standalone.internal.actions.{CommitInfo, Protocol}
 import io.delta.standalone.internal.util.JsonUtils
 
-import org.apache.hadoop.fs.Path
-
 /** A holder object for Delta errors. */
 private[internal] object DeltaErrors {
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala
index e90a1a7e184..f14dbd47eef 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/DeltaScanImpl.scala
@@ -22,6 +22,7 @@ import io.delta.standalone.DeltaScan
 import io.delta.standalone.actions.{AddFile => AddFileJ}
 import io.delta.standalone.data.CloseableIterator
 import io.delta.standalone.expressions.Expression
+
 import io.delta.standalone.internal.actions.AddFile
 import io.delta.standalone.internal.util.ConversionUtils
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala
index d3d88b5bbab..d2791248e80 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/scan/FilteredDeltaScanImpl.scala
@@ -20,6 +20,7 @@ import java.util.Optional
 
 import io.delta.standalone.expressions.Expression
 import io.delta.standalone.types.StructType
+
 import io.delta.standalone.internal.actions.AddFile
 import io.delta.standalone.internal.data.PartitionRowRecord
 import io.delta.standalone.internal.util.PartitionUtils
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala
index 3a093f61c42..ea0ffc97cbf 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HDFSLogStore.scala
@@ -24,14 +24,14 @@ import java.util.EnumSet
 import scala.collection.JavaConverters._
 import scala.util.control.NonFatal
 
-import io.delta.standalone.internal.exception.DeltaErrors
-import io.delta.standalone.internal.logging.Logging
-
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileContext, Options, Path, RawLocalFileSystem}
 import org.apache.hadoop.fs.CreateFlag.CREATE
 import org.apache.hadoop.fs.Options.{ChecksumOpt, CreateOpts}
 
+import io.delta.standalone.internal.exception.DeltaErrors
+import io.delta.standalone.internal.logging.Logging
+
 /**
  * The [[LogStore]] implementation for HDFS, which uses Hadoop [[FileContext]] APIs to
  * provide the necessary atomic and durability guarantees:
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala
index 413333b3d22..95878d9501e 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/HadoopFileSystemLogStore.scala
@@ -23,14 +23,15 @@ import java.util.UUID
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, Path}
+
 import io.delta.standalone.data.CloseableIterator
 import io.delta.standalone.storage.LogStore
 
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
-
 /**
- * Default implementation of [[LogStore]] for Hadoop [[FileSystem]] implementations.
+ * Default implementation of [[LogStore]] for Hadoop [[org.apache.hadoop.fs.FileSystem]]
+ * implementations.
  */
 abstract class HadoopFileSystemLogStore(override val initHadoopConf: Configuration)
   extends LogStore(initHadoopConf) {
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala
index f691c46c111..3914e4ac180 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LineCloseableIterator.scala
@@ -18,9 +18,10 @@ package io.delta.standalone.internal.storage
 
 import java.io.Reader
 
-import io.delta.standalone.data.CloseableIterator
 import org.apache.commons.io.IOUtils
 
+import io.delta.standalone.data.CloseableIterator
+
 /**
  * Turn a `Reader` to `ClosableIterator` which can be read on demand. Each element is
  * a trimmed line.
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala
index 828daeb99a9..b2de2f7ee68 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/storage/LogStoreProvider.scala
@@ -16,11 +16,12 @@
 
 package io.delta.standalone.internal.storage
 
+import org.apache.hadoop.conf.Configuration
+
 import io.delta.standalone.exceptions.DeltaStandaloneException
 import io.delta.standalone.storage.LogStore
-import io.delta.standalone.internal.sources.StandaloneHadoopConf
 
-import org.apache.hadoop.conf.Configuration
+import io.delta.standalone.internal.sources.StandaloneHadoopConf
 
 private[internal] object LogStoreProvider extends LogStoreProvider
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala
index aedf8071dae..d394cd97889 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/util/ConversionUtils.scala
@@ -21,7 +21,8 @@ import java.util.{Optional => OptionalJ}
 
 import scala.collection.JavaConverters._
 
-import io.delta.standalone.actions.{Action => ActionJ, AddFile => AddFileJ, AddCDCFile => AddCDCFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ}
+import io.delta.standalone.actions.{Action => ActionJ, AddCDCFile => AddCDCFileJ, AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, Protocol => ProtocolJ, RemoveFile => RemoveFileJ, SetTransaction => SetTransactionJ}
+
 import io.delta.standalone.internal.actions.{Action, AddCDCFile, AddFile, CommitInfo, Format, JobInfo, Metadata, NotebookInfo, Protocol, RemoveFile, SetTransaction}
 
 /**
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala
index 8fb3058db0d..7e081efe49f 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/util/DataTypeParser.scala
@@ -38,12 +38,14 @@
 
 package io.delta.standalone.internal.util
 
-import io.delta.standalone.types._
+import scala.collection.JavaConverters._
+
 import org.json4s._
-import org.json4s.jackson.JsonMethods._
-import org.json4s.JsonDSL._
 import org.json4s.JsonAST.JValue
-import scala.collection.JavaConverters._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods._
+
+import io.delta.standalone.types._
 
 private[standalone] object DataTypeParser {
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala
index b3603161b34..113966c6350 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/util/PartitionUtils.scala
@@ -21,9 +21,11 @@ import java.util.Locale
 import scala.collection.JavaConverters._
 
 import io.delta.standalone.expressions.{And, Expression, Literal}
+import io.delta.standalone.types.StructType
+
 import io.delta.standalone.internal.actions.AddFile
 import io.delta.standalone.internal.data.PartitionRowRecord
-import io.delta.standalone.types.StructType
+
 
 private[internal] object PartitionUtils {
 
diff --git a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala
index ce1329c5179..70db408071e 100644
--- a/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala
+++ b/standalone/src/main/scala/io/delta/standalone/internal/util/SchemaUtils.scala
@@ -18,6 +18,7 @@ package io.delta.standalone.internal.util
 
 import io.delta.standalone.exceptions.DeltaStandaloneException
 import io.delta.standalone.types.{ArrayType, DataType, MapType, StructField, StructType}
+
 import io.delta.standalone.internal.exception.DeltaErrors
 
 private[internal] object SchemaUtils {

From b0cbaa8d36f404f795d97ef3dc9988f150009f00 Mon Sep 17 00:00:00 2001
From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com>
Date: Thu, 28 Oct 2021 13:58:23 -0700
Subject: [PATCH 118/291] [DSW] [31] Copyright (#190)

---
 LICENSE.txt                                      |  2 +-
 NOTICE.txt                                       |  2 +-
 build.sbt                                        |  2 +-
 build/sbt                                        |  2 +-
 .../scala/io/delta/golden/GoldenTables.scala     |  2 +-
 .../test/scala/io/delta/hive/HiveMRSuite.scala   |  2 +-
 .../test/scala/io/delta/hive/HiveTezSuite.scala  |  2 +-
 .../main/java/io/delta/hive/DeltaInputSplit.java |  2 +-
 .../io/delta/hive/IndexPredicateAnalyzer.java    |  2 +-
 .../scala/io/delta/hive/CaseInsensitiveMap.scala |  2 +-
 .../main/scala/io/delta/hive/DeltaHelper.scala   |  2 +-
 .../scala/io/delta/hive/DeltaInputFormat.scala   |  2 +-
 .../scala/io/delta/hive/DeltaOutputFormat.scala  |  2 +-
 .../io/delta/hive/DeltaRecordReaderWrapper.scala |  2 +-
 .../io/delta/hive/DeltaStorageHandler.scala      |  2 +-
 .../scala/io/delta/hive/HiveInputFormat.scala    |  2 +-
 .../io/delta/hive/PartitionColumnInfo.scala      |  2 +-
 .../main/scala/io/delta/hive/SchemaUtils.scala   |  2 +-
 hive/src/test/resources/log4j.properties         |  2 +-
 .../scala/io/delta/hive/HiveConnectorTest.scala  |  2 +-
 .../test/scala/io/delta/hive/test/HiveTest.scala |  2 +-
 .../tests/OSSCompatibilitySuite.scala            |  2 +-
 .../tests/OssCompatibilitySuiteBase.scala        |  2 +-
 .../internal/util/ComparisonUtil.scala           |  2 +-
 .../delta/standalone/internal/util/OSSUtil.scala |  2 +-
 .../internal/util/StandaloneUtil.scala           |  2 +-
 project/StandaloneMimaExcludes.scala             |  2 +-
 project/build.properties                         |  2 +-
 project/plugins.sbt                              |  2 +-
 scalastyle-config.xml                            |  4 ++--
 .../src/main/scala/DataTransforms.scala          |  2 +-
 .../src/main/scala/ImportRunner.scala            |  2 +-
 sql-delta-import/src/main/scala/JDBCImport.scala |  2 +-
 sql-delta-import/src/test/scala/ImportTest.scala |  2 +-
 .../java/io/delta/standalone/CommitResult.java   | 16 +++++++++++++++-
 .../main/java/io/delta/standalone/DeltaLog.java  |  2 +-
 .../main/java/io/delta/standalone/DeltaScan.java |  2 +-
 .../main/java/io/delta/standalone/Operation.java | 16 +++++++++++++++-
 .../delta/standalone/OptimisticTransaction.java  | 16 +++++++++++++++-
 .../main/java/io/delta/standalone/Snapshot.java  |  2 +-
 .../java/io/delta/standalone/VersionLog.java     | 16 +++++++++++++++-
 .../java/io/delta/standalone/actions/Action.java | 16 +++++++++++++++-
 .../io/delta/standalone/actions/AddCDCFile.java  | 16 +++++++++++++++-
 .../io/delta/standalone/actions/AddFile.java     |  2 +-
 .../io/delta/standalone/actions/CommitInfo.java  |  2 +-
 .../io/delta/standalone/actions/FileAction.java  | 16 +++++++++++++++-
 .../java/io/delta/standalone/actions/Format.java |  2 +-
 .../io/delta/standalone/actions/JobInfo.java     |  2 +-
 .../io/delta/standalone/actions/Metadata.java    |  2 +-
 .../delta/standalone/actions/NotebookInfo.java   |  2 +-
 .../io/delta/standalone/actions/Protocol.java    |  2 +-
 .../io/delta/standalone/actions/RemoveFile.java  | 16 +++++++++++++++-
 .../delta/standalone/actions/SetTransaction.java | 16 +++++++++++++++-
 .../delta/standalone/data/CloseableIterator.java |  2 +-
 .../java/io/delta/standalone/data/RowRecord.java |  2 +-
 .../exceptions/ConcurrentAppendException.java    |  2 +-
 .../ConcurrentDeleteDeleteException.java         |  2 +-
 .../ConcurrentDeleteReadException.java           |  2 +-
 .../ConcurrentTransactionException.java          |  2 +-
 .../DeltaConcurrentModificationException.java    |  2 +-
 .../exceptions/DeltaStandaloneException.java     |  2 +-
 .../exceptions/MetadataChangedException.java     |  2 +-
 .../exceptions/ProtocolChangedException.java     |  2 +-
 .../standalone/expressions/BinaryExpression.java | 16 +++++++++++++++-
 .../delta/standalone/expressions/Expression.java | 16 +++++++++++++++-
 .../standalone/expressions/GreaterThan.java      | 16 +++++++++++++++-
 .../java/io/delta/standalone/expressions/In.java | 16 +++++++++++++++-
 .../standalone/expressions/UnaryExpression.java  | 16 +++++++++++++++-
 .../io/delta/standalone/storage/LogStore.java    |  4 ++--
 .../io/delta/standalone/types/ArrayType.java     |  2 +-
 .../io/delta/standalone/types/BinaryType.java    |  2 +-
 .../io/delta/standalone/types/BooleanType.java   |  2 +-
 .../java/io/delta/standalone/types/ByteType.java |  2 +-
 .../java/io/delta/standalone/types/DataType.java |  2 +-
 .../java/io/delta/standalone/types/DateType.java |  2 +-
 .../io/delta/standalone/types/DecimalType.java   |  2 +-
 .../io/delta/standalone/types/DoubleType.java    |  2 +-
 .../io/delta/standalone/types/FieldMetadata.java |  2 +-
 .../io/delta/standalone/types/FloatType.java     |  2 +-
 .../io/delta/standalone/types/IntegerType.java   |  2 +-
 .../java/io/delta/standalone/types/LongType.java |  2 +-
 .../java/io/delta/standalone/types/MapType.java  |  2 +-
 .../java/io/delta/standalone/types/NullType.java |  2 +-
 .../io/delta/standalone/types/ShortType.java     |  2 +-
 .../io/delta/standalone/types/StringType.java    |  2 +-
 .../io/delta/standalone/types/StructField.java   |  2 +-
 .../io/delta/standalone/types/StructType.java    |  2 +-
 .../io/delta/standalone/types/TimestampType.java |  2 +-
 .../standalone/util/ParquetSchemaConverter.java  |  2 +-
 .../internal/BufferingLogDeletionIterator.scala  |  2 +-
 .../delta/standalone/internal/Checkpoints.scala  |  2 +-
 .../standalone/internal/ConflictChecker.scala    |  2 +-
 .../delta/standalone/internal/DeltaConfig.scala  |  2 +-
 .../internal/DeltaHistoryManager.scala           |  2 +-
 .../delta/standalone/internal/DeltaLogImpl.scala |  2 +-
 .../standalone/internal/MetadataCleanup.scala    |  2 +-
 .../internal/OptimisticTransactionImpl.scala     |  2 +-
 .../delta/standalone/internal/SnapshotImpl.scala |  2 +-
 .../standalone/internal/SnapshotManagement.scala |  2 +-
 .../internal/actions/InMemoryLogReplay.scala     |  2 +-
 .../standalone/internal/actions/actions.scala    |  2 +-
 .../data/CloseableParquetDataIterator.scala      |  2 +-
 .../internal/data/PartitionRowRecord.scala       |  2 +-
 .../internal/data/RowParquetRecordImpl.scala     |  2 +-
 .../internal/exception/DeltaErrors.scala         |  2 +-
 .../standalone/internal/isolationLevels.scala    |  2 +-
 .../standalone/internal/logging/Logging.scala    |  2 +-
 .../standalone/internal/scan/DeltaScanImpl.scala |  2 +-
 .../internal/scan/FilteredDeltaScanImpl.scala    |  2 +-
 .../internal/sources/StandaloneHadoopConf.scala  |  2 +-
 .../internal/storage/HDFSLogStore.scala          |  2 +-
 .../storage/HadoopFileSystemLogStore.scala       |  2 +-
 .../internal/storage/LineCloseableIterator.scala |  2 +-
 .../internal/storage/LogStoreProvider.scala      |  2 +-
 .../internal/util/CalendarInterval.scala         |  2 +-
 .../internal/util/CaseInsensitiveMap.scala       |  2 +-
 .../delta/standalone/internal/util/Clock.scala   |  2 +-
 .../internal/util/ConversionUtils.scala          |  2 +-
 .../internal/util/DataTypeParser.scala           |  2 +-
 .../internal/util/DateTimeConstants.scala        |  2 +-
 .../standalone/internal/util/FileNames.scala     |  2 +-
 .../standalone/internal/util/Implicits.scala     |  2 +-
 .../standalone/internal/util/IntervalUtils.scala |  2 +-
 .../standalone/internal/util/JsonUtils.scala     |  2 +-
 .../standalone/internal/util/ManualClock.scala   |  2 +-
 .../internal/util/ParquetSchemaConverter.scala   |  2 +-
 .../internal/util/PartitionUtils.scala           |  2 +-
 .../internal/util/SchemaMergingUtils.scala       |  2 +-
 .../standalone/internal/util/SchemaUtils.scala   |  2 +-
 standalone/src/test/resources/log4j.properties   |  2 +-
 .../internal/ActionBuildersSuite.scala           |  2 +-
 .../BufferingLogDeletionIteratorSuite.scala      |  2 +-
 .../internal/ConversionUtilsSuite.scala          |  2 +-
 .../standalone/internal/DeltaConfigSuite.scala   |  2 +-
 .../internal/DeltaDataReaderSuite.scala          |  2 +-
 .../standalone/internal/DeltaLogSuite.scala      |  2 +-
 .../internal/DeltaRetentionSuite.scala           |  2 +-
 .../internal/DeltaRetentionSuiteBase.scala       |  2 +-
 .../standalone/internal/DeltaScanSuite.scala     |  2 +-
 .../internal/DeltaTimeTravelSuite.scala          |  2 +-
 .../standalone/internal/ExpressionSuite.scala    |  2 +-
 .../standalone/internal/LogStoreSuite.scala      |  2 +-
 .../OptimisticTransactionLegacySuite.scala       |  2 +-
 .../internal/OptimisticTransactionSuite.scala    |  2 +-
 .../OptimisticTransactionSuiteBase.scala         |  2 +-
 .../OptimisticTransactionSuiteTestVals.scala     |  2 +-
 .../internal/ParquetSchemaConverterSuite.scala   |  2 +-
 .../standalone/internal/SchemaUtilsSuite.scala   |  2 +-
 .../internal/util/GoldenTableUtils.scala         |  2 +-
 .../standalone/internal/util/TestUtils.scala     |  2 +-
 .../src/test/scala/ShadedJarSuite.scala          |  2 +-
 151 files changed, 349 insertions(+), 153 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index 06889e2bbe5..a1ceba72234 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -1,4 +1,4 @@
-Copyright (2020) The Delta Lake Project Authors.  All rights reserved.
+Copyright (2020-present) The Delta Lake Project Authors.  All rights reserved.
 
 
                                  Apache License
diff --git a/NOTICE.txt b/NOTICE.txt
index 4fe11e3f96f..1341a99ce5c 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -1,5 +1,5 @@
 Delta Lake Connectors
-Copyright (2020) The Delta Lake Project Authors.
+Copyright (2020-present) The Delta Lake Project Authors.
 
 Licensed under the Apache License, Version 2.0 (the "License");
 you may not use this file except in compliance with the License.
diff --git a/build.sbt b/build.sbt
index 70a0edc0fc0..4c0311ad193 100644
--- a/build.sbt
+++ b/build.sbt
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/build/sbt b/build/sbt
index 3a2a4e72222..e2b247e35c8 100755
--- a/build/sbt
+++ b/build/sbt
@@ -23,7 +23,7 @@
 #
 
 #
-#  Copyright (2020) The Delta Lake Project Authors.
+#  Copyright (2020-present) The Delta Lake Project Authors.
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
 #  You may obtain a copy of the License at
diff --git a/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala b/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala
index fb858351d2e..baa4a5b2723 100644
--- a/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala
+++ b/golden-tables/src/test/scala/io/delta/golden/GoldenTables.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala
index 10e637f14ff..e3dbf8c029a 100644
--- a/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala
+++ b/hive-mr/src/test/scala/io/delta/hive/HiveMRSuite.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala
index 37f78cac03a..0af5c386814 100644
--- a/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala
+++ b/hive-tez/src/test/scala/io/delta/hive/HiveTezSuite.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java
index faaa83ed9ee..6b381e1b5e1 100644
--- a/hive/src/main/java/io/delta/hive/DeltaInputSplit.java
+++ b/hive/src/main/java/io/delta/hive/DeltaInputSplit.java
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java
index f9cca4bbe54..267b6498629 100644
--- a/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java
+++ b/hive/src/main/java/io/delta/hive/IndexPredicateAnalyzer.java
@@ -22,7 +22,7 @@
  */
 
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/CaseInsensitiveMap.scala b/hive/src/main/scala/io/delta/hive/CaseInsensitiveMap.scala
index 04427a5c521..2ace7da1840 100644
--- a/hive/src/main/scala/io/delta/hive/CaseInsensitiveMap.scala
+++ b/hive/src/main/scala/io/delta/hive/CaseInsensitiveMap.scala
@@ -21,7 +21,7 @@
  */
 
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
index 95d12f3cdf6..697bab99dcd 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaHelper.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
index a36476e0fd8..249cea5a19b 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaInputFormat.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala
index c5734c102a1..0ccc130bb45 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaOutputFormat.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala
index bc49ebb8685..d55c633fe12 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaRecordReaderWrapper.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala
index 330fefa09ae..193d6bd9d90 100644
--- a/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala
+++ b/hive/src/main/scala/io/delta/hive/DeltaStorageHandler.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala
index 835c8d7a76a..1d017f4e9bb 100644
--- a/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala
+++ b/hive/src/main/scala/io/delta/hive/HiveInputFormat.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala
index b7367dd8451..52ccb1f5a6a 100644
--- a/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala
+++ b/hive/src/main/scala/io/delta/hive/PartitionColumnInfo.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/main/scala/io/delta/hive/SchemaUtils.scala b/hive/src/main/scala/io/delta/hive/SchemaUtils.scala
index 50079d9d1e9..99dfde4bc6d 100644
--- a/hive/src/main/scala/io/delta/hive/SchemaUtils.scala
+++ b/hive/src/main/scala/io/delta/hive/SchemaUtils.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/test/resources/log4j.properties b/hive/src/test/resources/log4j.properties
index ed93eca2537..37b5230dadd 100644
--- a/hive/src/test/resources/log4j.properties
+++ b/hive/src/test/resources/log4j.properties
@@ -1,5 +1,5 @@
 #
-#  Copyright (2020) The Delta Lake Project Authors.
+#  Copyright (2020-present) The Delta Lake Project Authors.
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
 #  You may obtain a copy of the License at
diff --git a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala
index f4f41505408..c5e588c9c98 100644
--- a/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala
+++ b/hive/src/test/scala/io/delta/hive/HiveConnectorTest.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala
index 9d1fc443910..159feabf222 100644
--- a/hive/src/test/scala/io/delta/hive/test/HiveTest.scala
+++ b/hive/src/test/scala/io/delta/hive/test/HiveTest.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala
index 1e3f6d4989b..8518b66f302 100644
--- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala
+++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OSSCompatibilitySuite.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2021) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala
index 399a736d212..c2b0ac97816 100644
--- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala
+++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/compatibility/tests/OssCompatibilitySuiteBase.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2021) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala
index db06b21e58a..b65d42e2400 100644
--- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala
+++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/ComparisonUtil.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2021) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala
index 3283e5bf125..446195efed7 100644
--- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala
+++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/OSSUtil.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2021) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala
index 689f745684c..247a44401ea 100644
--- a/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala
+++ b/oss-compatibility-tests/src/test/scala/io/delta/standalone/internal/util/StandaloneUtil.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2021) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/project/StandaloneMimaExcludes.scala b/project/StandaloneMimaExcludes.scala
index 2653d1c1561..a8449cb38da 100644
--- a/project/StandaloneMimaExcludes.scala
+++ b/project/StandaloneMimaExcludes.scala
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/project/build.properties b/project/build.properties
index a3c4896e6e7..a841192ca30 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -21,7 +21,7 @@
 #
 
 #
-#  Copyright (2020) The Delta Lake Project Authors.
+#  Copyright (2020-present) The Delta Lake Project Authors.
 #  Licensed under the Apache License, Version 2.0 (the "License");
 #  you may not use this file except in compliance with the License.
 #  You may obtain a copy of the License at
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 1c1c106a73f..3074b3504a0 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -1,5 +1,5 @@
 /*
- * Copyright (2020) The Delta Lake Project Authors.
+ * Copyright (2020-present) The Delta Lake Project Authors.
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index fd78efa4610..2143de7ae98 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -19,7 +19,7 @@
   ~It contains modifications, which are licensed as follows:
   -->
 
+
+
+
+All Classes (Delta Standalone Reader 0.2.1 JavaDoc)
+
+
+
+
+
+

All Classes

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html new file mode 100644 index 00000000000..f7c67cdedfd --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html @@ -0,0 +1,97 @@ + + + + + +All Classes (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

All Classes

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html new file mode 100644 index 00000000000..74286369323 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html @@ -0,0 +1,305 @@ + + + + + +Constant Field Values (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Constant Field Values

+

Contents

+ +
+
+ + +

io.delta.*

+ +
    +
  • + + + + + + + + + + + + + + + + + + + +
    io.delta.standalone.actions.Action 
    Modifier and TypeConstant FieldValue
    + +public static final intreaderVersion1
    + +public static final intwriterVersion2
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html new file mode 100644 index 00000000000..556d32ddc7c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html @@ -0,0 +1,122 @@ + + + + + +Deprecated List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

Deprecated API

+

Contents

+
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html new file mode 100644 index 00000000000..02ce022ccdc --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html @@ -0,0 +1,223 @@ + + + + + +API Help (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
+

How This API Document Is Organized

+
This API (Application Programming Interface) document has pages corresponding to the items in the navigation bar, described as follows.
+
+
+
    +
  • +

    Overview

    +

    The Overview page is the front page of this API document and provides a list of all packages with a summary for each. This page can also contain an overall description of the set of packages.

    +
  • +
  • +

    Package

    +

    Each package has a page that contains a list of its classes and interfaces, with a summary for each. This page can contain six categories:

    +
      +
    • Interfaces (italic)
    • +
    • Classes
    • +
    • Enums
    • +
    • Exceptions
    • +
    • Errors
    • +
    • Annotation Types
    • +
    +
  • +
  • +

    Class/Interface

    +

    Each class, interface, nested class and nested interface has its own separate page. Each of these pages has three sections consisting of a class/interface description, summary tables, and detailed member descriptions:

    +
      +
    • Class inheritance diagram
    • +
    • Direct Subclasses
    • +
    • All Known Subinterfaces
    • +
    • All Known Implementing Classes
    • +
    • Class/interface declaration
    • +
    • Class/interface description
    • +
    +
      +
    • Nested Class Summary
    • +
    • Field Summary
    • +
    • Constructor Summary
    • +
    • Method Summary
    • +
    +
      +
    • Field Detail
    • +
    • Constructor Detail
    • +
    • Method Detail
    • +
    +

    Each summary entry contains the first sentence from the detailed description for that item. The summary entries are alphabetical, while the detailed descriptions are in the order they appear in the source code. This preserves the logical groupings established by the programmer.

    +
  • +
  • +

    Annotation Type

    +

    Each annotation type has its own separate page with the following sections:

    +
      +
    • Annotation Type declaration
    • +
    • Annotation Type description
    • +
    • Required Element Summary
    • +
    • Optional Element Summary
    • +
    • Element Detail
    • +
    +
  • +
  • +

    Enum

    +

    Each enum has its own separate page with the following sections:

    +
      +
    • Enum declaration
    • +
    • Enum description
    • +
    • Enum Constant Summary
    • +
    • Enum Constant Detail
    • +
    +
  • +
  • +

    Tree (Class Hierarchy)

    +

    There is a Class Hierarchy page for all packages, plus a hierarchy for each package. Each hierarchy page contains a list of classes and a list of interfaces. The classes are organized by inheritance structure starting with java.lang.Object. The interfaces do not inherit from java.lang.Object.

    +
      +
    • When viewing the Overview page, clicking on "Tree" displays the hierarchy for all packages.
    • +
    • When viewing a particular package, class or interface page, clicking "Tree" displays the hierarchy for only that package.
    • +
    +
  • +
  • +

    Deprecated API

    +

    The Deprecated API page lists all of the API that have been deprecated. A deprecated API is not recommended for use, generally due to improvements, and a replacement API is usually given. Deprecated APIs may be removed in future implementations.

    +
  • +
  • +

    Index

    +

    The Index contains an alphabetic list of all classes, interfaces, constructors, methods, and fields.

    +
  • +
  • +

    Prev/Next

    +

    These links take you to the next or previous class, interface, package, or related page.

    +
  • +
  • +

    Frames/No Frames

    +

    These links show and hide the HTML frames. All pages are available with or without frames.

    +
  • +
  • +

    All Classes

    +

    The All Classes link shows all classes and interfaces except non-static nested types.

    +
  • +
  • +

    Serialized Form

    +

    Each serializable or externalizable class has a description of its serialization fields and methods. This information is of interest to re-implementors, not to developers using the API. While there is no link in the navigation bar, you can get to this information by going to any serialized class and clicking "Serialized Form" in the "See also" section of the class description.

    +
  • +
  • +

    Constant Field Values

    +

    The Constant Field Values page lists the static final fields and their values.

    +
  • +
+This help file applies to API documentation generated using the standard doclet.
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html new file mode 100644 index 00000000000..bb8cdff396f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html @@ -0,0 +1,1547 @@ + + + + + +Index (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + +
A B C D E F G H I J L M N O P R S T U V W  + + +

A

+
+
Action - Interface in io.delta.standalone.actions
+
+
A marker interface for all Actions that can be applied to a Delta Table.
+
+
add(StructField) - Method in class io.delta.standalone.types.StructType
+
+
Creates a new StructType by adding a new field.
+
+
add(String, DataType) - Method in class io.delta.standalone.types.StructType
+
+
Creates a new StructType by adding a new nullable field with no metadata.
+
+
add(String, DataType, boolean) - Method in class io.delta.standalone.types.StructType
+
+
Creates a new StructType by adding a new field with no metadata.
+
+
AddCDCFile - Class in io.delta.standalone.actions
+
+
A change file containing CDC data for the Delta version it's within.
+
+
AddCDCFile(String, Map<String, String>, long, Map<String, String>) - Constructor for class io.delta.standalone.actions.AddCDCFile
+
 
+
AddFile - Class in io.delta.standalone.actions
+
+
Represents an action that adds a new file to the table.
+
+
AddFile(String, Map<String, String>, long, long, boolean, String, Map<String, String>) - Constructor for class io.delta.standalone.actions.AddFile
+
 
+
AddFile.Builder - Class in io.delta.standalone.actions
+
+
Builder class for AddFile.
+
+
And - Class in io.delta.standalone.expressions
+
+
Usage: new And(expr1, expr2) - Logical AND
+
+
And(Expression, Expression) - Constructor for class io.delta.standalone.expressions.And
+
 
+
ArrayType - Class in io.delta.standalone.types
+
+
The data type for collections of multiple values.
+
+
ArrayType(DataType, boolean) - Constructor for class io.delta.standalone.types.ArrayType
+
 
+
+ + + +

B

+
+
BinaryComparison - Class in io.delta.standalone.expressions
+
+
A BinaryOperator that compares the left and right Expressions and returns a + boolean value.
+
+
BinaryComparison(Expression, Expression, String) - Constructor for class io.delta.standalone.expressions.BinaryComparison
+
 
+
BinaryExpression - Class in io.delta.standalone.expressions
+
+
An expression with two inputs and one output.
+
+
BinaryExpression(Expression, Expression) - Constructor for class io.delta.standalone.expressions.BinaryExpression
+
 
+
BinaryOperator - Class in io.delta.standalone.expressions
+
+
A BinaryExpression that is an operator, with two properties: + + The string representation is x symbol y, rather than funcName(x, y). + Two inputs are expected to be of the same type.
+
+
BinaryOperator(Expression, Expression, String) - Constructor for class io.delta.standalone.expressions.BinaryOperator
+
 
+
BinaryType - Class in io.delta.standalone.types
+
+
The data type representing byte[] values.
+
+
BinaryType() - Constructor for class io.delta.standalone.types.BinaryType
+
 
+
BooleanType - Class in io.delta.standalone.types
+
+
The data type representing boolean values.
+
+
BooleanType() - Constructor for class io.delta.standalone.types.BooleanType
+
 
+
build() - Method in class io.delta.standalone.actions.AddFile.Builder
+
 
+
build() - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
build() - Method in class io.delta.standalone.actions.JobInfo.Builder
+
 
+
build() - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
build() - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
build() - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
builder(String, Map<String, String>, long, long, boolean) - Static method in class io.delta.standalone.actions.AddFile
+
 
+
Builder(String, Map<String, String>, long, long, boolean) - Constructor for class io.delta.standalone.actions.AddFile.Builder
+
 
+
builder() - Static method in class io.delta.standalone.actions.CommitInfo
+
 
+
Builder() - Constructor for class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
builder(String) - Static method in class io.delta.standalone.actions.JobInfo
+
 
+
Builder(String) - Constructor for class io.delta.standalone.actions.JobInfo.Builder
+
 
+
builder() - Static method in class io.delta.standalone.actions.Metadata
+
 
+
Builder() - Constructor for class io.delta.standalone.actions.Metadata.Builder
+
 
+
builder(String) - Static method in class io.delta.standalone.actions.RemoveFile
+
 
+
Builder(String) - Constructor for class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
builder() - Static method in class io.delta.standalone.types.FieldMetadata
+
 
+
Builder() - Constructor for class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
ByteType - Class in io.delta.standalone.types
+
+
The data type representing byte values.
+
+
ByteType() - Constructor for class io.delta.standalone.types.ByteType
+
 
+
+ + + +

C

+
+
children() - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
children() - Method in interface io.delta.standalone.expressions.Expression
+
 
+
children() - Method in class io.delta.standalone.expressions.In
+
 
+
children() - Method in class io.delta.standalone.expressions.LeafExpression
+
 
+
children() - Method in class io.delta.standalone.expressions.UnaryExpression
+
 
+
CloseableIterator<T> - Interface in io.delta.standalone.data
+
+
An Iterator that also need to implement the Closeable interface.
+
+
clusterId(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
Column - Class in io.delta.standalone.expressions
+
+
A column whose row-value will be computed based on the data in a RowRecord.
+
+
Column(String, DataType) - Constructor for class io.delta.standalone.expressions.Column
+
 
+
column(String) - Method in class io.delta.standalone.types.StructType
+
 
+
commit(Iterable<T>, Operation, String) - Method in interface io.delta.standalone.OptimisticTransaction
+
+
Modifies the state of the log by adding a new commit that is based on a read at + the given `lastVersion`.
+
+
CommitInfo - Class in io.delta.standalone.actions
+
+
Holds provenance information about changes to the table.
+
+
CommitInfo(Optional<Long>, Timestamp, Optional<String>, Optional<String>, String, Map<String, String>, Optional<JobInfo>, Optional<NotebookInfo>, Optional<String>, Optional<Long>, Optional<String>, Optional<Boolean>, Optional<Map<String, String>>, Optional<String>) - Constructor for class io.delta.standalone.actions.CommitInfo
+
 
+
CommitInfo(Optional<Long>, Timestamp, Optional<String>, Optional<String>, String, Map<String, String>, Optional<JobInfo>, Optional<NotebookInfo>, Optional<String>, Optional<Long>, Optional<String>, Optional<Boolean>, Optional<Map<String, String>>, Optional<String>, Optional<String>) - Constructor for class io.delta.standalone.actions.CommitInfo
+
 
+
CommitInfo.Builder - Class in io.delta.standalone.actions
+
+
Builder class for CommitInfo.
+
+
CommitResult - Class in io.delta.standalone
+
+ +
+
CommitResult(long) - Constructor for class io.delta.standalone.CommitResult
+
 
+
ConcurrentAppendException - Exception in io.delta.standalone.exceptions
+
+
Thrown when files are added that would have been read by the current transaction.
+
+
ConcurrentAppendException(String) - Constructor for exception io.delta.standalone.exceptions.ConcurrentAppendException
+
 
+
ConcurrentDeleteDeleteException - Exception in io.delta.standalone.exceptions
+
+
Thrown when the current transaction deletes data that was deleted by a concurrent transaction.
+
+
ConcurrentDeleteDeleteException(String) - Constructor for exception io.delta.standalone.exceptions.ConcurrentDeleteDeleteException
+
 
+
ConcurrentDeleteReadException - Exception in io.delta.standalone.exceptions
+
+
Thrown when the current transaction reads data that was deleted by a concurrent transaction.
+
+
ConcurrentDeleteReadException(String) - Constructor for exception io.delta.standalone.exceptions.ConcurrentDeleteReadException
+
 
+
ConcurrentTransactionException - Exception in io.delta.standalone.exceptions
+
+
Thrown when concurrent transaction both attempt to update the same idempotent transaction.
+
+
ConcurrentTransactionException(String) - Constructor for exception io.delta.standalone.exceptions.ConcurrentTransactionException
+
 
+
configuration(Map<String, String>) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
contains(String) - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
containsNull() - Method in class io.delta.standalone.types.ArrayType
+
 
+
createdTime(Long) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
+ + + +

D

+
+
dataChange(boolean) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
dataType() - Method in class io.delta.standalone.expressions.Column
+
 
+
dataType() - Method in interface io.delta.standalone.expressions.Expression
+
 
+
dataType() - Method in class io.delta.standalone.expressions.Literal
+
 
+
dataType() - Method in interface io.delta.standalone.expressions.Predicate
+
 
+
DataType - Class in io.delta.standalone.types
+
+
The base type of all io.delta.standalone data types.
+
+
DataType() - Constructor for class io.delta.standalone.types.DataType
+
 
+
DateType - Class in io.delta.standalone.types
+
+
A date type, supporting "0001-01-01" through "9999-12-31".
+
+
DateType() - Constructor for class io.delta.standalone.types.DateType
+
 
+
DecimalType - Class in io.delta.standalone.types
+
+
The data type representing java.math.BigDecimal values.
+
+
DecimalType(int, int) - Constructor for class io.delta.standalone.types.DecimalType
+
 
+
deletionTimestamp(Long) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
DeltaConcurrentModificationException - Exception in io.delta.standalone.exceptions
+
+
The basic class for all Delta Standalone commit conflict exceptions.
+
+
DeltaConcurrentModificationException(String) - Constructor for exception io.delta.standalone.exceptions.DeltaConcurrentModificationException
+
 
+
DeltaLog - Interface in io.delta.standalone
+
+
DeltaLog is the representation of the transaction logs of a Delta table.
+
+
DeltaScan - Interface in io.delta.standalone
+
+
Provides access to an iterator over the files in this snapshot.
+
+
DeltaStandaloneException - Exception in io.delta.standalone.exceptions
+
+
Thrown when a query fails, usually because the query itself is invalid.
+
+
DeltaStandaloneException() - Constructor for exception io.delta.standalone.exceptions.DeltaStandaloneException
+
 
+
DeltaStandaloneException(String) - Constructor for exception io.delta.standalone.exceptions.DeltaStandaloneException
+
 
+
DeltaStandaloneException(String, Throwable) - Constructor for exception io.delta.standalone.exceptions.DeltaStandaloneException
+
 
+
description(String) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
DoubleType - Class in io.delta.standalone.types
+
+
The data type representing double values.
+
+
DoubleType() - Constructor for class io.delta.standalone.types.DoubleType
+
 
+
+ + + +

E

+
+
engineInfo(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.AddFile
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Format
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.JobInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Metadata
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.Protocol
+
 
+
equals(Object) - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
equals(Object) - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
equals(Object) - Method in class io.delta.standalone.expressions.Column
+
 
+
equals(Object) - Method in class io.delta.standalone.expressions.LeafExpression
+
 
+
equals(Object) - Method in class io.delta.standalone.expressions.Literal
+
 
+
equals(Object) - Method in class io.delta.standalone.expressions.UnaryExpression
+
 
+
equals(Object) - Method in class io.delta.standalone.types.ArrayType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DataType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.DecimalType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
equals(Object) - Method in class io.delta.standalone.types.MapType
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructField
+
 
+
equals(Object) - Method in class io.delta.standalone.types.StructType
+
 
+
EqualTo - Class in io.delta.standalone.expressions
+
+
Usage: new EqualTo(expr1, expr2) - Returns true if expr1 equals expr2, + else false.
+
+
EqualTo(Expression, Expression) - Constructor for class io.delta.standalone.expressions.EqualTo
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.Column
+
 
+
eval(RowRecord) - Method in interface io.delta.standalone.expressions.Expression
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.In
+
+
This implements the IN expression functionality outlined by the Databricks SQL Null + semantics reference guide.
+
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.IsNotNull
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.IsNull
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.Literal
+
 
+
eval(RowRecord) - Method in class io.delta.standalone.expressions.UnaryExpression
+
 
+
executionTimeMs - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Time taken to execute the entire operation.
+
+
Expression - Interface in io.delta.standalone.expressions
+
+
An expression in Delta Standalone.
+
+
extendedFileMetadata(boolean) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
+ + + +

F

+
+
False - Static variable in class io.delta.standalone.expressions.Literal
+
 
+
FieldMetadata - Class in io.delta.standalone.types
+
+
The metadata for a given StructField.
+
+
FieldMetadata.Builder - Class in io.delta.standalone.types
+
+
Builder class for FieldMetadata.
+
+
FileAction - Interface in io.delta.standalone.actions
+
+
Generic interface for Actions pertaining to the addition and removal of files.
+
+
FloatType - Class in io.delta.standalone.types
+
+
The data type representing float values.
+
+
FloatType() - Constructor for class io.delta.standalone.types.FloatType
+
 
+
Format - Class in io.delta.standalone.actions
+
+
A specification of the encoding for the files stored in a table.
+
+
Format(String, Map<String, String>) - Constructor for class io.delta.standalone.actions.Format
+
 
+
Format() - Constructor for class io.delta.standalone.actions.Format
+
 
+
format(Format) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
forTable(Configuration, String) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provided + path.
+
+
forTable(Configuration, Path) - Static method in interface io.delta.standalone.DeltaLog
+
+
Create a DeltaLog instance representing the table located at the provide + path.
+
+
+ + + +

G

+
+
get(String) - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
get(String) - Method in class io.delta.standalone.types.StructType
+
 
+
getActions() - Method in class io.delta.standalone.VersionLog
+
 
+
getAllFiles() - Method in interface io.delta.standalone.Snapshot
+
 
+
getAppId() - Method in class io.delta.standalone.actions.SetTransaction
+
 
+
getBigDecimal(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.math.BigDecimal.
+
+
getBinary(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as binary (byte array).
+
+
getBoolean(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive boolean.
+
+
getByte(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive byte.
+
+
getCatalogString() - Method in class io.delta.standalone.types.DataType
+
 
+
getChanges(long, boolean) - Method in interface io.delta.standalone.DeltaLog
+
+
Get all actions starting from startVersion (inclusive).
+
+
getChild() - Method in class io.delta.standalone.expressions.UnaryExpression
+
 
+
getClusterId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getCommitInfoAt(long) - Method in interface io.delta.standalone.DeltaLog
+
 
+
getConfiguration() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getCreatedTime() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDataType() - Method in class io.delta.standalone.types.StructField
+
 
+
getDate(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Date.
+
+
getDeletionTimestamp() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
getDescription() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getDouble(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive double.
+
+
getElementType() - Method in class io.delta.standalone.types.ArrayType
+
 
+
getEngineInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getEntries() - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
getFieldNames() - Method in class io.delta.standalone.types.StructType
+
 
+
getFields() - Method in class io.delta.standalone.types.StructType
+
 
+
getFiles() - Method in interface io.delta.standalone.DeltaScan
+
+
Creates a CloseableIterator which can iterate over files belonging to this snapshot.
+
+
getFloat(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive float.
+
+
getFormat() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getId() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getInputPredicate() - Method in interface io.delta.standalone.DeltaScan
+
 
+
getInt(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive int.
+
+
getIsBlindAppend() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getIsolationLevel() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getJobName() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getJobOwnerId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getKeyType() - Method in class io.delta.standalone.types.MapType
+
 
+
getLastUpdated() - Method in class io.delta.standalone.actions.SetTransaction
+
 
+
getLeft() - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
getLength() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getList(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.List<T> object.
+
+
getLong(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive long.
+
+
getMap(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
+
+
getMetadata() - Method in interface io.delta.standalone.Snapshot
+
 
+
getMetadata() - Method in class io.delta.standalone.types.StructField
+
 
+
getMetrics() - Method in class io.delta.standalone.Operation
+
 
+
getMinReaderVersion() - Method in class io.delta.standalone.actions.Protocol
+
 
+
getMinWriterVersion() - Method in class io.delta.standalone.actions.Protocol
+
 
+
getModificationTime() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getName() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getName() - Method in class io.delta.standalone.Operation
+
 
+
getName() - Method in class io.delta.standalone.types.StructField
+
 
+
getNotebookId() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
getNotebookInfo() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperation() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationMetrics() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOperationParameters() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getOptions() - Method in class io.delta.standalone.actions.Format
+
 
+
getParameters() - Method in class io.delta.standalone.Operation
+
 
+
getPartitionColumns() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getPartitionValues() - Method in class io.delta.standalone.actions.AddCDCFile
+
 
+
getPartitionValues() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPartitionValues() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
getPath() - Method in class io.delta.standalone.actions.AddCDCFile
+
 
+
getPath() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getPath() - Method in interface io.delta.standalone.actions.FileAction
+
 
+
getPath() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
getPath() - Method in interface io.delta.standalone.DeltaLog
+
 
+
getPrecision() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getProvider() - Method in class io.delta.standalone.actions.Format
+
 
+
getPushedPredicate() - Method in interface io.delta.standalone.DeltaScan
+
 
+
getReadVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getRecord(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a RowRecord object.
+
+
getResidualPredicate() - Method in interface io.delta.standalone.DeltaScan
+
 
+
getRight() - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
getRunId() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getScale() - Method in class io.delta.standalone.types.DecimalType
+
 
+
getSchema() - Method in class io.delta.standalone.actions.Metadata
+
 
+
getSchema() - Method in interface io.delta.standalone.data.RowRecord
+
 
+
getShort(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a primitive short.
+
+
getSimpleString() - Method in class io.delta.standalone.types.ByteType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.DataType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.IntegerType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.LongType
+
 
+
getSimpleString() - Method in class io.delta.standalone.types.ShortType
+
 
+
getSize() - Method in class io.delta.standalone.actions.AddCDCFile
+
 
+
getSize() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getSize() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
getSnapshotForTimestampAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the latest Snapshot that was generated at or before + timestamp.
+
+
getSnapshotForVersionAsOf(long) - Method in interface io.delta.standalone.DeltaLog
+
+
Travel back in time to the Snapshot with the provided version number.
+
+
getStats() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getString(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a String object.
+
+
getTags() - Method in class io.delta.standalone.actions.AddCDCFile
+
 
+
getTags() - Method in class io.delta.standalone.actions.AddFile
+
 
+
getTags() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
getTimestamp() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getTimestamp(String) - Method in interface io.delta.standalone.data.RowRecord
+
+
Retrieves value from data record and returns the value as a java.sql.Timestamp.
+
+
getTreeString() - Method in class io.delta.standalone.types.StructType
+
 
+
getTriggerType() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
getTypeName() - Method in class io.delta.standalone.types.DataType
+
 
+
getUserId() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserMetadata() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getUserMetadata() - Method in class io.delta.standalone.Operation
+
 
+
getUserName() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getValueType() - Method in class io.delta.standalone.types.MapType
+
 
+
getVersion() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
getVersion() - Method in class io.delta.standalone.actions.SetTransaction
+
 
+
getVersion() - Method in class io.delta.standalone.CommitResult
+
 
+
getVersion() - Method in interface io.delta.standalone.Snapshot
+
 
+
getVersion() - Method in class io.delta.standalone.VersionLog
+
 
+
GreaterThan - Class in io.delta.standalone.expressions
+
+
Usage: new GreaterThan(expr1, expr2) - Returns true if expr1 is greater than + expr2, else false.
+
+
GreaterThan(Expression, Expression) - Constructor for class io.delta.standalone.expressions.GreaterThan
+
 
+
GreaterThanOrEqual - Class in io.delta.standalone.expressions
+
+
Usage: new GreaterThanOrEqual(expr1, expr2) - Returns true if expr1 is greater + than or equal to expr2, else false.
+
+
GreaterThanOrEqual(Expression, Expression) - Constructor for class io.delta.standalone.expressions.GreaterThanOrEqual
+
 
+
+ + + +

H

+
+
hashCode() - Method in class io.delta.standalone.actions.AddFile
+
 
+
hashCode() - Method in class io.delta.standalone.actions.CommitInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Format
+
 
+
hashCode() - Method in class io.delta.standalone.actions.JobInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Metadata
+
 
+
hashCode() - Method in class io.delta.standalone.actions.NotebookInfo
+
 
+
hashCode() - Method in class io.delta.standalone.actions.Protocol
+
 
+
hashCode() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
hashCode() - Method in class io.delta.standalone.expressions.BinaryExpression
+
 
+
hashCode() - Method in class io.delta.standalone.expressions.Column
+
 
+
hashCode() - Method in class io.delta.standalone.expressions.LeafExpression
+
 
+
hashCode() - Method in class io.delta.standalone.expressions.Literal
+
 
+
hashCode() - Method in class io.delta.standalone.expressions.UnaryExpression
+
 
+
hashCode() - Method in class io.delta.standalone.types.ArrayType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DataType
+
 
+
hashCode() - Method in class io.delta.standalone.types.DecimalType
+
 
+
hashCode() - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
hashCode() - Method in class io.delta.standalone.types.MapType
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructField
+
 
+
hashCode() - Method in class io.delta.standalone.types.StructType
+
 
+
+ + + +

I

+
+
id(String) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
In - Class in io.delta.standalone.expressions
+
+
Usage: new In(expr, exprList) - Returns true if expr is equal to any in + exprList, else false.
+
+
In(Expression, List<? extends Expression>) - Constructor for class io.delta.standalone.expressions.In
+
 
+
initHadoopConf() - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
IntegerType - Class in io.delta.standalone.types
+
+
The data type representing int values.
+
+
IntegerType() - Constructor for class io.delta.standalone.types.IntegerType
+
 
+
io.delta.standalone - package io.delta.standalone
+
 
+
io.delta.standalone.actions - package io.delta.standalone.actions
+
 
+
io.delta.standalone.data - package io.delta.standalone.data
+
 
+
io.delta.standalone.exceptions - package io.delta.standalone.exceptions
+
 
+
io.delta.standalone.expressions - package io.delta.standalone.expressions
+
 
+
io.delta.standalone.storage - package io.delta.standalone.storage
+
 
+
io.delta.standalone.types - package io.delta.standalone.types
+
 
+
io.delta.standalone.util - package io.delta.standalone.util
+
 
+
isBlindAppend(Boolean) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
isDataChange() - Method in class io.delta.standalone.actions.AddCDCFile
+
 
+
isDataChange() - Method in class io.delta.standalone.actions.AddFile
+
 
+
isDataChange() - Method in interface io.delta.standalone.actions.FileAction
+
 
+
isDataChange() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
isExtendedFileMetadata() - Method in class io.delta.standalone.actions.RemoveFile
+
 
+
IsNotNull - Class in io.delta.standalone.expressions
+
+
Usage: new IsNotNull(expr) - Returns true if expr is not null, else false.
+
+
IsNotNull(Expression) - Constructor for class io.delta.standalone.expressions.IsNotNull
+
 
+
IsNull - Class in io.delta.standalone.expressions
+
+
Usage: new IsNull(expr) - Returns true if expr is null, else false.
+
+
IsNull(Expression) - Constructor for class io.delta.standalone.expressions.IsNull
+
 
+
isNullable() - Method in class io.delta.standalone.types.StructField
+
 
+
isNullAt(String) - Method in interface io.delta.standalone.data.RowRecord
+
 
+
isolationLevel(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
isPartialWriteVisible(Path, Configuration) - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
+ + + +

J

+
+
jobInfo(JobInfo) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
JobInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Job information that committed to the Delta table.
+
+
JobInfo(String, String, String, String, String) - Constructor for class io.delta.standalone.actions.JobInfo
+
 
+
JobInfo.Builder - Class in io.delta.standalone.actions
+
+
Builder class for JobInfo.
+
+
jobName(String) - Method in class io.delta.standalone.actions.JobInfo.Builder
+
 
+
jobOwnerId(String) - Method in class io.delta.standalone.actions.JobInfo.Builder
+
 
+
+ + + +

L

+
+
LeafExpression - Class in io.delta.standalone.expressions
+
 
+
LeafExpression() - Constructor for class io.delta.standalone.expressions.LeafExpression
+
 
+
length() - Method in class io.delta.standalone.types.StructType
+
 
+
LessThan - Class in io.delta.standalone.expressions
+
+
Usage: new LessThan(expr1, expr2) - Returns true if expr1 is less than + expr2, else false.
+
+
LessThan(Expression, Expression) - Constructor for class io.delta.standalone.expressions.LessThan
+
 
+
LessThanOrEqual - Class in io.delta.standalone.expressions
+
+
Usage: new LessThanOrEqual(expr1, expr2) - Returns true if expr1 is less than or + equal to expr2, else false.
+
+
LessThanOrEqual(Expression, Expression) - Constructor for class io.delta.standalone.expressions.LessThanOrEqual
+
 
+
listFrom(Path, Configuration) - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
Literal - Class in io.delta.standalone.expressions
+
 
+
LogStore - Class in io.delta.standalone.storage
+
+
:: DeveloperApi ::
+
+
LogStore(Configuration) - Constructor for class io.delta.standalone.storage.LogStore
+
 
+
LongType - Class in io.delta.standalone.types
+
+
The data type representing long values.
+
+
LongType() - Constructor for class io.delta.standalone.types.LongType
+
 
+
+ + + +

M

+
+
MapType - Class in io.delta.standalone.types
+
+
The data type for Maps.
+
+
MapType(DataType, DataType, boolean) - Constructor for class io.delta.standalone.types.MapType
+
 
+
markFilesAsRead(Expression) - Method in interface io.delta.standalone.OptimisticTransaction
+
+
Mark files matched by the readPredicates as read by this transaction.
+
+
Metadata - Class in io.delta.standalone.actions
+
+
Updates the metadata of the table.
+
+
Metadata(String, String, String, Format, List<String>, Map<String, String>, Optional<Long>, StructType) - Constructor for class io.delta.standalone.actions.Metadata
+
 
+
metadata() - Method in interface io.delta.standalone.OptimisticTransaction
+
 
+
Metadata.Builder - Class in io.delta.standalone.actions
+
+
Builder class for Metadata.
+
+
MetadataChangedException - Exception in io.delta.standalone.exceptions
+
+
Thrown when the metadata of the Delta table has changed between the time of read + and the time of commit.
+
+
MetadataChangedException(String) - Constructor for exception io.delta.standalone.exceptions.MetadataChangedException
+
 
+
Metrics() - Constructor for class io.delta.standalone.Operation.Metrics
+
 
+
+ + + +

N

+
+
name(String) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
name() - Method in class io.delta.standalone.expressions.Column
+
 
+
Not - Class in io.delta.standalone.expressions
+
+
Usage: new Not(expr) - Logical not.
+
+
Not(Expression) - Constructor for class io.delta.standalone.expressions.Not
+
 
+
notebookInfo(NotebookInfo) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
NotebookInfo - Class in io.delta.standalone.actions
+
+
Represents the Databricks Notebook information that committed to the Delta table.
+
+
NotebookInfo(String) - Constructor for class io.delta.standalone.actions.NotebookInfo
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.And
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.EqualTo
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.GreaterThan
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.GreaterThanOrEqual
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.LessThan
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.LessThanOrEqual
+
 
+
nullSafeEval(Object) - Method in class io.delta.standalone.expressions.Not
+
 
+
nullSafeEval(Object, Object) - Method in class io.delta.standalone.expressions.Or
+
 
+
NullType - Class in io.delta.standalone.types
+
+
The data type representing null values.
+
+
NullType() - Constructor for class io.delta.standalone.types.NullType
+
 
+
numAddedFiles - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of files added.
+
+
numConvertedFiles - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of parquet files that have been converted.
+
+
numCopiedRows - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows copied in the process of deleting files.
+
+
numDeletedRows - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows removed.
+
+
numFiles - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of files written.
+
+
numOutputBytes - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Size in bytes of the written contents.
+
+
numOutputRows - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows written.
+
+
numRemovedFiles - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of files removed.
+
+
numSourceRows - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows in the source table.
+
+
numTargetFilesAdded - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number files added to the sink(target).
+
+
numTargetFilesRemoved - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of files removed from the sink(target).
+
+
numTargetRowsCopied - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of target rows copied.
+
+
numTargetRowsDeleted - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows deleted in the target table.
+
+
numTargetRowsInserted - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows inserted into the target table.
+
+
numTargetRowsUpdated - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows updated in the target table.
+
+
numUpdatedRows - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Number of rows updated.
+
+
+ + + +

O

+
+
of(int) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(boolean) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(byte[]) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(Date) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(BigDecimal) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(double) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(float) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(long) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(short) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(String) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(Timestamp) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
of(byte) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
ofNull(DataType) - Static method in class io.delta.standalone.expressions.Literal
+
 
+
open() - Method in interface io.delta.standalone.Snapshot
+
+
Creates a CloseableIterator which can iterate over data belonging to this snapshot.
+
+
operation(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
Operation - Class in io.delta.standalone
+
+
An operation that can be performed on a Delta table.
+
+
Operation(Operation.Name) - Constructor for class io.delta.standalone.Operation
+
 
+
Operation(Operation.Name, Map<String, String>) - Constructor for class io.delta.standalone.Operation
+
 
+
Operation(Operation.Name, Map<String, String>, Map<String, String>) - Constructor for class io.delta.standalone.Operation
+
 
+
Operation(Operation.Name, Map<String, String>, Map<String, String>, Optional<String>) - Constructor for class io.delta.standalone.Operation
+
 
+
Operation.Metrics - Class in io.delta.standalone
+
+
Some possible operation metrics and their suggested corresponding operation types.
+
+
Operation.Name - Enum in io.delta.standalone
+
+
Supported operation types.
+
+
operationMetrics(Map<String, String>) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
operationParameters(Map<String, String>) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
OptimisticTransaction - Interface in io.delta.standalone
+
+
Used to perform a set of reads in a transaction and then commit a set of updates to the + state of the log.
+
+
Or - Class in io.delta.standalone.expressions
+
+
Usage: new Or(expr1, expr2) - Logical OR
+
+
Or(Expression, Expression) - Constructor for class io.delta.standalone.expressions.Or
+
 
+
+ + + +

P

+
+
ParquetSchemaConverter - Class in io.delta.standalone.util
+
+
Converter class to convert StructType to Parquet MessageType.
+
+
ParquetSchemaConverter() - Constructor for class io.delta.standalone.util.ParquetSchemaConverter
+
 
+
ParquetSchemaConverter.ParquetOutputTimestampType - Enum in io.delta.standalone.util
+
+
Represents Parquet timestamp types.
+
+
partitionColumns(List<String>) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
partitionValues(Map<String, String>) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
Predicate - Interface in io.delta.standalone.expressions
+
 
+
Protocol - Class in io.delta.standalone.actions
+
+
Used to block older clients from reading or writing the log when backwards + incompatible changes are made to the protocol.
+
+
Protocol() - Constructor for class io.delta.standalone.actions.Protocol
+
 
+
Protocol(int, int) - Constructor for class io.delta.standalone.actions.Protocol
+
 
+
ProtocolChangedException - Exception in io.delta.standalone.exceptions
+
+
Thrown when the protocol version has changed between the time of read and the time of commit.
+
+
ProtocolChangedException(String) - Constructor for exception io.delta.standalone.exceptions.ProtocolChangedException
+
 
+
putBoolean(String, boolean) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putBooleanArray(String, Boolean[]) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putDouble(String, double) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putDoubleArray(String, Double[]) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putLong(String, long) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putLongArray(String, Long[]) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putMetadata(String, FieldMetadata) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putMetadataArray(String, FieldMetadata[]) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putNull(String) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putString(String, String) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
putStringArray(String, String[]) - Method in class io.delta.standalone.types.FieldMetadata.Builder
+
 
+
+ + + +

R

+
+
read(Path, Configuration) - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
readerVersion - Static variable in interface io.delta.standalone.actions.Action
+
+
The maximum version of the protocol that this version of Delta Standalone understands.
+
+
readVersion(Long) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
readWholeTable() - Method in interface io.delta.standalone.OptimisticTransaction
+
+
Mark the entire table as tainted (i.e.
+
+
references() - Method in class io.delta.standalone.expressions.Column
+
 
+
references() - Method in interface io.delta.standalone.expressions.Expression
+
 
+
references() - Method in class io.delta.standalone.expressions.LeafExpression
+
 
+
RemoveFile - Class in io.delta.standalone.actions
+
+
Logical removal of a given file from the reservoir.
+
+
RemoveFile(String, Optional<Long>, boolean, boolean, Map<String, String>, long, Map<String, String>) - Constructor for class io.delta.standalone.actions.RemoveFile
+
 
+
RemoveFile.Builder - Class in io.delta.standalone.actions
+
+
Builder class for RemoveFile.
+
+
resolvePathOnPhysicalStorage(Path, Configuration) - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
rewriteTimeMs - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Time taken to rewrite the matched files.
+
+
RowRecord - Interface in io.delta.standalone.data
+
+
Represents one row of data containing a non-empty collection of fieldName - value pairs.
+
+
runId(String) - Method in class io.delta.standalone.actions.JobInfo.Builder
+
 
+
+ + + +

S

+
+
scan() - Method in interface io.delta.standalone.Snapshot
+
 
+
scan(Expression) - Method in interface io.delta.standalone.Snapshot
+
 
+
scanTimeMs - Static variable in class io.delta.standalone.Operation.Metrics
+
+
Time taken to scan the files for matches.
+
+
schema(StructType) - Method in class io.delta.standalone.actions.Metadata.Builder
+
 
+
SetTransaction - Class in io.delta.standalone.actions
+
+
Sets the committed version for a given application.
+
+
SetTransaction(String, long, Optional<Long>) - Constructor for class io.delta.standalone.actions.SetTransaction
+
 
+
ShortType - Class in io.delta.standalone.types
+
+
The data type representing short values.
+
+
ShortType() - Constructor for class io.delta.standalone.types.ShortType
+
 
+
size(long) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
snapshot() - Method in interface io.delta.standalone.DeltaLog
+
 
+
Snapshot - Interface in io.delta.standalone
+
+
Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version.
+
+
sparkToParquet(StructType) - Static method in class io.delta.standalone.util.ParquetSchemaConverter
+
+
Convert a StructType to Parquet MessageType.
+
+
sparkToParquet(StructType, Boolean) - Static method in class io.delta.standalone.util.ParquetSchemaConverter
+
+
Convert a StructType to Parquet MessageType.
+
+
sparkToParquet(StructType, ParquetSchemaConverter.ParquetOutputTimestampType) - Static method in class io.delta.standalone.util.ParquetSchemaConverter
+
+
Convert a StructType to Parquet MessageType.
+
+
sparkToParquet(StructType, Boolean, ParquetSchemaConverter.ParquetOutputTimestampType) - Static method in class io.delta.standalone.util.ParquetSchemaConverter
+
+
Convert a StructType to Parquet MessageType.
+
+
startTransaction() - Method in interface io.delta.standalone.DeltaLog
+
+
Returns a new OptimisticTransaction that can be used to read the current state of the + log and then commit updates.
+
+
stats(String) - Method in class io.delta.standalone.actions.AddFile.Builder
+
 
+
StringType - Class in io.delta.standalone.types
+
+
The data type representing String values.
+
+
StringType() - Constructor for class io.delta.standalone.types.StringType
+
 
+
StructField - Class in io.delta.standalone.types
+
+
A field inside a StructType.
+
+
StructField(String, DataType) - Constructor for class io.delta.standalone.types.StructField
+
+
Constructor with default nullable = true.
+
+
StructField(String, DataType, boolean) - Constructor for class io.delta.standalone.types.StructField
+
 
+
StructField(String, DataType, boolean, FieldMetadata) - Constructor for class io.delta.standalone.types.StructField
+
 
+
StructType - Class in io.delta.standalone.types
+
+
The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
+
+
StructType() - Constructor for class io.delta.standalone.types.StructType
+
 
+
StructType(StructField[]) - Constructor for class io.delta.standalone.types.StructType
+
 
+
+ + + +

T

+
+
tags(Map<String, String>) - Method in class io.delta.standalone.actions.AddFile.Builder
+
 
+
tags(Map<String, String>) - Method in class io.delta.standalone.actions.RemoveFile.Builder
+
 
+
timestamp(Timestamp) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
TimestampType - Class in io.delta.standalone.types
+
+
The data type representing java.sql.Timestamp values.
+
+
TimestampType() - Constructor for class io.delta.standalone.types.TimestampType
+
 
+
toJson() - Method in class io.delta.standalone.types.DataType
+
 
+
toPrettyJson() - Method in class io.delta.standalone.types.DataType
+
 
+
toString() - Method in class io.delta.standalone.expressions.BinaryOperator
+
 
+
toString() - Method in class io.delta.standalone.expressions.Column
+
 
+
toString() - Method in interface io.delta.standalone.expressions.Expression
+
 
+
toString() - Method in class io.delta.standalone.expressions.In
+
 
+
toString() - Method in class io.delta.standalone.expressions.IsNotNull
+
 
+
toString() - Method in class io.delta.standalone.expressions.IsNull
+
 
+
toString() - Method in class io.delta.standalone.expressions.Literal
+
 
+
toString() - Method in class io.delta.standalone.expressions.Not
+
 
+
toString() - Method in enum io.delta.standalone.Operation.Name
+
 
+
toString() - Method in class io.delta.standalone.types.FieldMetadata
+
 
+
triggerType(String) - Method in class io.delta.standalone.actions.JobInfo.Builder
+
 
+
True - Static variable in class io.delta.standalone.expressions.Literal
+
 
+
txnVersion(String) - Method in interface io.delta.standalone.OptimisticTransaction
+
 
+
+ + + +

U

+
+
UnaryExpression - Class in io.delta.standalone.expressions
+
+
An expression with one input and one output.
+
+
UnaryExpression(Expression) - Constructor for class io.delta.standalone.expressions.UnaryExpression
+
 
+
update() - Method in interface io.delta.standalone.DeltaLog
+
+
Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
+
+
updateMetadata(Metadata) - Method in interface io.delta.standalone.OptimisticTransaction
+
+
Records an update to the metadata that should be committed with this transaction.
+
+
USER_DEFAULT - Static variable in class io.delta.standalone.types.DecimalType
+
 
+
userId(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
userMetadata(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
userName(String) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
+ + + +

V

+
+
value() - Method in class io.delta.standalone.expressions.Literal
+
 
+
valueContainsNull() - Method in class io.delta.standalone.types.MapType
+
 
+
valueOf(String) - Static method in enum io.delta.standalone.Operation.Name
+
+
Returns the enum constant of this type with the specified name.
+
+
valueOf(String) - Static method in enum io.delta.standalone.util.ParquetSchemaConverter.ParquetOutputTimestampType
+
+
Returns the enum constant of this type with the specified name.
+
+
values() - Static method in enum io.delta.standalone.Operation.Name
+
+
Returns an array containing the constants of this enum type, in +the order they are declared.
+
+
values() - Static method in enum io.delta.standalone.util.ParquetSchemaConverter.ParquetOutputTimestampType
+
+
Returns an array containing the constants of this enum type, in +the order they are declared.
+
+
version(Long) - Method in class io.delta.standalone.actions.CommitInfo.Builder
+
 
+
VersionLog - Class in io.delta.standalone
+
+
VersionLog is the representation of all actions (changes) to the Delta Table + at a specific table version.
+
+
VersionLog(long, List<Action>) - Constructor for class io.delta.standalone.VersionLog
+
 
+
+ + + +

W

+
+
write(Path, Iterator<String>, Boolean, Configuration) - Method in class io.delta.standalone.storage.LogStore
+
+
:: DeveloperApi ::
+
+
writerVersion - Static variable in interface io.delta.standalone.actions.Action
+
 
+
+A B C D E F G H I J L M N O P R S T U V W 
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html new file mode 100644 index 00000000000..9858e9a615f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html @@ -0,0 +1,75 @@ + + + + + +Delta Standalone Reader 0.2.1 JavaDoc + + + + + + + + + +<noscript> +<div>JavaScript is disabled on your browser.</div> +</noscript> +<h2>Frame Alert</h2> +<p>This document is designed to be viewed using the frames feature. If you see this message, you are using a non-frame-capable web client. Link to <a href="overview-summary.html">Non-frame version</a>.</p> + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/CommitResult.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/CommitResult.html new file mode 100644 index 00000000000..cb946550b33 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/CommitResult.html @@ -0,0 +1,270 @@ + + + + + +CommitResult (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Class CommitResult

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.CommitResult
    • +
    +
  • +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      CommitResult(long version) 
      +
    • +
    + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + +
      All Methods Instance Methods Concrete Methods 
      Modifier and TypeMethod and Description
      longgetVersion() 
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        CommitResult

        +
        public CommitResult(long version)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getVersion

        +
        public long getVersion()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html new file mode 100644 index 00000000000..808c8b49c67 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html @@ -0,0 +1,423 @@ + + + + + +DeltaLog (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface DeltaLog

+
+
+
+
    +
  • +
    +
    +
    public interface DeltaLog
    +
    DeltaLog is the representation of the transaction logs of a Delta table. It provides APIs + to access the states of a Delta table. +

    + You can use the following codes to create a DeltaLog instance. +

    
    +   Configuration conf = ... // Create your own Hadoop Configuration instance
    +   DeltaLog deltaLog = DeltaLog.forTable(conf, "/the/delta/table/path");
    + 
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        update

        +
        Snapshot update()
        +
        Bring DeltaLog's current Snapshot to the latest state if there are any new + transaction logs.
        +
      • +
      + + + +
        +
      • +

        getSnapshotForVersionAsOf

        +
        Snapshot getSnapshotForVersionAsOf(long version)
        +
        Travel back in time to the Snapshot with the provided version number.
        +
        +
        Parameters:
        +
        version - the snapshot version to generate
        +
        Throws:
        +
        IllegalArgumentException - if the version is outside the range of available + versions
        +
        +
      • +
      + + + +
        +
      • +

        getSnapshotForTimestampAsOf

        +
        Snapshot getSnapshotForTimestampAsOf(long timestamp)
        +
        Travel back in time to the latest Snapshot that was generated at or before + timestamp.
        +
        +
        Parameters:
        +
        timestamp - the number of milliseconds since midnight, January 1, 1970 UTC
        +
        Throws:
        +
        RuntimeException - if the snapshot is unable to be recreated
        +
        IllegalArgumentException - if the timestamp is before the earliest possible + snapshot or after the latest possible snapshot
        +
        +
      • +
      + + + +
        +
      • +

        startTransaction

        +
        OptimisticTransaction startTransaction()
        +
        Returns a new OptimisticTransaction that can be used to read the current state of the + log and then commit updates. The reads and updates will be checked for logical conflicts + with any concurrent writes to the log. +

        + Note that all reads in a transaction must go through the returned transaction object, and not + directly to the DeltaLog otherwise they will not be checked for conflicts.

        +
      • +
      + + + +
        +
      • +

        getCommitInfoAt

        +
        CommitInfo getCommitInfoAt(long version)
        +
        +
        Parameters:
        +
        version - the commit version to retrieve CommitInfo
        +
        +
      • +
      + + + +
        +
      • +

        getPath

        +
        org.apache.hadoop.fs.Path getPath()
        +
      • +
      + + + +
        +
      • +

        getChanges

        +
        java.util.Iterator<VersionLog> getChanges(long startVersion,
        +                                          boolean failOnDataLoss)
        +
        Get all actions starting from startVersion (inclusive). + If startVersion doesn't exist, return an empty Iterator.
        +
        +
        Parameters:
        +
        startVersion - the table version to begin retrieving actions from (inclusive)
        +
        failOnDataLoss - whether to throw when data loss detected
        +
        Throws:
        +
        IllegalArgumentException - if startVersion is negative
        +
        IllegalStateException - if data loss detected and failOnDataLoss is true
        +
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         String path)
        +
        Create a DeltaLog instance representing the table located at the provided + path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      + + + +
        +
      • +

        forTable

        +
        static DeltaLog forTable(org.apache.hadoop.conf.Configuration hadoopConf,
        +                         org.apache.hadoop.fs.Path path)
        +
        Create a DeltaLog instance representing the table located at the provide + path.
        +
        +
        Parameters:
        +
        hadoopConf - Hadoop Configuration to use when accessing the Delta table
        +
        path - the path to the Delta table
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html new file mode 100644 index 00000000000..f4f615ead6f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html @@ -0,0 +1,269 @@ + + + + + +DeltaScan (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface DeltaScan

+
+
+
+
    +
  • +
    +
    +
    public interface DeltaScan
    +
    Provides access to an iterator over the files in this snapshot. +

    + Typically created with a read predicate Expression to let users filter files.

    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getFiles

        +
        CloseableIterator<AddFile> getFiles()
        +
        Creates a CloseableIterator which can iterate over files belonging to this snapshot. +

        + It provides no iteration ordering guarantee among files. +

        + Files returned are guaranteed to satisfy the predicate, if any, returned by + getPushedPredicate().

        +
      • +
      + + + +
        +
      • +

        getInputPredicate

        +
        java.util.Optional<Expression> getInputPredicate()
        +
      • +
      + + + +
        +
      • +

        getPushedPredicate

        +
        java.util.Optional<Expression> getPushedPredicate()
        +
      • +
      + + + +
        +
      • +

        getResidualPredicate

        +
        java.util.Optional<Expression> getResidualPredicate()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html new file mode 100644 index 00000000000..bcc63653599 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html @@ -0,0 +1,683 @@ + + + + + +Operation.Metrics (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Class Operation.Metrics

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.Operation.Metrics
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    Operation
    +
    +
    +
    +
    public static class Operation.Metrics
    +extends Object
    +
    Some possible operation metrics and their suggested corresponding operation types. + These are purely exemplary, and users may user whichever metrics best fit their application.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Field Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      Fields 
      Modifier and TypeField and Description
      static StringexecutionTimeMs +
      Time taken to execute the entire operation.
      +
      static StringnumAddedFiles +
      Number of files added.
      +
      static StringnumConvertedFiles +
      Number of parquet files that have been converted.
      +
      static StringnumCopiedRows +
      Number of rows copied in the process of deleting files.
      +
      static StringnumDeletedRows +
      Number of rows removed.
      +
      static StringnumFiles +
      Number of files written.
      +
      static StringnumOutputBytes +
      Size in bytes of the written contents.
      +
      static StringnumOutputRows +
      Number of rows written.
      +
      static StringnumRemovedFiles +
      Number of files removed.
      +
      static StringnumSourceRows +
      Number of rows in the source table.
      +
      static StringnumTargetFilesAdded +
      Number files added to the sink(target).
      +
      static StringnumTargetFilesRemoved +
      Number of files removed from the sink(target).
      +
      static StringnumTargetRowsCopied +
      Number of target rows copied.
      +
      static StringnumTargetRowsDeleted +
      Number of rows deleted in the target table.
      +
      static StringnumTargetRowsInserted +
      Number of rows inserted into the target table.
      +
      static StringnumTargetRowsUpdated +
      Number of rows updated in the target table.
      +
      static StringnumUpdatedRows +
      Number of rows updated.
      +
      static StringrewriteTimeMs +
      Time taken to rewrite the matched files.
      +
      static StringscanTimeMs +
      Time taken to scan the files for matches.
      +
      +
    • +
    + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      Metrics() 
      +
    • +
    + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        numFiles

        +
        public static final String numFiles
        +
        Number of files written. + + Usually used with WRITE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numOutputBytes

        +
        public static final String numOutputBytes
        +
        Size in bytes of the written contents. + + Usually used with WRITE, STREAMING_UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numOutputRows

        +
        public static final String numOutputRows
        +
        Number of rows written. + + Usually used with WRITE, STREAMING_UPDATE, MERGE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numAddedFiles

        +
        public static final String numAddedFiles
        +
        Number of files added. + + Usually used with STREAMING_UPDATE, DELETE, UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numRemovedFiles

        +
        public static final String numRemovedFiles
        +
        Number of files removed. + + Usually used with STREAMING_UPDATE, DELETE, DELETE_PARTITIONS, TRUNCATE, + UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numDeletedRows

        +
        public static final String numDeletedRows
        +
        Number of rows removed. + + Usually used with DELETE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numCopiedRows

        +
        public static final String numCopiedRows
        +
        Number of rows copied in the process of deleting files. + + Usually used with DELETE, UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        executionTimeMs

        +
        public static final String executionTimeMs
        +
        Time taken to execute the entire operation. + + Usually used with DELETE, DELETE_PARTITIONS, TRUNCATE, MERGE, UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        scanTimeMs

        +
        public static final String scanTimeMs
        +
        Time taken to scan the files for matches. + + Usually used with DELETE, DELETE_PARTITIONS, MERGE, UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        rewriteTimeMs

        +
        public static final String rewriteTimeMs
        +
        Time taken to rewrite the matched files. + + Usually used with DELETE, DELETE_PARTITIONS, MERGE, UPDATE operations.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numConvertedFiles

        +
        public static final String numConvertedFiles
        +
        Number of parquet files that have been converted. + + Usually used with the CONVERT operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numSourceRows

        +
        public static final String numSourceRows
        +
        Number of rows in the source table. + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetRowsInserted

        +
        public static final String numTargetRowsInserted
        +
        Number of rows inserted into the target table. + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetRowsUpdated

        +
        public static final String numTargetRowsUpdated
        +
        Number of rows updated in the target table. + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetRowsDeleted

        +
        public static final String numTargetRowsDeleted
        +
        Number of rows deleted in the target table. + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetRowsCopied

        +
        public static final String numTargetRowsCopied
        +
        Number of target rows copied. + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetFilesAdded

        +
        public static final String numTargetFilesAdded
        +
        Number files added to the sink(target). + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numTargetFilesRemoved

        +
        public static final String numTargetFilesRemoved
        +
        Number of files removed from the sink(target). + + Usually used with the MERGE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + +
        +
      • +

        numUpdatedRows

        +
        public static final String numUpdatedRows
        +
        Number of rows updated. + + Usually used with the UPDATE operation.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Metrics

        +
        public Metrics()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html new file mode 100644 index 00000000000..12a380a2ab6 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html @@ -0,0 +1,583 @@ + + + + + +Operation.Name (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Enum Operation.Name

+
+
+
    +
  • Object
  • +
  • + +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable, Comparable<Operation.Name>
    +
    +
    +
    Enclosing class:
    +
    Operation
    +
    +
    +
    +
    public static enum Operation.Name
    +extends Enum<Operation.Name>
    +
    Supported operation types.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Enum Constant Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      Enum Constants 
      Enum Constant and Description
      ADD_COLUMNS +
      Recorded when columns are added.
      +
      CHANGE_COLUMN +
      Recorded when columns are changed.
      +
      CONVERT +
      Recorded when converting a table into a Delta table.
      +
      CREATE_TABLE +
      Recorded when the table is created.
      +
      DELETE +
      Recorded while deleting certain partitions.
      +
      MANUAL_UPDATE 
      MERGE +
      Recorded when a merge operation is committed to the table.
      +
      REPLACE_COLUMNS +
      Recorded when columns are replaced.
      +
      REPLACE_TABLE +
      Recorded when the table is replaced.
      +
      SET_TABLE_PROPERTIES +
      Recorded when the table properties are set.
      +
      STREAMING_UPDATE +
      Recorded during streaming inserts.
      +
      TRUNCATE +
      Recorded when truncating the table.
      +
      UNSET_TABLE_PROPERTIES +
      Recorded when the table properties are unset.
      +
      UPDATE +
      Recorded when an update operation is committed to the table.
      +
      UPGRADE_PROTOCOL +
      Recorded when the table protocol is upgraded.
      +
      UPGRADE_SCHEMA +
      Recorded when the table schema is upgraded.
      +
      WRITE +
      Recorded during batch inserts.
      +
      +
    • +
    + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + +
      All Methods Static Methods Instance Methods Concrete Methods 
      Modifier and TypeMethod and Description
      StringtoString() 
      static Operation.NamevalueOf(String name) +
      Returns the enum constant of this type with the specified name.
      +
      static Operation.Name[]values() +
      Returns an array containing the constants of this enum type, in +the order they are declared.
      +
      +
        +
      • + + +

        Methods inherited from class Enum

        +compareTo, equals, getDeclaringClass, hashCode, name, ordinal, valueOf
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +getClass, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Enum Constant Detail

      + + + +
        +
      • +

        WRITE

        +
        public static final Operation.Name WRITE
        +
        Recorded during batch inserts.
        +
      • +
      + + + +
        +
      • +

        STREAMING_UPDATE

        +
        public static final Operation.Name STREAMING_UPDATE
        +
        Recorded during streaming inserts.
        +
      • +
      + + + +
        +
      • +

        DELETE

        +
        public static final Operation.Name DELETE
        +
        Recorded while deleting certain partitions.
        +
      • +
      + + + +
        +
      • +

        TRUNCATE

        +
        public static final Operation.Name TRUNCATE
        +
        Recorded when truncating the table.
        +
      • +
      + + + +
        +
      • +

        CONVERT

        +
        public static final Operation.Name CONVERT
        +
        Recorded when converting a table into a Delta table.
        +
      • +
      + + + +
        +
      • +

        MERGE

        +
        public static final Operation.Name MERGE
        +
        Recorded when a merge operation is committed to the table.
        +
      • +
      + + + +
        +
      • +

        UPDATE

        +
        public static final Operation.Name UPDATE
        +
        Recorded when an update operation is committed to the table.
        +
      • +
      + + + +
        +
      • +

        CREATE_TABLE

        +
        public static final Operation.Name CREATE_TABLE
        +
        Recorded when the table is created.
        +
      • +
      + + + +
        +
      • +

        REPLACE_TABLE

        +
        public static final Operation.Name REPLACE_TABLE
        +
        Recorded when the table is replaced.
        +
      • +
      + + + +
        +
      • +

        SET_TABLE_PROPERTIES

        +
        public static final Operation.Name SET_TABLE_PROPERTIES
        +
        Recorded when the table properties are set.
        +
      • +
      + + + +
        +
      • +

        UNSET_TABLE_PROPERTIES

        +
        public static final Operation.Name UNSET_TABLE_PROPERTIES
        +
        Recorded when the table properties are unset.
        +
      • +
      + + + +
        +
      • +

        ADD_COLUMNS

        +
        public static final Operation.Name ADD_COLUMNS
        +
        Recorded when columns are added.
        +
      • +
      + + + +
        +
      • +

        CHANGE_COLUMN

        +
        public static final Operation.Name CHANGE_COLUMN
        +
        Recorded when columns are changed.
        +
      • +
      + + + +
        +
      • +

        REPLACE_COLUMNS

        +
        public static final Operation.Name REPLACE_COLUMNS
        +
        Recorded when columns are replaced.
        +
      • +
      + + + +
        +
      • +

        UPGRADE_PROTOCOL

        +
        public static final Operation.Name UPGRADE_PROTOCOL
        +
        Recorded when the table protocol is upgraded.
        +
      • +
      + + + +
        +
      • +

        UPGRADE_SCHEMA

        +
        public static final Operation.Name UPGRADE_SCHEMA
        +
        Recorded when the table schema is upgraded.
        +
      • +
      + + + +
        +
      • +

        MANUAL_UPDATE

        +
        public static final Operation.Name MANUAL_UPDATE
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        values

        +
        public static Operation.Name[] values()
        +
        Returns an array containing the constants of this enum type, in +the order they are declared. This method may be used to iterate +over the constants as follows: +
        +for (Operation.Name c : Operation.Name.values())
        +    System.out.println(c);
        +
        +
      • +
      + + + +
        +
      • +

        valueOf

        +
        public static Operation.Name valueOf(String name)
        +
        Returns the enum constant of this type with the specified name. +The string must match exactly an identifier used to declare an +enum constant in this type. (Extraneous whitespace characters are +not permitted.)
        +
        +
        Parameters:
        +
        name - the name of the enum constant to be returned.
        +
        Throws:
        +
        IllegalArgumentException - if this enum type has no constant with the specified name
        +
        NullPointerException - if the argument is null
        +
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Overrides:
        +
        toString in class Enum<Operation.Name>
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html new file mode 100644 index 00000000000..e61ea8948e8 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html @@ -0,0 +1,426 @@ + + + + + +Operation (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Class Operation

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.Operation
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Operation
    +extends Object
    +
    An operation that can be performed on a Delta table. +

    + An operation is tracked as the first line in delta logs, and powers DESCRIBE HISTORY for + Delta tables. +

    + Operations must be constructed using one of the Operation.Name types below. + As well, optional Operation.Metrics values are given below.

    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Nested Class Summary

      + + + + + + + + + + + + + + +
      Nested Classes 
      Modifier and TypeClass and Description
      static class Operation.Metrics +
      Some possible operation metrics and their suggested corresponding operation types.
      +
      static class Operation.Name +
      Supported operation types.
      +
      +
    • +
    + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + + + + + + + + + + +
      Constructors 
      Constructor and Description
      Operation(Operation.Name name) 
      Operation(Operation.Name name, + java.util.Map<String,String> parameters) 
      Operation(Operation.Name name, + java.util.Map<String,String> parameters, + java.util.Map<String,String> metrics) 
      Operation(Operation.Name name, + java.util.Map<String,String> parameters, + java.util.Map<String,String> metrics, + java.util.Optional<String> userMetadata) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + + + + + +
        +
      • +

        Operation

        +
        public Operation(@Nonnull
        +                 Operation.Name name,
        +                 @Nullable
        +                 java.util.Map<String,String> parameters)
        +
        +
        Parameters:
        +
        name - The Operation.Name of the operation.
        +
        parameters - Any relevant operation parameters, where values are JSON-encoded.
        +
        +
      • +
      + + + +
        +
      • +

        Operation

        +
        public Operation(@Nonnull
        +                 Operation.Name name,
        +                 @Nullable
        +                 java.util.Map<String,String> parameters,
        +                 @Nullable
        +                 java.util.Map<String,String> metrics)
        +
        +
        Parameters:
        +
        name - The Operation.Name of the operation.
        +
        parameters - Any relevant operation parameters, where values are JSON-encoded.
        +
        metrics - Any relevant operation metrics. See Operation.Metrics for suggested keys.
        +
        +
      • +
      + + + +
        +
      • +

        Operation

        +
        public Operation(@Nonnull
        +                 Operation.Name name,
        +                 @Nullable
        +                 java.util.Map<String,String> parameters,
        +                 @Nullable
        +                 java.util.Map<String,String> metrics,
        +                 @Nonnull
        +                 java.util.Optional<String> userMetadata)
        +
        +
        Parameters:
        +
        name - The Operation.Name of the operation.
        +
        parameters - Any relevant operation parameters, where values are JSON-encoded.
        +
        metrics - Any relevant operation metrics. See Operation.Metrics for suggested keys.
        +
        userMetadata - Optional additional user metadata.
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getParameters

        +
        @Nullable
        +public java.util.Map<String,String> getParameters()
        +
      • +
      + + + +
        +
      • +

        getMetrics

        +
        @Nullable
        +public java.util.Map<String,String> getMetrics()
        +
      • +
      + + + +
        +
      • +

        getUserMetadata

        +
        @Nonnull
        +public java.util.Optional<String> getUserMetadata()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html new file mode 100644 index 00000000000..b8f7db626c1 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html @@ -0,0 +1,353 @@ + + + + + +OptimisticTransaction (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface OptimisticTransaction

+
+
+
+
    +
  • +
    +
    +
    public interface OptimisticTransaction
    +
    Used to perform a set of reads in a transaction and then commit a set of updates to the + state of the log. All reads from the DeltaLog, MUST go through this instance rather + than directly to the DeltaLog otherwise they will not be checked for logical conflicts + with concurrent updates. +

    + This class is not thread-safe.

    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        commit

        +
        <T extends ActionCommitResult commit(Iterable<T> actions,
        +                                       Operation op,
        +                                       String engineInfo)
        +
        Modifies the state of the log by adding a new commit that is based on a read at + the given `lastVersion`. In the case of a conflict with a concurrent writer this + method will throw an exception.
        +
        +
        Type Parameters:
        +
        T - A derived class of Action. This allows, for example, both a + List<Action> and a List<AddFile> to be accepted.
        +
        Parameters:
        +
        actions - Set of actions to commit.
        +
        op - Details of operation that is performing this transactional commit.
        +
        engineInfo - String used to identify the writer engine. It should resemble + "{engineName}/{engineVersion}", with dashes in place of whitespace. + For example, "Flink-Connector/1.1.0".
        +
        +
      • +
      + + + +
        +
      • +

        markFilesAsRead

        +
        DeltaScan markFilesAsRead(Expression readPredicate)
        +
        Mark files matched by the readPredicates as read by this transaction. +

        + Internally, the readPredicates parameter and the resultant readFiles will be + used to determine if logical conflicts between this transaction and previously-committed + transactions can be resolved (i.e. no error thrown). +

        + For example: +

          +
        • This transaction TXN1 reads partition 'date=2021-09-08' to perform an UPDATE and tries + to commit at the next table version N.
        • +
        • After TXN1 starts, another transaction TXN2 reads partition 'date=2021-09-07' and + commits first at table version N (with no other metadata changes).
        • +
        • TXN1 sees that another commit won, and needs to know whether to commit at version N+1 + or fail. Using the readPredicates and resultant readFiles, TXN1 can see + that none of its read files were changed by TXN2. Thus there are no logical conflicts and + TXN1 can commit at table version N+1.
        • +
        +
        +
        Parameters:
        +
        readPredicate - Predicate used to determine which files were read.
        +
        +
      • +
      + + + +
        +
      • +

        updateMetadata

        +
        void updateMetadata(Metadata metadata)
        +
        Records an update to the metadata that should be committed with this transaction. + Note that this must be done before writing out any files so that file writing + and checks happen with the final metadata for the table. +

        + IMPORTANT: It is the responsibility of the caller to ensure that files currently + present in the table are still valid under the new metadata.

        +
        +
        Parameters:
        +
        metadata - The new metadata for the delta table.
        +
        +
      • +
      + + + +
        +
      • +

        readWholeTable

        +
        void readWholeTable()
        +
        Mark the entire table as tainted (i.e. read) by this transaction.
        +
      • +
      + + + +
        +
      • +

        txnVersion

        +
        long txnVersion(String id)
        +
        +
        Parameters:
        +
        id - transaction id
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html new file mode 100644 index 00000000000..2d47ccd23a1 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html @@ -0,0 +1,297 @@ + + + + + +Snapshot (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Interface Snapshot

+
+
+
+
    +
  • +
    +
    +
    public interface Snapshot
    +
    Snapshot provides APIs to access the Delta table state (such as table metadata, active + files) at some version. +

    + See Delta Transaction Log Protocol + for more details about the transaction logs.

    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        scan

        +
        DeltaScan scan(Expression predicate)
        +
        +
        Parameters:
        +
        predicate - the predicate to be used to filter the files in this snapshot.
        +
        +
      • +
      + + + +
        +
      • +

        getAllFiles

        +
        java.util.List<AddFile> getAllFiles()
        +
      • +
      + + + +
        +
      • +

        getMetadata

        +
        Metadata getMetadata()
        +
      • +
      + + + +
        +
      • +

        getVersion

        +
        long getVersion()
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html new file mode 100644 index 00000000000..d09fe6d1875 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html @@ -0,0 +1,288 @@ + + + + + +VersionLog (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone
+

Class VersionLog

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.VersionLog
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class VersionLog
    +extends Object
    +
    VersionLog is the representation of all actions (changes) to the Delta Table + at a specific table version.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      VersionLog(long version, + java.util.List<Action> actions) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        VersionLog

        +
        public VersionLog(long version,
        +                  @Nonnull
        +                  java.util.List<Action> actions)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getVersion

        +
        public long getVersion()
        +
      • +
      + + + +
        +
      • +

        getActions

        +
        @Nonnull
        +public java.util.List<Action> getActions()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html new file mode 100644 index 00000000000..a5b87d2f29c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html @@ -0,0 +1,262 @@ + + + + + +Action (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Interface Action

+
+
+
+
    +
  • +
    +
    All Known Subinterfaces:
    +
    FileAction
    +
    +
    +
    All Known Implementing Classes:
    +
    AddCDCFile, AddFile, CommitInfo, Metadata, Protocol, RemoveFile, SetTransaction
    +
    +
    +
    +
    public interface Action
    +
    A marker interface for all Actions that can be applied to a Delta Table. + Each action represents a single change to the state of a Delta table. + + You can use the following code to extract the concrete type of an Action. +
    
    +   // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
    +   List<Action> actions = ...
    +   actions.forEach(x -> {
    +       if (x instanceof AddFile) {
    +          AddFile addFile = (AddFile) x;
    +          ...
    +       } else if (x instanceof AddCDCFile) {
    +          AddCDCFile addCDCFile = (AddCDCFile)x;
    +          ...
    +       } else if ...
    +   });
    + 
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Field Summary

      + + + + + + + + + + + + + + +
      Fields 
      Modifier and TypeField and Description
      static intreaderVersion +
      The maximum version of the protocol that this version of Delta Standalone understands.
      +
      static intwriterVersion 
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        readerVersion

        +
        static final int readerVersion
        +
        The maximum version of the protocol that this version of Delta Standalone understands.
        +
        +
        See Also:
        +
        Constant Field Values
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html new file mode 100644 index 00000000000..a27d76e3d17 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html @@ -0,0 +1,368 @@ + + + + + +AddCDCFile (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class AddCDCFile

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.AddCDCFile
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action, FileAction
    +
    +
    +
    +
    public final class AddCDCFile
    +extends Object
    +implements FileAction
    +
    A change file containing CDC data for the Delta version it's within. Non-CDC readers should + ignore this, CDC readers should scan all ChangeFiles in a version rather than computing + changes from AddFile and RemoveFile actions.
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      AddCDCFile(String path, + java.util.Map<String,String> partitionValues, + long size, + java.util.Map<String,String> tags) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        AddCDCFile

        +
        public AddCDCFile(@Nonnull
        +                  String path,
        +                  @Nonnull
        +                  java.util.Map<String,String> partitionValues,
        +                  long size,
        +                  @Nullable
        +                  java.util.Map<String,String> tags)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        @Nonnull
        +public String getPath()
        +
        +
        Specified by:
        +
        getPath in interface FileAction
        +
        +
      • +
      + + + + + + + +
        +
      • +

        getSize

        +
        public long getSize()
        +
      • +
      + + + +
        +
      • +

        getTags

        +
        @Nullable
        +public java.util.Map<String,String> getTags()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        public boolean isDataChange()
        +
        +
        Specified by:
        +
        isDataChange in interface FileAction
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html new file mode 100644 index 00000000000..07cc85451e7 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html @@ -0,0 +1,308 @@ + + + + + +AddFile.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class AddFile.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.AddFile.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    AddFile
    +
    +
    +
    +
    public static class AddFile.Builder
    +extends Object
    +
    Builder class for AddFile. Enables construction of AddFile object with default values.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      Builder(String path, + java.util.Map<String,String> partitionValues, + long size, + long modificationTime, + boolean dataChange) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Builder

        +
        public Builder(String path,
        +               java.util.Map<String,String> partitionValues,
        +               long size,
        +               long modificationTime,
        +               boolean dataChange)
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html new file mode 100644 index 00000000000..bb52032f472 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html @@ -0,0 +1,484 @@ + + + + + +AddFile (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class AddFile

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.AddFile
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action, FileAction
    +
    +
    +
    +
    public final class AddFile
    +extends Object
    +implements FileAction
    +
    Represents an action that adds a new file to the table. The path of a file acts as the primary + key for the entry in the set of files. + + Note: since actions within a given Delta file are not guaranteed to be applied in order, it is + not valid for multiple file operations with the same path to exist in a single version.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Nested Class Summary

      + + + + + + + + + + +
      Nested Classes 
      Modifier and TypeClass and Description
      static class AddFile.Builder +
      Builder class for AddFile.
      +
      +
    • +
    + + + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      AddFile(String path, + java.util.Map<String,String> partitionValues, + long size, + long modificationTime, + boolean dataChange, + String stats, + java.util.Map<String,String> tags) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        AddFile

        +
        public AddFile(@Nonnull
        +               String path,
        +               @Nonnull
        +               java.util.Map<String,String> partitionValues,
        +               long size,
        +               long modificationTime,
        +               boolean dataChange,
        +               @Nullable
        +               String stats,
        +               @Nullable
        +               java.util.Map<String,String> tags)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        @Nonnull
        +public String getPath()
        +
        +
        Specified by:
        +
        getPath in interface FileAction
        +
        +
      • +
      + + + + + + + +
        +
      • +

        getSize

        +
        public long getSize()
        +
      • +
      + + + +
        +
      • +

        getModificationTime

        +
        public long getModificationTime()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        public boolean isDataChange()
        +
        +
        Specified by:
        +
        isDataChange in interface FileAction
        +
        +
      • +
      + + + +
        +
      • +

        getStats

        +
        @Nullable
        +public String getStats()
        +
      • +
      + + + +
        +
      • +

        getTags

        +
        @Nullable
        +public java.util.Map<String,String> getTags()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + +
        +
      • +

        builder

        +
        public static AddFile.Builder builder(String path,
        +                                      java.util.Map<String,String> partitionValues,
        +                                      long size,
        +                                      long modificationTime,
        +                                      boolean dataChange)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html new file mode 100644 index 00000000000..0b3fd6c163e --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html @@ -0,0 +1,469 @@ + + + + + +CommitInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class CommitInfo.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.CommitInfo.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    CommitInfo
    +
    +
    +
    +
    public static class CommitInfo.Builder
    +extends Object
    +
    Builder class for CommitInfo. Enables construction of CommitInfo object with default values.
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html new file mode 100644 index 00000000000..6bd6c8fb8ef --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html @@ -0,0 +1,612 @@ + + + + + +CommitInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class CommitInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.CommitInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action
    +
    +
    +
    +
    public class CommitInfo
    +extends Object
    +implements Action
    +
    Holds provenance information about changes to the table. This CommitInfo + is not stored in the checkpoint and has reduced compatibility guarantees. + Information stored in it is best effort (i.e. can be falsified by a writer).
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Nested Class Summary

      + + + + + + + + + + +
      Nested Classes 
      Modifier and TypeClass and Description
      static class CommitInfo.Builder +
      Builder class for CommitInfo.
      +
      +
    • +
    + + + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + + + + +
      Constructors 
      Constructor and Description
      CommitInfo(java.util.Optional<Long> version, + java.sql.Timestamp timestamp, + java.util.Optional<String> userId, + java.util.Optional<String> userName, + String operation, + java.util.Map<String,String> operationParameters, + java.util.Optional<JobInfo> jobInfo, + java.util.Optional<NotebookInfo> notebookInfo, + java.util.Optional<String> clusterId, + java.util.Optional<Long> readVersion, + java.util.Optional<String> isolationLevel, + java.util.Optional<Boolean> isBlindAppend, + java.util.Optional<java.util.Map<String,String>> operationMetrics, + java.util.Optional<String> userMetadata) 
      CommitInfo(java.util.Optional<Long> version, + java.sql.Timestamp timestamp, + java.util.Optional<String> userId, + java.util.Optional<String> userName, + String operation, + java.util.Map<String,String> operationParameters, + java.util.Optional<JobInfo> jobInfo, + java.util.Optional<NotebookInfo> notebookInfo, + java.util.Optional<String> clusterId, + java.util.Optional<Long> readVersion, + java.util.Optional<String> isolationLevel, + java.util.Optional<Boolean> isBlindAppend, + java.util.Optional<java.util.Map<String,String>> operationMetrics, + java.util.Optional<String> userMetadata, + java.util.Optional<String> engineInfo) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        CommitInfo

        +
        public CommitInfo(java.util.Optional<Long> version,
        +                  java.sql.Timestamp timestamp,
        +                  java.util.Optional<String> userId,
        +                  java.util.Optional<String> userName,
        +                  String operation,
        +                  java.util.Map<String,String> operationParameters,
        +                  java.util.Optional<JobInfo> jobInfo,
        +                  java.util.Optional<NotebookInfo> notebookInfo,
        +                  java.util.Optional<String> clusterId,
        +                  java.util.Optional<Long> readVersion,
        +                  java.util.Optional<String> isolationLevel,
        +                  java.util.Optional<Boolean> isBlindAppend,
        +                  java.util.Optional<java.util.Map<String,String>> operationMetrics,
        +                  java.util.Optional<String> userMetadata)
        +
      • +
      + + + +
        +
      • +

        CommitInfo

        +
        public CommitInfo(java.util.Optional<Long> version,
        +                  java.sql.Timestamp timestamp,
        +                  java.util.Optional<String> userId,
        +                  java.util.Optional<String> userName,
        +                  String operation,
        +                  java.util.Map<String,String> operationParameters,
        +                  java.util.Optional<JobInfo> jobInfo,
        +                  java.util.Optional<NotebookInfo> notebookInfo,
        +                  java.util.Optional<String> clusterId,
        +                  java.util.Optional<Long> readVersion,
        +                  java.util.Optional<String> isolationLevel,
        +                  java.util.Optional<Boolean> isBlindAppend,
        +                  java.util.Optional<java.util.Map<String,String>> operationMetrics,
        +                  java.util.Optional<String> userMetadata,
        +                  java.util.Optional<String> engineInfo)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getVersion

        +
        public java.util.Optional<Long> getVersion()
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        public java.sql.Timestamp getTimestamp()
        +
      • +
      + + + +
        +
      • +

        getUserId

        +
        public java.util.Optional<String> getUserId()
        +
      • +
      + + + +
        +
      • +

        getUserName

        +
        public java.util.Optional<String> getUserName()
        +
      • +
      + + + +
        +
      • +

        getOperation

        +
        public String getOperation()
        +
      • +
      + + + +
        +
      • +

        getOperationParameters

        +
        public java.util.Map<String,String> getOperationParameters()
        +
      • +
      + + + +
        +
      • +

        getJobInfo

        +
        public java.util.Optional<JobInfo> getJobInfo()
        +
      • +
      + + + +
        +
      • +

        getNotebookInfo

        +
        public java.util.Optional<NotebookInfo> getNotebookInfo()
        +
      • +
      + + + +
        +
      • +

        getClusterId

        +
        public java.util.Optional<String> getClusterId()
        +
      • +
      + + + +
        +
      • +

        getReadVersion

        +
        public java.util.Optional<Long> getReadVersion()
        +
      • +
      + + + +
        +
      • +

        getIsolationLevel

        +
        public java.util.Optional<String> getIsolationLevel()
        +
      • +
      + + + +
        +
      • +

        getIsBlindAppend

        +
        public java.util.Optional<Boolean> getIsBlindAppend()
        +
      • +
      + + + +
        +
      • +

        getOperationMetrics

        +
        public java.util.Optional<java.util.Map<String,String>> getOperationMetrics()
        +
      • +
      + + + +
        +
      • +

        getUserMetadata

        +
        public java.util.Optional<String> getUserMetadata()
        +
      • +
      + + + +
        +
      • +

        getEngineInfo

        +
        public java.util.Optional<String> getEngineInfo()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html new file mode 100644 index 00000000000..e45950c8921 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html @@ -0,0 +1,257 @@ + + + + + +FileAction (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Interface FileAction

+
+
+
+
    +
  • +
    +
    All Superinterfaces:
    +
    Action
    +
    +
    +
    All Known Implementing Classes:
    +
    AddCDCFile, AddFile, RemoveFile
    +
    +
    +
    +
    public interface FileAction
    +extends Action
    +
    Generic interface for Actions pertaining to the addition and removal of files.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        String getPath()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        boolean isDataChange()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html new file mode 100644 index 00000000000..9ee08e73d68 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html @@ -0,0 +1,335 @@ + + + + + +Format (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Format

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Format
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class Format
    +extends Object
    +
    A specification of the encoding for the files stored in a table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + + + + +
      Constructors 
      Constructor and Description
      Format() 
      Format(String provider, + java.util.Map<String,String> options) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Format

        +
        public Format(String provider,
        +              java.util.Map<String,String> options)
        +
      • +
      + + + +
        +
      • +

        Format

        +
        public Format()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getProvider

        +
        public String getProvider()
        +
      • +
      + + + +
        +
      • +

        getOptions

        +
        public java.util.Map<String,String> getOptions()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html new file mode 100644 index 00000000000..78c11bc5098 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html @@ -0,0 +1,326 @@ + + + + + +JobInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class JobInfo.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.JobInfo.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    JobInfo
    +
    +
    +
    +
    public static class JobInfo.Builder
    +extends Object
    +
    Builder class for JobInfo. Enables construction of JobInfo object with default values.
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html new file mode 100644 index 00000000000..13364abc72c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html @@ -0,0 +1,398 @@ + + + + + +JobInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class JobInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.JobInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class JobInfo
    +extends Object
    +
    Represents the Databricks Job information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Nested Class Summary

      + + + + + + + + + + +
      Nested Classes 
      Modifier and TypeClass and Description
      static class JobInfo.Builder +
      Builder class for JobInfo.
      +
      +
    • +
    + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      JobInfo(String jobId, + String jobName, + String runId, + String jobOwnerId, + String triggerType) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        JobInfo

        +
        public JobInfo(String jobId,
        +               String jobName,
        +               String runId,
        +               String jobOwnerId,
        +               String triggerType)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getJobId

        +
        public String getJobId()
        +
      • +
      + + + +
        +
      • +

        getJobName

        +
        public String getJobName()
        +
      • +
      + + + +
        +
      • +

        getRunId

        +
        public String getRunId()
        +
      • +
      + + + +
        +
      • +

        getJobOwnerId

        +
        public String getJobOwnerId()
        +
      • +
      + + + +
        +
      • +

        getTriggerType

        +
        public String getTriggerType()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html new file mode 100644 index 00000000000..a33e6f8afeb --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html @@ -0,0 +1,378 @@ + + + + + +Metadata.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Metadata.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Metadata.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    Metadata
    +
    +
    +
    +
    public static class Metadata.Builder
    +extends Object
    +
    Builder class for Metadata. Enables construction of Metadata object with default values.
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html new file mode 100644 index 00000000000..0123d50b353 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html @@ -0,0 +1,472 @@ + + + + + +Metadata (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Metadata

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Metadata
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action
    +
    +
    +
    +
    public final class Metadata
    +extends Object
    +implements Action
    +
    Updates the metadata of the table. The first version of a table must contain + a Metadata action. Subsequent Metadata actions completely + overwrite the current metadata of the table. It is the responsibility of the + writer to ensure that any data already present in the table is still valid + after any change. There can be at most one Metadata action in a + given version of the table.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Metadata

        +
        public Metadata(String id,
        +                String name,
        +                String description,
        +                Format format,
        +                java.util.List<String> partitionColumns,
        +                java.util.Map<String,String> configuration,
        +                java.util.Optional<Long> createdTime,
        +                StructType schema)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getId

        +
        public String getId()
        +
      • +
      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDescription

        +
        public String getDescription()
        +
      • +
      + + + +
        +
      • +

        getFormat

        +
        public Format getFormat()
        +
      • +
      + + + +
        +
      • +

        getPartitionColumns

        +
        public java.util.List<String> getPartitionColumns()
        +
      • +
      + + + +
        +
      • +

        getConfiguration

        +
        public java.util.Map<String,String> getConfiguration()
        +
      • +
      + + + +
        +
      • +

        getCreatedTime

        +
        public java.util.Optional<Long> getCreatedTime()
        +
      • +
      + + + +
        +
      • +

        getSchema

        +
        public StructType getSchema()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html new file mode 100644 index 00000000000..b431ef39da0 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html @@ -0,0 +1,304 @@ + + + + + +NotebookInfo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class NotebookInfo

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.NotebookInfo
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class NotebookInfo
    +extends Object
    +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      NotebookInfo(String notebookId) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NotebookInfo

        +
        public NotebookInfo(String notebookId)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getNotebookId

        +
        public String getNotebookId()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html new file mode 100644 index 00000000000..e01d2379cb5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html @@ -0,0 +1,358 @@ + + + + + +Protocol (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class Protocol

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.Protocol
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action
    +
    +
    +
    +
    public class Protocol
    +extends Object
    +implements Action
    +
    Used to block older clients from reading or writing the log when backwards + incompatible changes are made to the protocol. Readers and writers are + responsible for checking that they meet the minimum versions before performing + any other operations. + + Since this action allows us to explicitly block older clients in the case of a + breaking change to the protocol, clients should be tolerant of messages and + fields that they do not understand.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Protocol

        +
        public Protocol()
        +
      • +
      + + + +
        +
      • +

        Protocol

        +
        public Protocol(int minReaderVersion,
        +                int minWriterVersion)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getMinReaderVersion

        +
        public int getMinReaderVersion()
        +
      • +
      + + + +
        +
      • +

        getMinWriterVersion

        +
        public int getMinWriterVersion()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html new file mode 100644 index 00000000000..5ff3d0a1a15 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html @@ -0,0 +1,352 @@ + + + + + +RemoveFile.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class RemoveFile.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.RemoveFile.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    RemoveFile
    +
    +
    +
    +
    public static class RemoveFile.Builder
    +extends Object
    +
    Builder class for RemoveFile. Enables construction of RemoveFile object with default values.
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html new file mode 100644 index 00000000000..0529ff791e5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html @@ -0,0 +1,472 @@ + + + + + +RemoveFile (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class RemoveFile

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.RemoveFile
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action, FileAction
    +
    +
    +
    +
    public class RemoveFile
    +extends Object
    +implements FileAction
    +
    Logical removal of a given file from the reservoir. Acts as a tombstone before a file is + deleted permanently. + + Note that for protocol compatibility reasons, the fields partitionValues, size, + and tags are only present when the extendedFileMetadata flag is true. New writers should + generally be setting this flag, but old writers (and FSCK) won't, so readers must check this flag + before attempting to consume those values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        RemoveFile

        +
        public RemoveFile(@Nonnull
        +                  String path,
        +                  @Nonnull
        +                  java.util.Optional<Long> deletionTimestamp,
        +                  boolean dataChange,
        +                  boolean extendedFileMetadata,
        +                  @Nullable
        +                  java.util.Map<String,String> partitionValues,
        +                  long size,
        +                  @Nullable
        +                  java.util.Map<String,String> tags)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPath

        +
        public String getPath()
        +
        +
        Specified by:
        +
        getPath in interface FileAction
        +
        +
      • +
      + + + +
        +
      • +

        getDeletionTimestamp

        +
        public java.util.Optional<Long> getDeletionTimestamp()
        +
      • +
      + + + +
        +
      • +

        isDataChange

        +
        public boolean isDataChange()
        +
        +
        Specified by:
        +
        isDataChange in interface FileAction
        +
        +
      • +
      + + + +
        +
      • +

        isExtendedFileMetadata

        +
        public boolean isExtendedFileMetadata()
        +
      • +
      + + + + + + + +
        +
      • +

        getSize

        +
        public long getSize()
        +
      • +
      + + + +
        +
      • +

        getTags

        +
        @Nullable
        +public java.util.Map<String,String> getTags()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html new file mode 100644 index 00000000000..f98587d36ec --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html @@ -0,0 +1,325 @@ + + + + + +SetTransaction (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.actions
+

Class SetTransaction

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.actions.SetTransaction
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Action
    +
    +
    +
    +
    public final class SetTransaction
    +extends Object
    +implements Action
    +
    Sets the committed version for a given application. Used to make operations like streaming append + idempotent.
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      SetTransaction(String appId, + long version, + java.util.Optional<Long> lastUpdated) 
      +
    • +
    + + +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        SetTransaction

        +
        public SetTransaction(@Nonnull
        +                      String appId,
        +                      long version,
        +                      @Nonnull
        +                      java.util.Optional<Long> lastUpdated)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getAppId

        +
        @Nonnull
        +public String getAppId()
        +
      • +
      + + + +
        +
      • +

        getVersion

        +
        public long getVersion()
        +
      • +
      + + + +
        +
      • +

        getLastUpdated

        +
        @Nonnull
        +public java.util.Optional<Long> getLastUpdated()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html new file mode 100644 index 00000000000..94fdf2c1316 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html @@ -0,0 +1,39 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.actions

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html new file mode 100644 index 00000000000..8364d091727 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html @@ -0,0 +1,250 @@ + + + + + +io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.actions

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    Action +
    A marker interface for all Actions that can be applied to a Delta Table.
    +
    FileAction +
    Generic interface for Actions pertaining to the addition and removal of files.
    +
    +
  • +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    AddCDCFile +
    A change file containing CDC data for the Delta version it's within.
    +
    AddFile +
    Represents an action that adds a new file to the table.
    +
    AddFile.Builder +
    Builder class for AddFile.
    +
    CommitInfo +
    Holds provenance information about changes to the table.
    +
    CommitInfo.Builder +
    Builder class for CommitInfo.
    +
    Format +
    A specification of the encoding for the files stored in a table.
    +
    JobInfo +
    Represents the Databricks Job information that committed to the Delta table.
    +
    JobInfo.Builder +
    Builder class for JobInfo.
    +
    Metadata +
    Updates the metadata of the table.
    +
    Metadata.Builder +
    Builder class for Metadata.
    +
    NotebookInfo +
    Represents the Databricks Notebook information that committed to the Delta table.
    +
    Protocol +
    Used to block older clients from reading or writing the log when backwards + incompatible changes are made to the protocol.
    +
    RemoveFile +
    Logical removal of a given file from the reservoir.
    +
    RemoveFile.Builder +
    Builder class for RemoveFile.
    +
    SetTransaction +
    Sets the committed version for a given application.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html new file mode 100644 index 00000000000..e0d09f4bb32 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html @@ -0,0 +1,157 @@ + + + + + +io.delta.standalone.actions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.actions

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +

Interface Hierarchy

+
    +
  • io.delta.standalone.actions.Action + +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html new file mode 100644 index 00000000000..6f1962f31c5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html @@ -0,0 +1,200 @@ + + + + + +CloseableIterator (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface CloseableIterator<T>

+
+
+
+
    +
  • +
    +
    All Superinterfaces:
    +
    AutoCloseable, java.io.Closeable, java.util.Iterator<T>
    +
    +
    +
    +
    public interface CloseableIterator<T>
    +extends java.util.Iterator<T>, java.io.Closeable
    +
    An Iterator that also need to implement the Closeable interface. The caller + should call Closeable.close() method to free all resources properly after using the iterator.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from interface java.util.Iterator

        +forEachRemaining, hasNext, next, remove
      • +
      +
        +
      • + + +

        Methods inherited from interface java.io.Closeable

        +close
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html new file mode 100644 index 00000000000..a65c692ba95 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html @@ -0,0 +1,634 @@ + + + + + +RowRecord (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.data
+

Interface RowRecord

+
+
+
+
    +
  • +
    +
    +
    public interface RowRecord
    +
    Represents one row of data containing a non-empty collection of fieldName - value pairs. + It provides APIs to allow retrieval of values through fieldName lookup. For example, + +
    
    +   if (row.isNullAt("int_field")) {
    +     // handle the null value.
    +   } else {
    +     int x = getInt("int_field");
    +   }
    + 
    +
    +
    See Also:
    +
    StructType, +StructField
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      All Methods Instance Methods Abstract Methods 
      Modifier and TypeMethod and Description
      java.math.BigDecimalgetBigDecimal(String fieldName) +
      Retrieves value from data record and returns the value as a java.math.BigDecimal.
      +
      byte[]getBinary(String fieldName) +
      Retrieves value from data record and returns the value as binary (byte array).
      +
      booleangetBoolean(String fieldName) +
      Retrieves value from data record and returns the value as a primitive boolean.
      +
      bytegetByte(String fieldName) +
      Retrieves value from data record and returns the value as a primitive byte.
      +
      java.sql.DategetDate(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Date.
      +
      doublegetDouble(String fieldName) +
      Retrieves value from data record and returns the value as a primitive double.
      +
      floatgetFloat(String fieldName) +
      Retrieves value from data record and returns the value as a primitive float.
      +
      intgetInt(String fieldName) +
      Retrieves value from data record and returns the value as a primitive int.
      +
      intgetLength() 
      <T> java.util.List<T>getList(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.List<T> object.
      +
      longgetLong(String fieldName) +
      Retrieves value from data record and returns the value as a primitive long.
      +
      <K,V> java.util.Map<K,V>getMap(String fieldName) +
      Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
      +
      RowRecordgetRecord(String fieldName) +
      Retrieves value from data record and returns the value as a RowRecord object.
      +
      StructTypegetSchema() 
      shortgetShort(String fieldName) +
      Retrieves value from data record and returns the value as a primitive short.
      +
      StringgetString(String fieldName) +
      Retrieves value from data record and returns the value as a String object.
      +
      java.sql.TimestampgetTimestamp(String fieldName) +
      Retrieves value from data record and returns the value as a java.sql.Timestamp.
      +
      booleanisNullAt(String fieldName) 
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        getLength

        +
        int getLength()
        +
      • +
      + + + +
        +
      • +

        isNullAt

        +
        boolean isNullAt(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        +
      • +
      + + + +
        +
      • +

        getInt

        +
        int getInt(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive int.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getLong

        +
        long getLong(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive long.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getByte

        +
        byte getByte(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive byte.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getShort

        +
        short getShort(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive short.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBoolean

        +
        boolean getBoolean(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive boolean.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getFloat

        +
        float getFloat(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive float.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDouble

        +
        double getDouble(String fieldName)
        +
        Retrieves value from data record and returns the value as a primitive double.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getString

        +
        String getString(String fieldName)
        +
        Retrieves value from data record and returns the value as a String object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBinary

        +
        byte[] getBinary(String fieldName)
        +
        Retrieves value from data record and returns the value as binary (byte array).
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getBigDecimal

        +
        java.math.BigDecimal getBigDecimal(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.math.BigDecimal.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getTimestamp

        +
        java.sql.Timestamp getTimestamp(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Timestamp.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getDate

        +
        java.sql.Date getDate(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.sql.Date.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - if field is not nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getRecord

        +
        RowRecord getRecord(String fieldName)
        +
        Retrieves value from data record and returns the value as a RowRecord object.
        +
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any nested field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getList

        +
        <T> java.util.List<T> getList(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.List<T> object.
        +
        +
        Type Parameters:
        +
        T - element type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any element field, if that field is not + nullable and null data value read
        +
        +
      • +
      + + + +
        +
      • +

        getMap

        +
        <K,V> java.util.Map<K,V> getMap(String fieldName)
        +
        Retrieves value from data record and returns the value as a java.util.Map<K, V> + object.
        +
        +
        Type Parameters:
        +
        K - key type
        +
        V - value type
        +
        Parameters:
        +
        fieldName - name of field/column, not null
        +
        Throws:
        +
        IllegalArgumentException - if fieldName does not exist in this schema
        +
        ClassCastException - if data type does not match
        +
        NullPointerException - for this field or any key/value field, if that field is not + nullable and null data value read
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html new file mode 100644 index 00000000000..66dc9668ace --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html @@ -0,0 +1,21 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.data

+
+

Interfaces

+ +
+ + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html new file mode 100644 index 00000000000..e64760188fd --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html @@ -0,0 +1,148 @@ + + + + + +io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.data

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    CloseableIterator<T> +
    An Iterator that also need to implement the Closeable interface.
    +
    RowRecord +
    Represents one row of data containing a non-empty collection of fieldName - value pairs.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html new file mode 100644 index 00000000000..7c31ff3d2a3 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html @@ -0,0 +1,145 @@ + + + + + +io.delta.standalone.data Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.data

+Package Hierarchies: + +
+
+

Interface Hierarchy

+
    +
  • AutoCloseable +
      +
    • java.io.Closeable +
        +
      • io.delta.standalone.data.CloseableIterator<T> (also extends java.util.Iterator<E>)
      • +
      +
    • +
    +
  • +
  • java.util.Iterator<E> + +
  • +
  • io.delta.standalone.data.RowRecord
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html new file mode 100644 index 00000000000..d299243300e --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html @@ -0,0 +1,276 @@ + + + + + +ConcurrentAppendException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class ConcurrentAppendException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class ConcurrentAppendException
    +extends DeltaConcurrentModificationException
    +
    Thrown when files are added that would have been read by the current transaction.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ConcurrentAppendException

        +
        public ConcurrentAppendException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html new file mode 100644 index 00000000000..6ec968e7a83 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html @@ -0,0 +1,276 @@ + + + + + +ConcurrentDeleteDeleteException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class ConcurrentDeleteDeleteException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class ConcurrentDeleteDeleteException
    +extends DeltaConcurrentModificationException
    +
    Thrown when the current transaction deletes data that was deleted by a concurrent transaction.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ConcurrentDeleteDeleteException

        +
        public ConcurrentDeleteDeleteException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html new file mode 100644 index 00000000000..10201119c29 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html @@ -0,0 +1,276 @@ + + + + + +ConcurrentDeleteReadException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class ConcurrentDeleteReadException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class ConcurrentDeleteReadException
    +extends DeltaConcurrentModificationException
    +
    Thrown when the current transaction reads data that was deleted by a concurrent transaction.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ConcurrentDeleteReadException

        +
        public ConcurrentDeleteReadException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html new file mode 100644 index 00000000000..df3042ae086 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html @@ -0,0 +1,276 @@ + + + + + +ConcurrentTransactionException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class ConcurrentTransactionException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class ConcurrentTransactionException
    +extends DeltaConcurrentModificationException
    +
    Thrown when concurrent transaction both attempt to update the same idempotent transaction.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ConcurrentTransactionException

        +
        public ConcurrentTransactionException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html new file mode 100644 index 00000000000..3bde7b89ca9 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html @@ -0,0 +1,275 @@ + + + + + +DeltaConcurrentModificationException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class DeltaConcurrentModificationException

+
+
+
    +
  • Object
  • +
  • +
      +
    • Throwable
    • +
    • +
        +
      • Exception
      • +
      • +
          +
        • RuntimeException
        • +
        • +
            +
          • java.util.ConcurrentModificationException
          • +
          • +
              +
            • io.delta.standalone.exceptions.DeltaConcurrentModificationException
            • +
            +
          • +
          +
        • +
        +
      • +
      +
    • +
    +
  • +
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DeltaConcurrentModificationException

        +
        public DeltaConcurrentModificationException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html new file mode 100644 index 00000000000..3484b0607c3 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html @@ -0,0 +1,292 @@ + + + + + +DeltaStandaloneException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class DeltaStandaloneException

+
+
+
    +
  • Object
  • +
  • +
      +
    • Throwable
    • +
    • +
        +
      • Exception
      • +
      • +
          +
        • RuntimeException
        • +
        • +
            +
          • io.delta.standalone.exceptions.DeltaStandaloneException
          • +
          +
        • +
        +
      • +
      +
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class DeltaStandaloneException
    +extends RuntimeException
    +
    Thrown when a query fails, usually because the query itself is invalid.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DeltaStandaloneException

        +
        public DeltaStandaloneException()
        +
      • +
      + + + +
        +
      • +

        DeltaStandaloneException

        +
        public DeltaStandaloneException(String message)
        +
      • +
      + + + +
        +
      • +

        DeltaStandaloneException

        +
        public DeltaStandaloneException(String message,
        +                                Throwable cause)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html new file mode 100644 index 00000000000..c8542865d28 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html @@ -0,0 +1,277 @@ + + + + + +MetadataChangedException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class MetadataChangedException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class MetadataChangedException
    +extends DeltaConcurrentModificationException
    +
    Thrown when the metadata of the Delta table has changed between the time of read + and the time of commit.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        MetadataChangedException

        +
        public MetadataChangedException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html new file mode 100644 index 00000000000..3e268d6305f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html @@ -0,0 +1,276 @@ + + + + + +ProtocolChangedException (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.exceptions
+

Class ProtocolChangedException

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable
    +
    +
    +
    +
    public class ProtocolChangedException
    +extends DeltaConcurrentModificationException
    +
    Thrown when the protocol version has changed between the time of read and the time of commit.
    +
    +
    See Also:
    +
    Serialized Form
    +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Summary

      +
        +
      • + + +

        Methods inherited from class Throwable

        +addSuppressed, fillInStackTrace, getCause, getLocalizedMessage, getMessage, getStackTrace, getSuppressed, initCause, printStackTrace, printStackTrace, printStackTrace, setStackTrace, toString
      • +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ProtocolChangedException

        +
        public ProtocolChangedException(String message)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html new file mode 100644 index 00000000000..8f997950539 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html @@ -0,0 +1,27 @@ + + + + + +io.delta.standalone.exceptions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.exceptions

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html new file mode 100644 index 00000000000..45007a6de2f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html @@ -0,0 +1,185 @@ + + + + + +io.delta.standalone.exceptions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.exceptions

+
+
+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html new file mode 100644 index 00000000000..8f402764dc5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html @@ -0,0 +1,161 @@ + + + + + +io.delta.standalone.exceptions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.exceptions

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html new file mode 100644 index 00000000000..b6c9c5182f6 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html @@ -0,0 +1,317 @@ + + + + + +And (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class And

+
+
+ +
+ +
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html new file mode 100644 index 00000000000..fab5bd63945 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html @@ -0,0 +1,288 @@ + + + + + +BinaryComparison (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class BinaryComparison

+
+
+ +
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BinaryComparison

        +
        public BinaryComparison(Expression left,
        +                        Expression right,
        +                        String symbol)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html new file mode 100644 index 00000000000..640d949cf94 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html @@ -0,0 +1,372 @@ + + + + + +BinaryExpression (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class BinaryExpression

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.expressions.BinaryExpression
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression
    +
    +
    +
    Direct Known Subclasses:
    +
    BinaryOperator
    +
    +
    +
    +
    public abstract class BinaryExpression
    +extends Object
    +implements Expression
    +
    An expression with two inputs and one output. The output is by default evaluated to null + if any input is evaluated to null.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + + + + + + + + + +
        +
      • +

        eval

        +
        public final Object eval(RowRecord record)
        +
        +
        Specified by:
        +
        eval in interface Expression
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        +
      • +
      + + + + + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html new file mode 100644 index 00000000000..d89ee5a291b --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html @@ -0,0 +1,312 @@ + + + + + +BinaryOperator (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class BinaryOperator

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression
    +
    +
    +
    Direct Known Subclasses:
    +
    And, BinaryComparison, Or
    +
    +
    +
    +
    public abstract class BinaryOperator
    +extends BinaryExpression
    +
    A BinaryExpression that is an operator, with two properties: +
      +
    1. The string representation is x symbol y, rather than funcName(x, y).
    2. +
    3. Two inputs are expected to be of the same type. If the two inputs have different types, an + IllegalArgumentException will be thrown.
    4. +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BinaryOperator

        +
        public BinaryOperator(Expression left,
        +                      Expression right,
        +                      String symbol)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html new file mode 100644 index 00000000000..726a401b8bd --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html @@ -0,0 +1,395 @@ + + + + + +Column (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class Column

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression
    +
    +
    +
    +
    public final class Column
    +extends LeafExpression
    +
    A column whose row-value will be computed based on the data in a RowRecord. +

    + Usage: new Column(columnName, columnDataType). +

    + It is recommended that you instantiate using an existing table schema (StructType). + e.g. mySchema.column(columnName)

    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        Column

        +
        public Column(String name,
        +              DataType dataType)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        name

        +
        public String name()
        +
      • +
      + + + +
        +
      • +

        eval

        +
        public Object eval(RowRecord record)
        +
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        +
      • +
      + + + +
        +
      • +

        dataType

        +
        public DataType dataType()
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      + + + + + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Specified by:
        +
        equals in class LeafExpression
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html new file mode 100644 index 00000000000..f4d777239dd --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html @@ -0,0 +1,323 @@ + + + + + +EqualTo (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class EqualTo

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class EqualTo
    +extends BinaryComparison
    +implements Predicate
    +
    Usage: new EqualTo(expr1, expr2) - Returns true if expr1 equals expr2, + else false.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Expression.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Expression.html new file mode 100644 index 00000000000..4a0c9ee686d --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Expression.html @@ -0,0 +1,288 @@ + + + + + +Expression (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Interface Expression

+
+
+
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        eval

        +
        Object eval(RowRecord record)
        +
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        +
      • +
      + + + + + + + +
        +
      • +

        toString

        +
        String toString()
        +
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      + + + +
        +
      • +

        references

        +
        default java.util.Set<String> references()
        +
      • +
      + + + +
        +
      • +

        children

        +
        java.util.List<Expression> children()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThan.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThan.html new file mode 100644 index 00000000000..ee88ade7ead --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThan.html @@ -0,0 +1,323 @@ + + + + + +GreaterThan (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class GreaterThan

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class GreaterThan
    +extends BinaryComparison
    +implements Predicate
    +
    Usage: new GreaterThan(expr1, expr2) - Returns true if expr1 is greater than + expr2, else false.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThanOrEqual.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThanOrEqual.html new file mode 100644 index 00000000000..0b627243ced --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/GreaterThanOrEqual.html @@ -0,0 +1,323 @@ + + + + + +GreaterThanOrEqual (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class GreaterThanOrEqual

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class GreaterThanOrEqual
    +extends BinaryComparison
    +implements Predicate
    +
    Usage: new GreaterThanOrEqual(expr1, expr2) - Returns true if expr1 is greater + than or equal to expr2, else false.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/In.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/In.html new file mode 100644 index 00000000000..181fae6cffa --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/In.html @@ -0,0 +1,348 @@ + + + + + +In (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class In

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.expressions.In
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class In
    +extends Object
    +implements Predicate
    +
    Usage: new In(expr, exprList) - Returns true if expr is equal to any in + exprList, else false.
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      In(Expression value, + java.util.List<? extends Expression> elems) 
      +
    • +
    + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + +
      All Methods Instance Methods Concrete Methods 
      Modifier and TypeMethod and Description
      java.util.List<Expression>children() 
      Booleaneval(RowRecord record) +
      This implements the IN expression functionality outlined by the Databricks SQL Null + semantics reference guide.
      +
      StringtoString() 
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
      • +
      + + +
    • +
    +
  • +
+
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        eval

        +
        public Boolean eval(RowRecord record)
        +
        This implements the IN expression functionality outlined by the Databricks SQL Null + semantics reference guide. The logic is as follows: +
          +
        • TRUE if the non-NULL value is found in the list
        • +
        • FALSE if the non-NULL value is not found in the list and the list does not contain + NULL values
        • +
        • NULL if the value is NULL, or the non-NULL value is not found in the list and the + list contains at least one NULL value
        • +
        +
        +
        Specified by:
        +
        eval in interface Expression
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        See Also:
        +
        NULL Semantics
        +
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNotNull.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNotNull.html new file mode 100644 index 00000000000..ecc160bd8f5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNotNull.html @@ -0,0 +1,328 @@ + + + + + +IsNotNull (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class IsNotNull

+
+
+ +
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        IsNotNull

        +
        public IsNotNull(Expression child)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNull.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNull.html new file mode 100644 index 00000000000..9cacd51360b --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/IsNull.html @@ -0,0 +1,328 @@ + + + + + +IsNull (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class IsNull

+
+
+ +
+ +
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LeafExpression.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LeafExpression.html new file mode 100644 index 00000000000..56c38ae581b --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LeafExpression.html @@ -0,0 +1,340 @@ + + + + + +LeafExpression (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class LeafExpression

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.expressions.LeafExpression
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression
    +
    +
    +
    Direct Known Subclasses:
    +
    Column, Literal
    +
    +
    +
    +
    public abstract class LeafExpression
    +extends Object
    +implements Expression
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        LeafExpression

        +
        public LeafExpression()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        references

        +
        public java.util.Set<String> references()
        +
        +
        Specified by:
        +
        references in interface Expression
        +
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public abstract boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public abstract int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThan.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThan.html new file mode 100644 index 00000000000..261b2c06454 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThan.html @@ -0,0 +1,323 @@ + + + + + +LessThan (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class LessThan

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class LessThan
    +extends BinaryComparison
    +implements Predicate
    +
    Usage: new LessThan(expr1, expr2) - Returns true if expr1 is less than + expr2, else false.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThanOrEqual.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThanOrEqual.html new file mode 100644 index 00000000000..ef3ef2c2283 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/LessThanOrEqual.html @@ -0,0 +1,323 @@ + + + + + +LessThanOrEqual (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class LessThanOrEqual

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression, Predicate
    +
    +
    +
    +
    public final class LessThanOrEqual
    +extends BinaryComparison
    +implements Predicate
    +
    Usage: new LessThanOrEqual(expr1, expr2) - Returns true if expr1 is less than or + equal to expr2, else false.
    +
  • +
+
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Literal.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Literal.html new file mode 100644 index 00000000000..6743c339e4a --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Literal.html @@ -0,0 +1,552 @@ + + + + + +Literal (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class Literal

+
+
+ +
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        True

        +
        public static final Literal True
        +
      • +
      + + + +
        +
      • +

        False

        +
        public static final Literal False
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        value

        +
        public Object value()
        +
      • +
      + + + +
        +
      • +

        eval

        +
        public Object eval(RowRecord record)
        +
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        +
      • +
      + + + +
        +
      • +

        dataType

        +
        public DataType dataType()
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Specified by:
        +
        equals in class LeafExpression
        +
        +
      • +
      + + + + + + + +
        +
      • +

        of

        +
        public static Literal of(int value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(boolean value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(byte[] value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(java.sql.Date value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(java.math.BigDecimal value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(double value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(float value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(long value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(short value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(String value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(java.sql.Timestamp value)
        +
      • +
      + + + +
        +
      • +

        of

        +
        public static Literal of(byte value)
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Not.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Not.html new file mode 100644 index 00000000000..dd1f831d27e --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Not.html @@ -0,0 +1,320 @@ + + + + + +Not (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class Not

+
+
+ +
+ +
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object childResult)
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Specified by:
        +
        toString in interface Expression
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Or.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Or.html new file mode 100644 index 00000000000..b1f5327d7ac --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Or.html @@ -0,0 +1,317 @@ + + + + + +Or (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class Or

+
+
+ +
+ +
+
+ +
+
+
    +
  • + + + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        nullSafeEval

        +
        public Object nullSafeEval(Object leftResult,
        +                           Object rightResult)
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Predicate.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Predicate.html new file mode 100644 index 00000000000..f66bb1cb64a --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Predicate.html @@ -0,0 +1,239 @@ + + + + + +Predicate (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Interface Predicate

+
+
+
+ +
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/UnaryExpression.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/UnaryExpression.html new file mode 100644 index 00000000000..399fde096d3 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/UnaryExpression.html @@ -0,0 +1,357 @@ + + + + + +UnaryExpression (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.expressions
+

Class UnaryExpression

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.expressions.UnaryExpression
    • +
    +
  • +
+
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    Expression
    +
    +
    +
    Direct Known Subclasses:
    +
    IsNotNull, IsNull, Not
    +
    +
    +
    +
    public abstract class UnaryExpression
    +extends Object
    +implements Expression
    +
    An expression with one input and one output. The output is by default evaluated to null + if the input is evaluated to null.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        UnaryExpression

        +
        public UnaryExpression(Expression child)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + + + + + +
        +
      • +

        eval

        +
        public Object eval(RowRecord record)
        +
        +
        Specified by:
        +
        eval in interface Expression
        +
        Parameters:
        +
        record - the input record to evaluate.
        +
        +
      • +
      + + + + + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-frame.html new file mode 100644 index 00000000000..d7675e672d6 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-frame.html @@ -0,0 +1,42 @@ + + + + + +io.delta.standalone.expressions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.expressions

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-summary.html new file mode 100644 index 00000000000..83ac7cccba4 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-summary.html @@ -0,0 +1,271 @@ + + + + + +io.delta.standalone.expressions (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.expressions

+
+
+
    +
  • + + + + + + + + + + + + + + + + +
    Interface Summary 
    InterfaceDescription
    Expression +
    An expression in Delta Standalone.
    +
    Predicate 
    +
  • +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    And +
    Usage: new And(expr1, expr2) - Logical AND
    +
    BinaryComparison +
    A BinaryOperator that compares the left and right Expressions and returns a + boolean value.
    +
    BinaryExpression +
    An expression with two inputs and one output.
    +
    BinaryOperator +
    A BinaryExpression that is an operator, with two properties: + + The string representation is x symbol y, rather than funcName(x, y). + Two inputs are expected to be of the same type.
    +
    Column +
    A column whose row-value will be computed based on the data in a RowRecord.
    +
    EqualTo +
    Usage: new EqualTo(expr1, expr2) - Returns true if expr1 equals expr2, + else false.
    +
    GreaterThan +
    Usage: new GreaterThan(expr1, expr2) - Returns true if expr1 is greater than + expr2, else false.
    +
    GreaterThanOrEqual +
    Usage: new GreaterThanOrEqual(expr1, expr2) - Returns true if expr1 is greater + than or equal to expr2, else false.
    +
    In +
    Usage: new In(expr, exprList) - Returns true if expr is equal to any in + exprList, else false.
    +
    IsNotNull +
    Usage: new IsNotNull(expr) - Returns true if expr is not null, else false.
    +
    IsNull +
    Usage: new IsNull(expr) - Returns true if expr is null, else false.
    +
    LeafExpression 
    LessThan +
    Usage: new LessThan(expr1, expr2) - Returns true if expr1 is less than + expr2, else false.
    +
    LessThanOrEqual +
    Usage: new LessThanOrEqual(expr1, expr2) - Returns true if expr1 is less than or + equal to expr2, else false.
    +
    Literal 
    Not +
    Usage: new Not(expr) - Logical not.
    +
    Or +
    Usage: new Or(expr1, expr2) - Logical OR
    +
    UnaryExpression +
    An expression with one input and one output.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-tree.html new file mode 100644 index 00000000000..f0e304c0ec6 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/package-tree.html @@ -0,0 +1,175 @@ + + + + + +io.delta.standalone.expressions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.expressions

+Package Hierarchies: + +
+
+

Class Hierarchy

+
    +
  • Object +
      +
    • io.delta.standalone.expressions.BinaryExpression (implements io.delta.standalone.expressions.Expression) +
        +
      • io.delta.standalone.expressions.BinaryOperator +
          +
        • io.delta.standalone.expressions.And (implements io.delta.standalone.expressions.Predicate)
        • +
        • io.delta.standalone.expressions.BinaryComparison (implements io.delta.standalone.expressions.Predicate) + +
        • +
        • io.delta.standalone.expressions.Or (implements io.delta.standalone.expressions.Predicate)
        • +
        +
      • +
      +
    • +
    • io.delta.standalone.expressions.In (implements io.delta.standalone.expressions.Predicate)
    • +
    • io.delta.standalone.expressions.LeafExpression (implements io.delta.standalone.expressions.Expression) +
        +
      • io.delta.standalone.expressions.Column
      • +
      • io.delta.standalone.expressions.Literal
      • +
      +
    • +
    • io.delta.standalone.expressions.UnaryExpression (implements io.delta.standalone.expressions.Expression) +
        +
      • io.delta.standalone.expressions.IsNotNull (implements io.delta.standalone.expressions.Predicate)
      • +
      • io.delta.standalone.expressions.IsNull (implements io.delta.standalone.expressions.Predicate)
      • +
      • io.delta.standalone.expressions.Not (implements io.delta.standalone.expressions.Predicate)
      • +
      +
    • +
    +
  • +
+

Interface Hierarchy

+
    +
  • io.delta.standalone.expressions.Expression +
      +
    • io.delta.standalone.expressions.Predicate
    • +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-frame.html new file mode 100644 index 00000000000..1608e61b7de --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-frame.html @@ -0,0 +1,34 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-summary.html new file mode 100644 index 00000000000..4e96fef86c4 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-summary.html @@ -0,0 +1,215 @@ + + + + + +io.delta.standalone (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone

+
+
+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-tree.html new file mode 100644 index 00000000000..130b4546cc3 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/package-tree.html @@ -0,0 +1,157 @@ + + + + + +io.delta.standalone Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +

Interface Hierarchy

+ +

Enum Hierarchy

+
    +
  • Object +
      +
    • Enum<E> (implements Comparable<T>, java.io.Serializable) + +
    • +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/LogStore.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/LogStore.html new file mode 100644 index 00000000000..7f106085408 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/LogStore.html @@ -0,0 +1,464 @@ + + + + + +LogStore (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.storage
+

Class LogStore

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.storage.LogStore
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public abstract class LogStore
    +extends Object
    +
    :: DeveloperApi :: +

    + General interface for all critical file system operations required to read and write the + Delta logs. The correctness is predicated on the atomicity and durability guarantees of + the implementation of this interface. Specifically, +

      +
    1. + Atomic visibility of files: If isPartialWriteVisible is false, any file written through + this store must be made visible atomically. In other words, this should not generate + partial files. +
    2. +
    3. + Mutual exclusion: Only one writer must be able to create (or rename) a file at the final + destination. +
    4. +
    5. + Consistent listing: Once a file has been written in a directory, all future listings for + that directory must return that file. +
    6. +
    +

    + All subclasses of this interface is required to have a constructor that takes Configuration + as a single parameter. This constructor is used to dynamically create the LogStore. +

    + LogStore and its implementations are not meant for direct access but for configuration based + on storage system.

    +
    +
    Since:
    +
    0.3.0
    +
    See Also:
    +
    Delta Storage
    +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Summary

      + + + + + + + + +
      Constructors 
      Constructor and Description
      LogStore(org.apache.hadoop.conf.Configuration initHadoopConf) 
      +
    • +
    + +
      +
    • + + +

      Method Summary

      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      All Methods Instance Methods Abstract Methods Concrete Methods 
      Modifier and TypeMethod and Description
      org.apache.hadoop.conf.ConfigurationinitHadoopConf() +
      :: DeveloperApi ::
      +
      abstract BooleanisPartialWriteVisible(org.apache.hadoop.fs.Path path, + org.apache.hadoop.conf.Configuration hadoopConf) +
      :: DeveloperApi ::
      +
      abstract java.util.Iterator<org.apache.hadoop.fs.FileStatus>listFrom(org.apache.hadoop.fs.Path path, + org.apache.hadoop.conf.Configuration hadoopConf) +
      :: DeveloperApi ::
      +
      abstract CloseableIterator<String>read(org.apache.hadoop.fs.Path path, + org.apache.hadoop.conf.Configuration hadoopConf) +
      :: DeveloperApi ::
      +
      abstract org.apache.hadoop.fs.PathresolvePathOnPhysicalStorage(org.apache.hadoop.fs.Path path, + org.apache.hadoop.conf.Configuration hadoopConf) +
      :: DeveloperApi ::
      +
      abstract voidwrite(org.apache.hadoop.fs.Path path, + java.util.Iterator<String> actions, + Boolean overwrite, + org.apache.hadoop.conf.Configuration hadoopConf) +
      :: DeveloperApi ::
      +
      +
        +
      • + + +

        Methods inherited from class Object

        +equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
      • +
      +
    • +
    +
  • +
+
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        LogStore

        +
        public LogStore(org.apache.hadoop.conf.Configuration initHadoopConf)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        initHadoopConf

        +
        public org.apache.hadoop.conf.Configuration initHadoopConf()
        +
        :: DeveloperApi :: +

        + Hadoop configuration that should only be used during initialization of LogStore. Each method + should use their hadoopConf parameter rather than this (potentially outdated) hadoop + configuration.

        +
      • +
      + + + +
        +
      • +

        read

        +
        public abstract CloseableIterator<String> read(org.apache.hadoop.fs.Path path,
        +                                               org.apache.hadoop.conf.Configuration hadoopConf)
        +
        :: DeveloperApi :: +

        + Load the given file and return an Iterator of lines, with line breaks removed from + each line. Callers of this function are responsible to close the iterator if they are done + with it.

        +
        +
        Parameters:
        +
        path - the path to load
        +
        hadoopConf - the latest hadoopConf
        +
        Since:
        +
        0.3.0
        +
        +
      • +
      + + + +
        +
      • +

        write

        +
        public abstract void write(org.apache.hadoop.fs.Path path,
        +                           java.util.Iterator<String> actions,
        +                           Boolean overwrite,
        +                           org.apache.hadoop.conf.Configuration hadoopConf)
        +                    throws java.nio.file.FileAlreadyExistsException
        +
        :: DeveloperApi :: +

        + Write the given actions to the given Path with or without overwrite as indicated. +

        + Implementation must throw FileAlreadyExistsException exception if the + file already exists and overwrite = false. Furthermore, if + isPartialWriteVisible(org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration) returns false, implementation must ensure that the + entire file is made visible atomically, that is, it should not generate partial files.

        +
        +
        Parameters:
        +
        path - the path to write to
        +
        actions - actions to be written
        +
        overwrite - if true, overwrites the file if it already exists
        +
        hadoopConf - the latest hadoopConf
        +
        Throws:
        +
        java.nio.file.FileAlreadyExistsException - if the file already exists and overwrite is + false
        +
        Since:
        +
        0.3.0
        +
        +
      • +
      + + + +
        +
      • +

        listFrom

        +
        public abstract java.util.Iterator<org.apache.hadoop.fs.FileStatus> listFrom(org.apache.hadoop.fs.Path path,
        +                                                                             org.apache.hadoop.conf.Configuration hadoopConf)
        +                                                                      throws java.io.FileNotFoundException
        +
        :: DeveloperApi :: +

        + List the paths in the same directory that are lexicographically greater or equal to + (UTF-8 sorting) the given Path. The result should also be sorted by the file name.

        +
        +
        Parameters:
        +
        path - the path to load
        +
        hadoopConf - the latest hadoopConf
        +
        Throws:
        +
        java.io.FileNotFoundException - if the file does not exist
        +
        Since:
        +
        0.3.0
        +
        +
      • +
      + + + +
        +
      • +

        resolvePathOnPhysicalStorage

        +
        public abstract org.apache.hadoop.fs.Path resolvePathOnPhysicalStorage(org.apache.hadoop.fs.Path path,
        +                                                                       org.apache.hadoop.conf.Configuration hadoopConf)
        +
        :: DeveloperApi :: +

        + Resolve the fully qualified path for the given Path.

        +
        +
        Parameters:
        +
        path - the path to resolve
        +
        hadoopConf - the latest hadoopConf
        +
        Since:
        +
        0.3.0
        +
        +
      • +
      + + + +
        +
      • +

        isPartialWriteVisible

        +
        public abstract Boolean isPartialWriteVisible(org.apache.hadoop.fs.Path path,
        +                                              org.apache.hadoop.conf.Configuration hadoopConf)
        +
        :: DeveloperApi :: +

        + Whether a partial write is visible for the underlying file system of the given Path.

        +
        +
        Parameters:
        +
        path - the path in question
        +
        hadoopConf - the latest hadoopConf
        +
        Since:
        +
        0.3.0
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-frame.html new file mode 100644 index 00000000000..cfa901365d0 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-frame.html @@ -0,0 +1,20 @@ + + + + + +io.delta.standalone.storage (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.storage

+
+

Classes

+ +
+ + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-summary.html new file mode 100644 index 00000000000..265dc8fda6f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-summary.html @@ -0,0 +1,142 @@ + + + + + +io.delta.standalone.storage (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.storage

+
+
+
    +
  • + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    LogStore +
    :: DeveloperApi ::
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-tree.html new file mode 100644 index 00000000000..78141e134dc --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/storage/package-tree.html @@ -0,0 +1,135 @@ + + + + + +io.delta.standalone.storage Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.storage

+Package Hierarchies: + +
+
+

Class Hierarchy

+
    +
  • Object +
      +
    • io.delta.standalone.storage.LogStore
    • +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html new file mode 100644 index 00000000000..cd06c1afd90 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ArrayType.html @@ -0,0 +1,336 @@ + + + + + +ArrayType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ArrayType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ArrayType
    +extends DataType
    +
    The data type for collections of multiple values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ArrayType

        +
        public ArrayType(DataType elementType,
        +                 boolean containsNull)
        +
        +
        Parameters:
        +
        elementType - the data type of values
        +
        containsNull - indicates if values have null value
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getElementType

        +
        public DataType getElementType()
        +
      • +
      + + + +
        +
      • +

        containsNull

        +
        public boolean containsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html new file mode 100644 index 00000000000..17b06fd1a5f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BinaryType.html @@ -0,0 +1,248 @@ + + + + + +BinaryType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BinaryType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BinaryType
    +extends DataType
    +
    The data type representing byte[] values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BinaryType

        +
        public BinaryType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html new file mode 100644 index 00000000000..5e333a9a35d --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/BooleanType.html @@ -0,0 +1,248 @@ + + + + + +BooleanType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class BooleanType

+
+
+ +
+
    +
  • +
    +
    +
    public final class BooleanType
    +extends DataType
    +
    The data type representing boolean values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        BooleanType

        +
        public BooleanType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ByteType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ByteType.html new file mode 100644 index 00000000000..3ca6b4b65dd --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ByteType.html @@ -0,0 +1,286 @@ + + + + + +ByteType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ByteType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ByteType
    +extends DataType
    +
    The data type representing byte values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ByteType

        +
        public ByteType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DataType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DataType.html new file mode 100644 index 00000000000..e939922bcc8 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DataType.html @@ -0,0 +1,363 @@ + + + + + +DataType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DataType

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.DataType
    • +
    +
  • +
+
+ +
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DataType

        +
        public DataType()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getTypeName

        +
        public String getTypeName()
        +
      • +
      + + + +
        +
      • +

        getSimpleString

        +
        public String getSimpleString()
        +
      • +
      + + + +
        +
      • +

        getCatalogString

        +
        public String getCatalogString()
        +
      • +
      + + + +
        +
      • +

        toJson

        +
        public String toJson()
        +
      • +
      + + + +
        +
      • +

        toPrettyJson

        +
        public String toPrettyJson()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DateType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DateType.html new file mode 100644 index 00000000000..f8341dbea90 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DateType.html @@ -0,0 +1,249 @@ + + + + + +DateType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DateType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DateType
    +extends DataType
    +
    A date type, supporting "0001-01-01" through "9999-12-31". + Internally, this is represented as the number of days from 1970-01-01.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DateType

        +
        public DateType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html new file mode 100644 index 00000000000..a3834c89adb --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DecimalType.html @@ -0,0 +1,373 @@ + + + + + +DecimalType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DecimalType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DecimalType
    +extends DataType
    +
    The data type representing java.math.BigDecimal values. + A Decimal that must have fixed precision (the maximum number of digits) and scale (the number + of digits on right side of dot). + + The precision can be up to 38, scale can also be up to 38 (less or equal to precision). + + The default precision and scale is (10, 0).
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Field Detail

      + + + +
        +
      • +

        USER_DEFAULT

        +
        public static final DecimalType USER_DEFAULT
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DecimalType

        +
        public DecimalType(int precision,
        +                   int scale)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getPrecision

        +
        public int getPrecision()
        +
      • +
      + + + +
        +
      • +

        getScale

        +
        public int getScale()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html new file mode 100644 index 00000000000..fd56e895b5c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/DoubleType.html @@ -0,0 +1,248 @@ + + + + + +DoubleType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class DoubleType

+
+
+ +
+
    +
  • +
    +
    +
    public final class DoubleType
    +extends DataType
    +
    The data type representing double values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        DoubleType

        +
        public DoubleType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.Builder.html new file mode 100644 index 00000000000..2fdcd11116e --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.Builder.html @@ -0,0 +1,437 @@ + + + + + +FieldMetadata.Builder (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class FieldMetadata.Builder

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.FieldMetadata.Builder
    • +
    +
  • +
+
+
    +
  • +
    +
    Enclosing class:
    +
    FieldMetadata
    +
    +
    +
    +
    public static class FieldMetadata.Builder
    +extends Object
    +
    Builder class for FieldMetadata.
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.html new file mode 100644 index 00000000000..678da0eef0c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FieldMetadata.html @@ -0,0 +1,355 @@ + + + + + +FieldMetadata (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class FieldMetadata

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.FieldMetadata
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class FieldMetadata
    +extends Object
    +
    The metadata for a given StructField.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getEntries

        +
        public java.util.Map<String,Object> getEntries()
        +
      • +
      + + + +
        +
      • +

        contains

        +
        public boolean contains(String key)
        +
        +
        Parameters:
        +
        key - the key to check for
        +
        +
      • +
      + + + +
        +
      • +

        get

        +
        public Object get(String key)
        +
        +
        Parameters:
        +
        key - the key to check for
        +
        +
      • +
      + + + +
        +
      • +

        toString

        +
        public String toString()
        +
        +
        Overrides:
        +
        toString in class Object
        +
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      + + + + +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FloatType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FloatType.html new file mode 100644 index 00000000000..14bd12ea3ea --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/FloatType.html @@ -0,0 +1,248 @@ + + + + + +FloatType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class FloatType

+
+
+ +
+
    +
  • +
    +
    +
    public final class FloatType
    +extends DataType
    +
    The data type representing float values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        FloatType

        +
        public FloatType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html new file mode 100644 index 00000000000..ee8df221144 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/IntegerType.html @@ -0,0 +1,286 @@ + + + + + +IntegerType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class IntegerType

+
+
+ +
+
    +
  • +
    +
    +
    public final class IntegerType
    +extends DataType
    +
    The data type representing int values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        IntegerType

        +
        public IntegerType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/LongType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/LongType.html new file mode 100644 index 00000000000..892db8dd247 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/LongType.html @@ -0,0 +1,286 @@ + + + + + +LongType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class LongType

+
+
+ +
+
    +
  • +
    +
    +
    public final class LongType
    +extends DataType
    +
    The data type representing long values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        LongType

        +
        public LongType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/MapType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/MapType.html new file mode 100644 index 00000000000..f2f12d13ab5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/MapType.html @@ -0,0 +1,352 @@ + + + + + +MapType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class MapType

+
+
+ +
+
    +
  • +
    +
    +
    public final class MapType
    +extends DataType
    +
    The data type for Maps. Keys in a map are not allowed to have null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        MapType

        +
        public MapType(DataType keyType,
        +               DataType valueType,
        +               boolean valueContainsNull)
        +
        +
        Parameters:
        +
        keyType - the data type of map keys
        +
        valueType - the data type of map values
        +
        valueContainsNull - indicates if map values have null values
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getKeyType

        +
        public DataType getKeyType()
        +
      • +
      + + + +
        +
      • +

        getValueType

        +
        public DataType getValueType()
        +
      • +
      + + + +
        +
      • +

        valueContainsNull

        +
        public boolean valueContainsNull()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/NullType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/NullType.html new file mode 100644 index 00000000000..f09c959ae26 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/NullType.html @@ -0,0 +1,248 @@ + + + + + +NullType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class NullType

+
+
+ +
+
    +
  • +
    +
    +
    public final class NullType
    +extends DataType
    +
    The data type representing null values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        NullType

        +
        public NullType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ShortType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ShortType.html new file mode 100644 index 00000000000..32924739063 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/ShortType.html @@ -0,0 +1,286 @@ + + + + + +ShortType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class ShortType

+
+
+ +
+
    +
  • +
    +
    +
    public final class ShortType
    +extends DataType
    +
    The data type representing short values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ShortType

        +
        public ShortType()
        +
      • +
      +
    • +
    + + +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StringType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StringType.html new file mode 100644 index 00000000000..738548bbeb5 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StringType.html @@ -0,0 +1,248 @@ + + + + + +StringType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StringType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StringType
    +extends DataType
    +
    The data type representing String values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StringType

        +
        public StringType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructField.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructField.html new file mode 100644 index 00000000000..a1fd1b34de6 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructField.html @@ -0,0 +1,400 @@ + + + + + +StructField (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructField

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.types.StructField
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public final class StructField
    +extends Object
    +
    A field inside a StructType.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType)
        +
        Constructor with default nullable = true.
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        +
      • +
      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType,
        +                   boolean nullable)
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        nullable - indicates if values of this field can be null values
        +
        +
      • +
      + + + +
        +
      • +

        StructField

        +
        public StructField(String name,
        +                   DataType dataType,
        +                   boolean nullable,
        +                   FieldMetadata metadata)
        +
        +
        Parameters:
        +
        name - the name of this field
        +
        dataType - the data type of this field
        +
        nullable - indicates if values of this field can be null values
        +
        metadata - metadata for this field
        +
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        getName

        +
        public String getName()
        +
      • +
      + + + +
        +
      • +

        getDataType

        +
        public DataType getDataType()
        +
      • +
      + + + +
        +
      • +

        isNullable

        +
        public boolean isNullable()
        +
      • +
      + + + + + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class Object
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class Object
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructType.html new file mode 100644 index 00000000000..012c155d2d0 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/StructType.html @@ -0,0 +1,494 @@ + + + + + +StructType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class StructType

+
+
+ +
+
    +
  • +
    +
    +
    public final class StructType
    +extends DataType
    +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    +
    See Also:
    +
    StructField
    +
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        StructType

        +
        public StructType()
        +
      • +
      + + + +
        +
      • +

        StructType

        +
        public StructType(StructField[] fields)
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        add

        +
        public StructType add(StructField field)
        +
        Creates a new StructType by adding a new field. + +
        
        + StructType schema = new StructType()
        +     .add(new StructField("a", new IntegerType(), true))
        +     .add(new StructField("b", new LongType(), false))
        +     .add(new StructField("c", new StringType(), true))
        + 
        +
        +
        Parameters:
        +
        field - The new field to add.
        +
        +
      • +
      + + + +
        +
      • +

        add

        +
        public StructType add(String fieldName,
        +                      DataType dataType)
        +
        Creates a new StructType by adding a new nullable field with no metadata. + +
        
        + StructType schema = new StructType()
        +     .add("a", new IntegerType())
        +     .add("b", new LongType())
        +     .add("c", new StringType())
        + 
        +
        +
        Parameters:
        +
        fieldName - The name of the new field.
        +
        dataType - The datatype for the new field.
        +
        +
      • +
      + + + +
        +
      • +

        add

        +
        public StructType add(String fieldName,
        +                      DataType dataType,
        +                      boolean nullable)
        +
        Creates a new StructType by adding a new field with no metadata. + +
        
        + StructType schema = new StructType()
        +     .add("a", new IntegerType(), true)
        +     .add("b", new LongType(), false)
        +     .add("c", new StringType(), true)
        + 
        +
        +
        Parameters:
        +
        fieldName - The name of the new field.
        +
        dataType - The datatype for the new field.
        +
        nullable - Whether or not the new field is nullable.
        +
        +
      • +
      + + + + + + + +
        +
      • +

        getFieldNames

        +
        public String[] getFieldNames()
        +
      • +
      + + + +
        +
      • +

        length

        +
        public int length()
        +
      • +
      + + + +
        +
      • +

        get

        +
        public StructField get(String fieldName)
        +
        +
        Parameters:
        +
        fieldName - the name of the desired StructField, not null
        +
        Throws:
        +
        IllegalArgumentException - if a field with the given name does not exist
        +
        +
      • +
      + + + +
        +
      • +

        column

        +
        public Column column(String fieldName)
        +
      • +
      + + + +
        +
      • +

        getTreeString

        +
        public String getTreeString()
        +
      • +
      + + + +
        +
      • +

        equals

        +
        public boolean equals(Object o)
        +
        +
        Overrides:
        +
        equals in class DataType
        +
        +
      • +
      + + + +
        +
      • +

        hashCode

        +
        public int hashCode()
        +
        +
        Overrides:
        +
        hashCode in class DataType
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html new file mode 100644 index 00000000000..9bb4a12baa0 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/TimestampType.html @@ -0,0 +1,248 @@ + + + + + +TimestampType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.types
+

Class TimestampType

+
+
+ +
+
    +
  • +
    +
    +
    public final class TimestampType
    +extends DataType
    +
    The data type representing java.sql.Timestamp values.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        TimestampType

        +
        public TimestampType()
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-frame.html new file mode 100644 index 00000000000..a0422c02fe9 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-frame.html @@ -0,0 +1,39 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.types

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-summary.html new file mode 100644 index 00000000000..33a9d89a40e --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-summary.html @@ -0,0 +1,257 @@ + + + + + +io.delta.standalone.types (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.types

+
+
+
    +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Class Summary 
    ClassDescription
    ArrayType +
    The data type for collections of multiple values.
    +
    BinaryType +
    The data type representing byte[] values.
    +
    BooleanType +
    The data type representing boolean values.
    +
    ByteType +
    The data type representing byte values.
    +
    DataType +
    The base type of all io.delta.standalone data types.
    +
    DateType +
    A date type, supporting "0001-01-01" through "9999-12-31".
    +
    DecimalType +
    The data type representing java.math.BigDecimal values.
    +
    DoubleType +
    The data type representing double values.
    +
    FieldMetadata +
    The metadata for a given StructField.
    +
    FieldMetadata.Builder +
    Builder class for FieldMetadata.
    +
    FloatType +
    The data type representing float values.
    +
    IntegerType +
    The data type representing int values.
    +
    LongType +
    The data type representing long values.
    +
    MapType +
    The data type for Maps.
    +
    NullType +
    The data type representing null values.
    +
    ShortType +
    The data type representing short values.
    +
    StringType +
    The data type representing String values.
    +
    StructField +
    A field inside a StructType.
    +
    StructType +
    The data type representing a table's schema, consisting of a collection of + fields (that is, fieldName to dataType pairs).
    +
    TimestampType +
    The data type representing java.sql.Timestamp values.
    +
    +
  • +
+
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-tree.html new file mode 100644 index 00000000000..a19f5ab3e72 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/types/package-tree.html @@ -0,0 +1,157 @@ + + + + + +io.delta.standalone.types Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.types

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.ParquetOutputTimestampType.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.ParquetOutputTimestampType.html new file mode 100644 index 00000000000..07261d0131f --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.ParquetOutputTimestampType.html @@ -0,0 +1,357 @@ + + + + + +ParquetSchemaConverter.ParquetOutputTimestampType (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.util
+

Enum ParquetSchemaConverter.ParquetOutputTimestampType

+
+
+ +
+
    +
  • +
    +
    All Implemented Interfaces:
    +
    java.io.Serializable, Comparable<ParquetSchemaConverter.ParquetOutputTimestampType>
    +
    +
    +
    Enclosing class:
    +
    ParquetSchemaConverter
    +
    +
    +
    +
    public static enum ParquetSchemaConverter.ParquetOutputTimestampType
    +extends Enum<ParquetSchemaConverter.ParquetOutputTimestampType>
    +
    Represents Parquet timestamp types. +
      +
    • INT96 is a non-standard but commonly used timestamp type in Parquet.
    • +
    • TIMESTAMP_MICROS is a standard timestamp type in Parquet, which stores number of + microseconds from the Unix epoch.
    • +
    • TIMESTAMP_MILLIS is also standard, but with millisecond precision, which means the + microsecond portion of the timestamp value is truncated.
    • +
    +
  • +
+
+
+ +
+
+ +
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.html new file mode 100644 index 00000000000..c660753da20 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/ParquetSchemaConverter.html @@ -0,0 +1,384 @@ + + + + + +ParquetSchemaConverter (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + + +
+
io.delta.standalone.util
+

Class ParquetSchemaConverter

+
+
+
    +
  • Object
  • +
  • +
      +
    • io.delta.standalone.util.ParquetSchemaConverter
    • +
    +
  • +
+
+
    +
  • +
    +
    +
    public class ParquetSchemaConverter
    +extends Object
    +
    Converter class to convert StructType to Parquet MessageType.
    +
  • +
+
+
+ +
+
+
    +
  • + +
      +
    • + + +

      Constructor Detail

      + + + +
        +
      • +

        ParquetSchemaConverter

        +
        public ParquetSchemaConverter()
        +
      • +
      +
    • +
    + +
      +
    • + + +

      Method Detail

      + + + +
        +
      • +

        sparkToParquet

        +
        public static org.apache.parquet.schema.MessageType sparkToParquet(StructType schema)
        +
        Convert a StructType to Parquet MessageType.
        +
        +
        Parameters:
        +
        schema - the schema to convert
        +
        Throws:
        +
        IllegalArgumentException - if a StructField name contains invalid character(s)
        +
        +
      • +
      + + + +
        +
      • +

        sparkToParquet

        +
        public static org.apache.parquet.schema.MessageType sparkToParquet(StructType schema,
        +                                                                   Boolean writeLegacyParquetFormat)
        +
        Convert a StructType to Parquet MessageType.
        +
        +
        Parameters:
        +
        schema - the schema to convert
        +
        writeLegacyParquetFormat - Whether to use legacy Parquet format compatible with Spark + 1.4 and prior versions when converting a StructType to a Parquet + MessageType. When set to false, use standard format defined in parquet-format + spec.
        +
        Throws:
        +
        IllegalArgumentException - if a StructField name contains invalid character(s)
        +
        +
      • +
      + + + +
        +
      • +

        sparkToParquet

        +
        public static org.apache.parquet.schema.MessageType sparkToParquet(StructType schema,
        +                                                                   ParquetSchemaConverter.ParquetOutputTimestampType outputTimestampType)
        +
        Convert a StructType to Parquet MessageType.
        +
        +
        Parameters:
        +
        schema - the schema to convert
        +
        outputTimestampType - which parquet timestamp type to use when writing
        +
        Throws:
        +
        IllegalArgumentException - if a StructField name contains invalid character(s)
        +
        +
      • +
      + + + +
        +
      • +

        sparkToParquet

        +
        public static org.apache.parquet.schema.MessageType sparkToParquet(StructType schema,
        +                                                                   Boolean writeLegacyParquetFormat,
        +                                                                   ParquetSchemaConverter.ParquetOutputTimestampType outputTimestampType)
        +
        Convert a StructType to Parquet MessageType.
        +
        +
        Parameters:
        +
        schema - the schema to convert
        +
        writeLegacyParquetFormat - Whether to use legacy Parquet format compatible with Spark + 1.4 and prior versions when converting a StructType to a Parquet + MessageType. When set to false, use standard format defined in parquet-format + spec.
        +
        outputTimestampType - which parquet timestamp type to use when writing
        +
        Throws:
        +
        IllegalArgumentException - if a StructField name contains invalid character(s)
        +
        +
      • +
      +
    • +
    +
  • +
+
+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-frame.html new file mode 100644 index 00000000000..e1e0a07c097 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-frame.html @@ -0,0 +1,24 @@ + + + + + +io.delta.standalone.util (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + +

io.delta.standalone.util

+ + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-summary.html new file mode 100644 index 00000000000..09d56a49d67 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-summary.html @@ -0,0 +1,159 @@ + + + + + +io.delta.standalone.util (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Package io.delta.standalone.util

+
+
+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-tree.html new file mode 100644 index 00000000000..6e670fd383c --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/util/package-tree.html @@ -0,0 +1,147 @@ + + + + + +io.delta.standalone.util Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Hierarchy For Package io.delta.standalone.util

+Package Hierarchies: + +
+
+

Class Hierarchy

+ +

Enum Hierarchy

+ +
+ + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-frame.html new file mode 100644 index 00000000000..425d6c8779b --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-frame.html @@ -0,0 +1,28 @@ + + + + + +Overview List (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + +

 

+ + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-summary.html new file mode 100644 index 00000000000..7cf7306d5f0 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-summary.html @@ -0,0 +1,161 @@ + + + + + +Overview (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + + + +
+ + + + + + + +
+ + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-tree.html new file mode 100644 index 00000000000..92eb5527ccf --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/overview-tree.html @@ -0,0 +1,290 @@ + + + + + +Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + +
+ + + + + + + +
+ + + +
+

Class Hierarchy

+ +

Interface Hierarchy

+ +

Enum Hierarchy

+ +
+ +
+ + + + + + + +
+ + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/package-list b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/package-list new file mode 100644 index 00000000000..14c216e7f77 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/package-list @@ -0,0 +1,8 @@ +io.delta.standalone +io.delta.standalone.actions +io.delta.standalone.data +io.delta.standalone.exceptions +io.delta.standalone.expressions +io.delta.standalone.storage +io.delta.standalone.types +io.delta.standalone.util diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/script.js b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/script.js new file mode 100644 index 00000000000..b3463569314 --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/script.js @@ -0,0 +1,30 @@ +function show(type) +{ + count = 0; + for (var key in methods) { + var row = document.getElementById(key); + if ((methods[key] & type) != 0) { + row.style.display = ''; + row.className = (count++ % 2) ? rowColor : altColor; + } + else + row.style.display = 'none'; + } + updateTabs(type); +} + +function updateTabs(type) +{ + for (var value in tabs) { + var sNode = document.getElementById(tabs[value][0]); + var spanNode = sNode.firstChild; + if (value == type) { + sNode.className = activeTableTab; + spanNode.innerHTML = tabs[value][1]; + } + else { + sNode.className = tableTab; + spanNode.innerHTML = "" + tabs[value][1] + ""; + } + } +} diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/serialized-form.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/serialized-form.html new file mode 100644 index 00000000000..4df21cb709a --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/serialized-form.html @@ -0,0 +1,170 @@ + + + + + +Serialized Form (Delta Standalone Reader 0.2.1 JavaDoc) + + + + + + + + + + + +
+

Serialized Form

+
+ + + + + + + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/stylesheet.css b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/stylesheet.css new file mode 100644 index 00000000000..98055b22d6d --- /dev/null +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/stylesheet.css @@ -0,0 +1,574 @@ +/* Javadoc style sheet */ +/* +Overall document style +*/ + +@import url('resources/fonts/dejavu.css'); + +body { + background-color:#ffffff; + color:#353833; + font-family:'DejaVu Sans', Arial, Helvetica, sans-serif; + font-size:14px; + margin:0; +} +a:link, a:visited { + text-decoration:none; + color:#4A6782; +} +a:hover, a:focus { + text-decoration:none; + color:#bb7a2a; +} +a:active { + text-decoration:none; + color:#4A6782; +} +a[name] { + color:#353833; +} +a[name]:hover { + text-decoration:none; + color:#353833; +} +pre { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; +} +h1 { + font-size:20px; +} +h2 { + font-size:18px; +} +h3 { + font-size:16px; + font-style:italic; +} +h4 { + font-size:13px; +} +h5 { + font-size:12px; +} +h6 { + font-size:11px; +} +ul { + list-style-type:disc; +} +code, tt { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; + margin-top:8px; + line-height:1.4em; +} +dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + padding-top:4px; +} +table tr td dt code { + font-family:'DejaVu Sans Mono', monospace; + font-size:14px; + vertical-align:top; + padding-top:4px; +} +sup { + font-size:8px; +} +/* +Document title and Copyright styles +*/ +.clear { + clear:both; + height:0px; + overflow:hidden; +} +.aboutLanguage { + float:right; + padding:0px 21px; + font-size:11px; + z-index:200; + margin-top:-9px; +} +.legalCopy { + margin-left:.5em; +} +.bar a, .bar a:link, .bar a:visited, .bar a:active { + color:#FFFFFF; + text-decoration:none; +} +.bar a:hover, .bar a:focus { + color:#bb7a2a; +} +.tab { + background-color:#0066FF; + color:#ffffff; + padding:8px; + width:5em; + font-weight:bold; +} +/* +Navigation bar styles +*/ +.bar { + background-color:#4D7A97; + color:#FFFFFF; + padding:.8em .5em .4em .8em; + height:auto;/*height:1.8em;*/ + font-size:11px; + margin:0; +} +.topNav { + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.bottomNav { + margin-top:10px; + background-color:#4D7A97; + color:#FFFFFF; + float:left; + padding:0; + width:100%; + clear:right; + height:2.8em; + padding-top:10px; + overflow:hidden; + font-size:12px; +} +.subNav { + background-color:#dee3e9; + float:left; + width:100%; + overflow:hidden; + font-size:12px; +} +.subNav div { + clear:left; + float:left; + padding:0 0 5px 6px; + text-transform:uppercase; +} +ul.navList, ul.subNavList { + float:left; + margin:0 25px 0 0; + padding:0; +} +ul.navList li{ + list-style:none; + float:left; + padding: 5px 6px; + text-transform:uppercase; +} +ul.subNavList li{ + list-style:none; + float:left; +} +.topNav a:link, .topNav a:active, .topNav a:visited, .bottomNav a:link, .bottomNav a:active, .bottomNav a:visited { + color:#FFFFFF; + text-decoration:none; + text-transform:uppercase; +} +.topNav a:hover, .bottomNav a:hover { + text-decoration:none; + color:#bb7a2a; + text-transform:uppercase; +} +.navBarCell1Rev { + background-color:#F8981D; + color:#253441; + margin: auto 5px; +} +.skipNav { + position:absolute; + top:auto; + left:-9999px; + overflow:hidden; +} +/* +Page header and footer styles +*/ +.header, .footer { + clear:both; + margin:0 20px; + padding:5px 0 0 0; +} +.indexHeader { + margin:10px; + position:relative; +} +.indexHeader span{ + margin-right:15px; +} +.indexHeader h1 { + font-size:13px; +} +.title { + color:#2c4557; + margin:10px 0; +} +.subTitle { + margin:5px 0 0 0; +} +.header ul { + margin:0 0 15px 0; + padding:0; +} +.footer ul { + margin:20px 0 5px 0; +} +.header ul li, .footer ul li { + list-style:none; + font-size:13px; +} +/* +Heading styles +*/ +div.details ul.blockList ul.blockList ul.blockList li.blockList h4, div.details ul.blockList ul.blockList ul.blockListLast li.blockList h4 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList ul.blockList li.blockList h3 { + background-color:#dee3e9; + border:1px solid #d0d9e0; + margin:0 0 6px -8px; + padding:7px 5px; +} +ul.blockList ul.blockList li.blockList h3 { + padding:0; + margin:15px 0; +} +ul.blockList li.blockList h2 { + padding:0px 0 20px 0; +} +/* +Page layout container styles +*/ +.contentContainer, .sourceContainer, .classUseContainer, .serializedFormContainer, .constantValuesContainer { + clear:both; + padding:10px 20px; + position:relative; +} +.indexContainer { + margin:10px; + position:relative; + font-size:12px; +} +.indexContainer h2 { + font-size:13px; + padding:0 0 3px 0; +} +.indexContainer ul { + margin:0; + padding:0; +} +.indexContainer ul li { + list-style:none; + padding-top:2px; +} +.contentContainer .description dl dt, .contentContainer .details dl dt, .serializedFormContainer dl dt { + font-size:12px; + font-weight:bold; + margin:10px 0 0 0; + color:#4E4E4E; +} +.contentContainer .description dl dd, .contentContainer .details dl dd, .serializedFormContainer dl dd { + margin:5px 0 10px 0px; + font-size:14px; + font-family:'DejaVu Sans Mono',monospace; +} +.serializedFormContainer dl.nameValue dt { + margin-left:1px; + font-size:1.1em; + display:inline; + font-weight:bold; +} +.serializedFormContainer dl.nameValue dd { + margin:0 0 0 1px; + font-size:1.1em; + display:inline; +} +/* +List styles +*/ +ul.horizontal li { + display:inline; + font-size:0.9em; +} +ul.inheritance { + margin:0; + padding:0; +} +ul.inheritance li { + display:inline; + list-style:none; +} +ul.inheritance li ul.inheritance { + margin-left:15px; + padding-left:15px; + padding-top:1px; +} +ul.blockList, ul.blockListLast { + margin:10px 0 10px 0; + padding:0; +} +ul.blockList li.blockList, ul.blockListLast li.blockList { + list-style:none; + margin-bottom:15px; + line-height:1.4; +} +ul.blockList ul.blockList li.blockList, ul.blockList ul.blockListLast li.blockList { + padding:0px 20px 5px 10px; + border:1px solid #ededed; + background-color:#f8f8f8; +} +ul.blockList ul.blockList ul.blockList li.blockList, ul.blockList ul.blockList ul.blockListLast li.blockList { + padding:0 0 5px 8px; + background-color:#ffffff; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockList { + margin-left:0; + padding-left:0; + padding-bottom:15px; + border:none; +} +ul.blockList ul.blockList ul.blockList ul.blockList li.blockListLast { + list-style:none; + border-bottom:none; + padding-bottom:0; +} +table tr td dl, table tr td dl dt, table tr td dl dd { + margin-top:0; + margin-bottom:1px; +} +/* +Table styles +*/ +.overviewSummary, .memberSummary, .typeSummary, .useSummary, .constantsSummary, .deprecatedSummary { + width:100%; + border-left:1px solid #EEE; + border-right:1px solid #EEE; + border-bottom:1px solid #EEE; +} +.overviewSummary, .memberSummary { + padding:0px; +} +.overviewSummary caption, .memberSummary caption, .typeSummary caption, +.useSummary caption, .constantsSummary caption, .deprecatedSummary caption { + position:relative; + text-align:left; + background-repeat:no-repeat; + color:#253441; + font-weight:bold; + clear:none; + overflow:hidden; + padding:0px; + padding-top:10px; + padding-left:1px; + margin:0px; + white-space:pre; +} +.overviewSummary caption a:link, .memberSummary caption a:link, .typeSummary caption a:link, +.useSummary caption a:link, .constantsSummary caption a:link, .deprecatedSummary caption a:link, +.overviewSummary caption a:hover, .memberSummary caption a:hover, .typeSummary caption a:hover, +.useSummary caption a:hover, .constantsSummary caption a:hover, .deprecatedSummary caption a:hover, +.overviewSummary caption a:active, .memberSummary caption a:active, .typeSummary caption a:active, +.useSummary caption a:active, .constantsSummary caption a:active, .deprecatedSummary caption a:active, +.overviewSummary caption a:visited, .memberSummary caption a:visited, .typeSummary caption a:visited, +.useSummary caption a:visited, .constantsSummary caption a:visited, .deprecatedSummary caption a:visited { + color:#FFFFFF; +} +.overviewSummary caption span, .memberSummary caption span, .typeSummary caption span, +.useSummary caption span, .constantsSummary caption span, .deprecatedSummary caption span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + padding-bottom:7px; + display:inline-block; + float:left; + background-color:#F8981D; + border: none; + height:16px; +} +.memberSummary caption span.activeTableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#F8981D; + height:16px; +} +.memberSummary caption span.tableTab span { + white-space:nowrap; + padding-top:5px; + padding-left:12px; + padding-right:12px; + margin-right:3px; + display:inline-block; + float:left; + background-color:#4D7A97; + height:16px; +} +.memberSummary caption span.tableTab, .memberSummary caption span.activeTableTab { + padding-top:0px; + padding-left:0px; + padding-right:0px; + background-image:none; + float:none; + display:inline; +} +.overviewSummary .tabEnd, .memberSummary .tabEnd, .typeSummary .tabEnd, +.useSummary .tabEnd, .constantsSummary .tabEnd, .deprecatedSummary .tabEnd { + display:none; + width:5px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .activeTableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + float:left; + background-color:#F8981D; +} +.memberSummary .tableTab .tabEnd { + display:none; + width:5px; + margin-right:3px; + position:relative; + background-color:#4D7A97; + float:left; + +} +.overviewSummary td, .memberSummary td, .typeSummary td, +.useSummary td, .constantsSummary td, .deprecatedSummary td { + text-align:left; + padding:0px 0px 12px 10px; +} +th.colOne, th.colFirst, th.colLast, .useSummary th, .constantsSummary th, +td.colOne, td.colFirst, td.colLast, .useSummary td, .constantsSummary td{ + vertical-align:top; + padding-right:0px; + padding-top:8px; + padding-bottom:3px; +} +th.colFirst, th.colLast, th.colOne, .constantsSummary th { + background:#dee3e9; + text-align:left; + padding:8px 3px 3px 7px; +} +td.colFirst, th.colFirst { + white-space:nowrap; + font-size:13px; +} +td.colLast, th.colLast { + font-size:13px; +} +td.colOne, th.colOne { + font-size:13px; +} +.overviewSummary td.colFirst, .overviewSummary th.colFirst, +.useSummary td.colFirst, .useSummary th.colFirst, +.overviewSummary td.colOne, .overviewSummary th.colOne, +.memberSummary td.colFirst, .memberSummary th.colFirst, +.memberSummary td.colOne, .memberSummary th.colOne, +.typeSummary td.colFirst{ + width:25%; + vertical-align:top; +} +td.colOne a:link, td.colOne a:active, td.colOne a:visited, td.colOne a:hover, td.colFirst a:link, td.colFirst a:active, td.colFirst a:visited, td.colFirst a:hover, td.colLast a:link, td.colLast a:active, td.colLast a:visited, td.colLast a:hover, .constantValuesContainer td a:link, .constantValuesContainer td a:active, .constantValuesContainer td a:visited, .constantValuesContainer td a:hover { + font-weight:bold; +} +.tableSubHeadingColor { + background-color:#EEEEFF; +} +.altColor { + background-color:#FFFFFF; +} +.rowColor { + background-color:#EEEEEF; +} +/* +Content styles +*/ +.description pre { + margin-top:0; +} +.deprecatedContent { + margin:0; + padding:10px 0; +} +.docSummary { + padding:0; +} + +ul.blockList ul.blockList ul.blockList li.blockList h3 { + font-style:normal; +} + +div.block { + font-size:14px; + font-family:'DejaVu Serif', Georgia, "Times New Roman", Times, serif; +} + +td.colLast div { + padding-top:0px; +} + + +td.colLast a { + padding-bottom:3px; +} +/* +Formatting effect styles +*/ +.sourceLineNo { + color:green; + padding:0 30px 0 0; +} +h1.hidden { + visibility:hidden; + overflow:hidden; + font-size:10px; +} +.block { + display:block; + margin:3px 10px 2px 0px; + color:#474747; +} +.deprecatedLabel, .descfrmTypeLabel, .memberNameLabel, .memberNameLink, +.overrideSpecifyLabel, .packageHierarchyLabel, .paramLabel, .returnLabel, +.seeLabel, .simpleTagLabel, .throwsLabel, .typeNameLabel, .typeNameLink { + font-weight:bold; +} +.deprecationComment, .emphasizedPhrase, .interfaceName { + font-style:italic; +} + +div.block div.block span.deprecationComment, div.block div.block span.emphasizedPhrase, +div.block div.block span.interfaceName { + font-style:normal; +} + +div.contentContainer ul.blockList li.blockList h2{ + padding-bottom:0px; +} From cdfd7fc459a1319db338f3a5533dc75d36c302ab Mon Sep 17 00:00:00 2001 From: Scott Sandre <59617782+scottsand-db@users.noreply.github.com> Date: Wed, 10 Nov 2021 09:14:22 -0800 Subject: [PATCH 130/291] Add `remove` API to AddFile (#201) --- .../io/delta/standalone/actions/Action.java | 2 +- .../delta/standalone/actions/AddCDCFile.java | 7 +- .../io/delta/standalone/actions/AddFile.java | 50 ++++++- .../delta/standalone/actions/CommitInfo.java | 135 +++++++++++------- .../io/delta/standalone/actions/JobInfo.java | 8 +- .../io/delta/standalone/actions/Metadata.java | 77 ++++++---- .../io/delta/standalone/actions/Protocol.java | 2 +- .../delta/standalone/actions/RemoveFile.java | 101 +++---------- .../standalone/actions/SetTransaction.java | 6 +- .../internal/ActionBuildersSuite.scala | 33 +---- .../standalone/internal/DeltaLogSuite.scala | 15 +- .../OptimisticTransactionSuiteTestVals.scala | 6 +- 12 files changed, 227 insertions(+), 215 deletions(-) diff --git a/standalone/src/main/java/io/delta/standalone/actions/Action.java b/standalone/src/main/java/io/delta/standalone/actions/Action.java index 8126df3876a..8c79442d597 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Action.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Action.java @@ -19,7 +19,7 @@ /** * A marker interface for all Actions that can be applied to a Delta Table. * Each action represents a single change to the state of a Delta table. - * + *

* You can use the following code to extract the concrete type of an {@link Action}. *

{@code
  *   // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
index 14d0424d85c..f790582076b 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/AddCDCFile.java
@@ -38,8 +38,11 @@ public final class AddCDCFile implements FileAction {
     @Nullable
     private final Map tags;
 
-    public AddCDCFile(@Nonnull String path, @Nonnull Map partitionValues, long size,
-                      @Nullable Map tags) {
+    public AddCDCFile(
+            @Nonnull String path,
+            @Nonnull Map partitionValues,
+            long size,
+            @Nullable Map tags) {
         this.path = path;
         this.partitionValues = partitionValues;
         this.size = size;
diff --git a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
index b5c9662270f..73a74a784b8 100644
--- a/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
+++ b/standalone/src/main/java/io/delta/standalone/actions/AddFile.java
@@ -19,13 +19,14 @@
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 /**
  * Represents an action that adds a new file to the table. The path of a file acts as the primary
  * key for the entry in the set of files.
- *
+ * 

* Note: since actions within a given Delta file are not guaranteed to be applied in order, it is * not valid for multiple file operations with the same path to exist in a single version. * @@ -50,9 +51,14 @@ public final class AddFile implements FileAction { @Nullable private final Map tags; - public AddFile(@Nonnull String path, @Nonnull Map partitionValues, long size, - long modificationTime, boolean dataChange, @Nullable String stats, - @Nullable Map tags) { + public AddFile( + @Nonnull String path, + @Nonnull Map partitionValues, + long size, + long modificationTime, + boolean dataChange, + @Nullable String stats, + @Nullable Map tags) { this.path = path; this.partitionValues = partitionValues; this.size = size; @@ -62,6 +68,42 @@ public AddFile(@Nonnull String path, @Nonnull Map partitionValue this.tags = tags; } + /** + * @return the corresponding {@link RemoveFile} for this file + */ + @Nonnull + public RemoveFile remove() { + return remove(System.currentTimeMillis(), dataChange); + } + + /** + * @return the corresponding {@link RemoveFile} for this file, instantiated with the given + * {@code deletionTimestamp} + */ + @Nonnull + public RemoveFile remove(long deletionTimestamp) { + return remove(deletionTimestamp, dataChange); + } + + /** + * @return the corresponding {@link RemoveFile} for this file, instantiated with the given + * {@code dataChange} flag + */ + @Nonnull + public RemoveFile remove(boolean dataChange) { + return remove(System.currentTimeMillis(), dataChange); + } + + /** + * @return the corresponding {@link RemoveFile} for this file, instantiated with the given + * {@code deletionTimestamp} value and {@code dataChange} flag + */ + @Nonnull + public RemoveFile remove(long deletionTimestamp, boolean dataChange) { + return new RemoveFile(path, Optional.of(deletionTimestamp), dataChange, true, + partitionValues, size, tags); + } + /** * @return the relative path or the absolute path that should be added to the table. If it's a * relative path, it's relative to the root of the table. Note: the path is encoded and diff --git a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java index e24597b7f63..635e57fb02e 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/CommitInfo.java @@ -21,6 +21,8 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; /** * Holds provenance information about changes to the table. This CommitInfo @@ -30,31 +32,38 @@ * @see Delta Transaction Log Protocol */ public class CommitInfo implements Action { - private final Optional version; - private final Timestamp timestamp; - private final Optional userId; - private final Optional userName; - private final String operation; - private final Map operationParameters; - private final Optional jobInfo; - private final Optional notebookInfo; - private final Optional clusterId; - private final Optional readVersion; - private final Optional isolationLevel; - private final Optional isBlindAppend; - private final Optional> operationMetrics; - private final Optional userMetadata; - private final Optional engineInfo; + @Nonnull private final Optional version; + @Nullable private final Timestamp timestamp; + @Nonnull private final Optional userId; + @Nonnull private final Optional userName; + @Nullable private final String operation; + @Nullable private final Map operationParameters; + @Nonnull private final Optional jobInfo; + @Nonnull private final Optional notebookInfo; + @Nonnull private final Optional clusterId; + @Nonnull private final Optional readVersion; + @Nonnull private final Optional isolationLevel; + @Nonnull private final Optional isBlindAppend; + @Nonnull private final Optional> operationMetrics; + @Nonnull private final Optional userMetadata; + @Nonnull private final Optional engineInfo; // For binary compatibility with version 0.2.0 - public CommitInfo(Optional version, Timestamp timestamp, Optional userId, - Optional userName, String operation, - Map operationParameters, Optional jobInfo, - Optional notebookInfo, Optional clusterId, - Optional readVersion, Optional isolationLevel, - Optional isBlindAppend, - Optional> operationMetrics, - Optional userMetadata) { + public CommitInfo( + @Nonnull Optional version, + @Nullable Timestamp timestamp, + @Nonnull Optional userId, + @Nonnull Optional userName, + @Nullable String operation, + @Nullable Map operationParameters, + @Nonnull Optional jobInfo, + @Nonnull Optional notebookInfo, + @Nonnull Optional clusterId, + @Nonnull Optional readVersion, + @Nonnull Optional isolationLevel, + @Nonnull Optional isBlindAppend, + @Nonnull Optional> operationMetrics, + @Nonnull Optional userMetadata) { this.version = version; this.timestamp = timestamp; this.userId = userId; @@ -72,14 +81,22 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional this.engineInfo = Optional.empty(); } - public CommitInfo(Optional version, Timestamp timestamp, Optional userId, - Optional userName, String operation, - Map operationParameters, Optional jobInfo, - Optional notebookInfo, Optional clusterId, - Optional readVersion, Optional isolationLevel, - Optional isBlindAppend, - Optional> operationMetrics, - Optional userMetadata, Optional engineInfo) { + public CommitInfo( + @Nonnull Optional version, + @Nullable Timestamp timestamp, + @Nonnull Optional userId, + @Nonnull Optional userName, + @Nullable String operation, + @Nullable Map operationParameters, + @Nonnull Optional jobInfo, + @Nonnull Optional notebookInfo, + @Nonnull Optional clusterId, + @Nonnull Optional readVersion, + @Nonnull Optional isolationLevel, + @Nonnull Optional isBlindAppend, + @Nonnull Optional> operationMetrics, + @Nonnull Optional userMetadata, + @Nonnull Optional engineInfo) { this.version = version; this.timestamp = timestamp; this.userId = userId; @@ -100,6 +117,7 @@ public CommitInfo(Optional version, Timestamp timestamp, Optional /** * @return the log version for this commit */ + @Nonnull public Optional getVersion() { return version; } @@ -107,6 +125,7 @@ public Optional getVersion() { /** * @return the time the files in this commit were committed */ + @Nullable public Timestamp getTimestamp() { return timestamp; } @@ -114,6 +133,7 @@ public Timestamp getTimestamp() { /** * @return the userId of the user who committed this file */ + @Nonnull public Optional getUserId() { return userId; } @@ -121,6 +141,7 @@ public Optional getUserId() { /** * @return the userName of the user who committed this file */ + @Nonnull public Optional getUserName() { return userName; } @@ -128,6 +149,7 @@ public Optional getUserName() { /** * @return the type of operation for this commit. e.g. "WRITE" */ + @Nullable public String getOperation() { return operation; } @@ -135,6 +157,7 @@ public String getOperation() { /** * @return any relevant operation parameters. e.g. "mode", "partitionBy" */ + @Nullable public Map getOperationParameters() { if (operationParameters != null) return Collections.unmodifiableMap(operationParameters); return null; @@ -143,6 +166,7 @@ public Map getOperationParameters() { /** * @return the JobInfo for this commit */ + @Nonnull public Optional getJobInfo() { return jobInfo; } @@ -150,6 +174,7 @@ public Optional getJobInfo() { /** * @return the NotebookInfo for this commit */ + @Nonnull public Optional getNotebookInfo() { return notebookInfo; } @@ -157,6 +182,7 @@ public Optional getNotebookInfo() { /** * @return the ID of the cluster used to generate this commit */ + @Nonnull public Optional getClusterId() { return clusterId; } @@ -164,6 +190,7 @@ public Optional getClusterId() { /** * @return the version that the transaction used to generate this commit is reading from */ + @Nonnull public Optional getReadVersion() { return readVersion; } @@ -171,6 +198,7 @@ public Optional getReadVersion() { /** * @return the isolation level at which this commit was generated */ + @Nonnull public Optional getIsolationLevel() { return isolationLevel; } @@ -178,6 +206,7 @@ public Optional getIsolationLevel() { /** * @return whether this commit has blindly appended without caring about existing files */ + @Nonnull public Optional getIsBlindAppend() { return isBlindAppend; } @@ -185,16 +214,15 @@ public Optional getIsBlindAppend() { /** * @return any operation metrics calculated */ + @Nonnull public Optional> getOperationMetrics() { - if (operationMetrics.isPresent()) { - return Optional.of(Collections.unmodifiableMap(operationMetrics.get())); - } - return operationMetrics; + return operationMetrics.map(Collections::unmodifiableMap); } /** * @return any additional user metadata */ + @Nonnull public Optional getUserMetadata() { return userMetadata; } @@ -203,6 +231,7 @@ public Optional getUserMetadata() { * @return the engineInfo of the operation that performed this commit. It should be of the form * "{engineName}/{engineVersion} Delta-Standalone/{deltaStandaloneVersion}" */ + @Nonnull public Optional getEngineInfo() { return engineInfo; } @@ -247,33 +276,33 @@ public static Builder builder() { * Builder class for CommitInfo. Enables construction of CommitInfo object with default values. */ public static class Builder { - private Optional version = Optional.empty(); - private Timestamp timestamp; - private Optional userId = Optional.empty(); - private Optional userName = Optional.empty(); - private String operation; - private Map operationParameters; - private Optional jobInfo = Optional.empty(); - private Optional notebookInfo = Optional.empty(); - private Optional clusterId = Optional.empty(); - private Optional readVersion = Optional.empty(); - private Optional isolationLevel = Optional.empty(); - private Optional isBlindAppend = Optional.empty(); - private Optional> operationMetrics = Optional.empty(); - private Optional userMetadata = Optional.empty(); - private Optional engineInfo = Optional.empty(); + @Nonnull private Optional version = Optional.empty(); + @Nullable private Timestamp timestamp; + @Nonnull private Optional userId = Optional.empty(); + @Nonnull private Optional userName = Optional.empty(); + @Nullable private String operation; + @Nullable private Map operationParameters; + @Nonnull private Optional jobInfo = Optional.empty(); + @Nonnull private Optional notebookInfo = Optional.empty(); + @Nonnull private Optional clusterId = Optional.empty(); + @Nonnull private Optional readVersion = Optional.empty(); + @Nonnull private Optional isolationLevel = Optional.empty(); + @Nonnull private Optional isBlindAppend = Optional.empty(); + @Nonnull private Optional> operationMetrics = Optional.empty(); + @Nonnull private Optional userMetadata = Optional.empty(); + @Nonnull private Optional engineInfo = Optional.empty(); public Builder version(Long version) { this.version = Optional.of(version); return this; } - public Builder timestamp(Timestamp timestamp) { + public Builder timestamp(@Nullable Timestamp timestamp) { this.timestamp = timestamp; return this; } - public Builder userId(String userId) { + public Builder userId(@Nullable String userId) { this.userId = Optional.of(userId); return this; } @@ -288,7 +317,7 @@ public Builder operation(String operation) { return this; } - public Builder operationParameters(Map operationParameters) { + public Builder operationParameters(@Nullable Map operationParameters) { this.operationParameters = operationParameters; return this; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java index 0e3e4678b0f..67d93185cb7 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java +++ b/standalone/src/main/java/io/delta/standalone/actions/JobInfo.java @@ -26,8 +26,12 @@ public class JobInfo { private final String jobOwnerId; private final String triggerType; - public JobInfo(String jobId, String jobName, String runId, String jobOwnerId, - String triggerType) { + public JobInfo( + String jobId, + String jobName, + String runId, + String jobOwnerId, + String triggerType) { this.jobId = jobId; this.jobName = jobName; this.runId = runId; diff --git a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java index cd1b7014ee8..734ea127a62 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Metadata.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Metadata.java @@ -22,6 +22,9 @@ import java.util.Objects; import java.util.Optional; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + import io.delta.standalone.types.StructType; /** @@ -35,18 +38,24 @@ * @see Delta Transaction Log Protocol */ public final class Metadata implements Action { - private final String id; - private final String name; - private final String description; - private final Format format; - private final List partitionColumns; - private final Map configuration; - private final Optional createdTime; - private final StructType schema; - - public Metadata(String id, String name, String description, Format format, - List partitionColumns, Map configuration, - Optional createdTime, StructType schema) { + @Nonnull private final String id; + @Nullable private final String name; + @Nullable private final String description; + @Nonnull private final Format format; + @Nonnull private final List partitionColumns; + @Nonnull private final Map configuration; + @Nonnull private final Optional createdTime; + @Nullable private final StructType schema; + + public Metadata( + @Nonnull String id, + @Nullable String name, + @Nullable String description, + @Nonnull Format format, + @Nonnull List partitionColumns, + @Nonnull Map configuration, + @Nonnull Optional createdTime, + @Nullable StructType schema) { this.id = id; this.name = name; this.description = description; @@ -60,6 +69,7 @@ public Metadata(String id, String name, String description, Format format, /** * @return the unique identifier for this table */ + @Nonnull public String getId() { return id; } @@ -67,6 +77,7 @@ public String getId() { /** * @return the user-provided identifier for this table */ + @Nullable public String getName() { return name; } @@ -74,6 +85,7 @@ public String getName() { /** * @return the user-provided description for this table */ + @Nullable public String getDescription() { return description; } @@ -81,6 +93,7 @@ public String getDescription() { /** * @return the {@link Format} for this table */ + @Nonnull public Format getFormat() { return format; } @@ -89,22 +102,25 @@ public Format getFormat() { * @return an unmodifiable {@code java.util.List} containing the names of * columns by which the data should be partitioned */ + @Nonnull public List getPartitionColumns() { - return partitionColumns != null ? Collections.unmodifiableList(partitionColumns) : null; + return Collections.unmodifiableList(partitionColumns); } /** * @return an unmodifiable {@code java.util.Map} containing configuration * options for this metadata */ + @Nonnull public Map getConfiguration() { - return configuration != null ? Collections.unmodifiableMap(configuration) : null; + return Collections.unmodifiableMap(configuration); } /** * @return the time when this metadata action was created, in milliseconds * since the Unix epoch */ + @Nonnull public Optional getCreatedTime() { return createdTime; } @@ -112,6 +128,7 @@ public Optional getCreatedTime() { /** * @return the schema of the table as a {@link StructType} */ + @Nullable public StructType getSchema() { return schema; } @@ -148,41 +165,41 @@ public static Builder builder() { * Builder class for Metadata. Enables construction of Metadata object with default values. */ public static class Builder { - private String id = java.util.UUID.randomUUID().toString(); - private String name; - private String description; - private Format format = new Format("parquet", Collections.emptyMap()); - private List partitionColumns = Collections.emptyList(); - private Map configuration = Collections.emptyMap(); - private Optional createdTime = Optional.of(System.currentTimeMillis()); - private StructType schema; - - public Builder id(String id) { + @Nonnull private String id = java.util.UUID.randomUUID().toString(); + @Nullable private String name; + @Nullable private String description; + @Nonnull private Format format = new Format("parquet", Collections.emptyMap()); + @Nonnull private List partitionColumns = Collections.emptyList(); + @Nonnull private Map configuration = Collections.emptyMap(); + @Nonnull private Optional createdTime = Optional.of(System.currentTimeMillis()); + @Nullable private StructType schema; + + public Builder id(@Nonnull String id) { this.id = id; return this; } - public Builder name(String name) { + public Builder name(@Nullable String name) { this.name = name; return this; } - public Builder description(String description) { + public Builder description(@Nullable String description) { this.description = description; return this; } - public Builder format(Format format) { + public Builder format(@Nonnull Format format) { this.format = format; return this; } - public Builder partitionColumns(List partitionColumns) { + public Builder partitionColumns(@Nonnull List partitionColumns) { this.partitionColumns = partitionColumns; return this; } - public Builder configuration(Map configuration) { + public Builder configuration(@Nonnull Map configuration) { this.configuration = configuration; return this; } @@ -192,7 +209,7 @@ public Builder createdTime(Long createdTime) { return this; } - public Builder schema(StructType schema) { + public Builder schema(@Nullable StructType schema) { this.schema = schema; return this; } diff --git a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java index 6c4fb4870fc..db2d77b6d18 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/Protocol.java +++ b/standalone/src/main/java/io/delta/standalone/actions/Protocol.java @@ -23,7 +23,7 @@ * incompatible changes are made to the protocol. Readers and writers are * responsible for checking that they meet the minimum versions before performing * any other operations. - * + *

* Since this action allows us to explicitly block older clients in the case of a * breaking change to the protocol, clients should be tolerant of messages and * fields that they do not understand. diff --git a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java index 0314977fe9c..5a9aa255edd 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java +++ b/standalone/src/main/java/io/delta/standalone/actions/RemoveFile.java @@ -26,11 +26,14 @@ /** * Logical removal of a given file from the reservoir. Acts as a tombstone before a file is * deleted permanently. - * - * Note that for protocol compatibility reasons, the fields {@code partitionValues}, {@code size}, - * and {@code tags} are only present when the extendedFileMetadata flag is true. New writers should - * generally be setting this flag, but old writers (and FSCK) won't, so readers must check this flag - * before attempting to consume those values. + *

+ * Note that users should onlu instantiate {@link RemoveFile} instances using one of the various + * {@link AddFile#remove()} methods. + *

+ * As well, note that for protocol compatibility reasons, the fields {@code partitionValues}, + * {@code size}, and {@code tags} are only present when the extendedFileMetadata flag is true. New + * writers should generally be setting this flag, but old writers (and FSCK) won't, so readers must + * check this flag before attempting to consume those values. */ public class RemoveFile implements FileAction { @Nonnull @@ -51,10 +54,19 @@ public class RemoveFile implements FileAction { @Nullable private final Map tags; - public RemoveFile(@Nonnull String path, @Nonnull Optional deletionTimestamp, - boolean dataChange, boolean extendedFileMetadata, - @Nullable Map partitionValues, long size, - @Nullable Map tags) { + /** + * Users should not construct {@link RemoveFile}s themselves, and should instead use one + * of the various {@link AddFile#remove()} methods to instantiate the correct {@link RemoveFile} + * for a given {@link AddFile} instance. + */ + public RemoveFile( + @Nonnull String path, + @Nonnull Optional deletionTimestamp, + boolean dataChange, + boolean extendedFileMetadata, + @Nullable Map partitionValues, + long size, + @Nullable Map tags) { this.path = path; this.deletionTimestamp = deletionTimestamp; this.dataChange = dataChange; @@ -144,75 +156,4 @@ public int hashCode() { return Objects.hash(path, deletionTimestamp, dataChange, extendedFileMetadata, partitionValues, size, tags); } - - /** - * @return a new {@code RemoveFile.Builder} - */ - public static Builder builder(String path) { - return new Builder(path); - } - - /** - * Builder class for RemoveFile. Enables construction of RemoveFile object with default values. - */ - public static class Builder { - // required RemoveFile fields - private final String path; - // optional RemoveFile fields - private Optional deletionTimestamp = Optional.empty(); - private boolean dataChange = true; - private boolean extendedFileMetadata = false; - private Map partitionValues; - private long size = 0; - private Map tags; - - public Builder(String path) { - this.path = path; - } - - public Builder deletionTimestamp(Long deletionTimestamp) { - this.deletionTimestamp = Optional.of(deletionTimestamp); - return this; - } - - public Builder dataChange(boolean dataChange) { - this.dataChange = dataChange; - return this; - } - - public Builder extendedFileMetadata(boolean extendedFileMetadata) { - this.extendedFileMetadata = extendedFileMetadata; - return this; - } - - public Builder partitionValues(Map partitionValues) { - this.partitionValues = partitionValues; - return this; - } - - public Builder size(long size) { - this.size = size; - return this; - } - - public Builder tags(Map tags) { - this.tags = tags; - return this; - } - - /** - * @return a new {@code RemoveFile} with the same properties as {@code this} - */ - public RemoveFile build() { - RemoveFile removeFile = new RemoveFile( - this.path, - this.deletionTimestamp, - this.dataChange, - this.extendedFileMetadata, - this.partitionValues, - this.size, - this.tags); - return removeFile; - } - } } diff --git a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java index 10612bafc32..de7db9cc709 100644 --- a/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java +++ b/standalone/src/main/java/io/delta/standalone/actions/SetTransaction.java @@ -32,8 +32,10 @@ public final class SetTransaction implements Action { @Nonnull private final Optional lastUpdated; - public SetTransaction(@Nonnull String appId, long version, - @Nonnull Optional lastUpdated) { + public SetTransaction( + @Nonnull String appId, + long version, + @Nonnull Optional lastUpdated) { this.appId = appId; this.version = version; this.lastUpdated = lastUpdated; diff --git a/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala index 07ee309b19d..d5f83beb07c 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/ActionBuildersSuite.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.scalatest.FunSuite -import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.actions.{AddFile => AddFileJ, CommitInfo => CommitInfoJ, Format => FormatJ, JobInfo => JobInfoJ, Metadata => MetadataJ, NotebookInfo => NotebookInfoJ} import io.delta.standalone.types.{IntegerType, StructField => StructFieldJ, StructType => StructTypeJ} class ActionBuildersSuite extends FunSuite { @@ -124,37 +124,6 @@ class ActionBuildersSuite extends FunSuite { assert(jobInfoFromBuilder == jobInfoFromConstructor) } - test("builder action class constructor for RemoveFile") { - val removeFileJFromBuilderDefaults = RemoveFileJ.builder("/test").build() - val removeFileJFromConstructorDefaults = new RemoveFileJ( - "/test", - Optional.empty(), - true, - false, - null, - 0L, - null) - assert(removeFileJFromBuilderDefaults == removeFileJFromConstructorDefaults) - - val removeFileJFromBuilder = RemoveFileJ.builder("/test") - .deletionTimestamp(0L) - .dataChange(false) - .extendedFileMetadata(true) - .partitionValues(Map("test"->"foo").asJava) - .size(1L) - .tags(Map("tag"->"foo").asJava) - .build() - val removeFileJFromConstructor = new RemoveFileJ( - "/test", - Optional.of(0L), - false, - true, - Map("test"->"foo").asJava, - 1L, - Map("tag"->"foo").asJava) - assert(removeFileJFromBuilder == removeFileJFromConstructor) - } - test("builder action class constructor for CommitInfo") { val commitInfoFromBuilderDefaults = CommitInfoJ.builder().build() val commitInfoFromConstructorDefaults = new CommitInfoJ( diff --git a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala index 8a78558244b..14ec833fb0c 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/DeltaLogSuite.scala @@ -255,11 +255,16 @@ abstract class DeltaLogSuiteBase extends FunSuite { assert(new File(log.logPath.toUri).mkdirs()) val path = new File(dir, "a/b/c").getCanonicalPath - val removeFile = RemoveFileJ - .builder(path) - .deletionTimestamp(System.currentTimeMillis()) - .dataChange(true) - .build() + val removeFile = new RemoveFileJ( + path, + java.util.Optional.of(System.currentTimeMillis()), + true, // dataChange + false, // extendedFileMetadata + null, // partitionValues + 0L, // size + null // null + ) + val metadata = MetadataJ.builder().build() val actions = java.util.Arrays.asList(removeFile, metadata) diff --git a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala index f828eec47fb..ca9a7bf8e06 100644 --- a/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala +++ b/standalone/src/test/scala/io/delta/standalone/internal/OptimisticTransactionSuiteTestVals.scala @@ -20,7 +20,7 @@ import java.util.Collections import scala.collection.JavaConverters._ -import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ, RemoveFile => RemoveFileJ} +import io.delta.standalone.actions.{AddFile => AddFileJ, Metadata => MetadataJ} import io.delta.standalone.expressions.{EqualTo, Literal} import io.delta.standalone.types.{IntegerType, StructField, StructType} @@ -33,8 +33,8 @@ trait OptimisticTransactionSuiteTestVals { val addA = new AddFileJ("a", Collections.emptyMap(), 1, 1, true, null, null) val addB = new AddFileJ("b", Collections.emptyMap(), 1, 1, true, null, null) - val removeA = RemoveFileJ.builder("a").deletionTimestamp(4L).build() - val removeA_time5 = RemoveFileJ.builder("a").deletionTimestamp(5L).build() + val removeA = addA.remove(4L) + val removeA_time5 = addA.remove(5L) val addA_partX1 = new AddFileJ("a", Map("x" -> "1").asJava, 1, 1, true, null, null) val addA_partX2 = new AddFileJ("a", Map("x" -> "2").asJava, 1, 1, true, null, null) From 53be736f515fd7d1296403ff0edd6427c6958254 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Wed, 10 Nov 2021 23:24:43 -0800 Subject: [PATCH 131/291] update package object path (#206) --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 54cca866819..b82988ab29c 100644 --- a/build.sbt +++ b/build.sbt @@ -430,7 +430,7 @@ lazy val standalone = (project in file("standalone")) "org.slf4j" % "slf4j-log4j12" % "1.7.25" ), sourceGenerators in Compile += Def.task { - val file = (sourceManaged in Compile).value / "meta" / "package.scala" + val file = (sourceManaged in Compile).value / "io" / "delta" / "standalone" / "package.scala" IO.write(file, s"""package io.delta | From d5f79f14f44c7a65f54887b4bccd3014fabb9b17 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Thu, 11 Nov 2021 15:01:53 -0800 Subject: [PATCH 132/291] Close ParquetIterable in SnapshotImpl.loadInMemory (#207) --- .../io/delta/standalone/internal/SnapshotImpl.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala index 5d64a5ed3d5..9f3bdc501f4 100644 --- a/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala +++ b/standalone/src/main/scala/io/delta/standalone/internal/SnapshotImpl.scala @@ -175,12 +175,17 @@ private[internal] class SnapshotImpl( .toArray .map { line => JsonUtils.mapper.readValue[SingleAction](line) } } else if (path.endsWith("parquet")) { - ParquetReader.read[Parquet4sSingleActionWrapper]( + val parquetIterable = ParquetReader.read[Parquet4sSingleActionWrapper]( path, ParquetReader.Options( timeZone = deltaLog.timezone, hadoopConf = hadoopConf) - ).toSeq.map(_.unwrap) + ) + try { + parquetIterable.toArray.map(_.unwrap) + } finally { + parquetIterable.close() + } } else Seq.empty[SingleAction] }.toList } From f0c41c01a21afb2341c433af42e36b2a190eda86 Mon Sep 17 00:00:00 2001 From: allisonport-db <89107911+allisonport-db@users.noreply.github.com> Date: Mon, 15 Nov 2021 11:36:40 -0800 Subject: [PATCH 133/291] javadocs with return statements' (#210) --- build.sbt | 7 +- .../api/java/allclasses-frame.html | 5 +- .../api/java/allclasses-noframe.html | 5 +- .../api/java/constant-values.html | 4 +- .../api/java/deprecated-list.html | 4 +- .../delta-standalone/api/java/help-doc.html | 4 +- .../delta-standalone/api/java/index-all.html | 36 +- .../delta-standalone/api/java/index.html | 2 +- .../io/delta/standalone/CommitResult.html | 8 +- .../java/io/delta/standalone/DeltaLog.html | 33 +- .../java/io/delta/standalone/DeltaScan.html | 24 +- .../delta/standalone/Operation.Metrics.html | 4 +- .../io/delta/standalone/Operation.Name.html | 10 +- .../java/io/delta/standalone/Operation.html | 20 +- .../standalone/OptimisticTransaction.html | 17 +- .../java/io/delta/standalone/Snapshot.html | 26 +- .../java/io/delta/standalone/VersionLog.html | 12 +- .../io/delta/standalone/actions/Action.html | 6 +- .../delta/standalone/actions/AddCDCFile.html | 22 +- .../standalone/actions/AddFile.Builder.html | 8 +- .../io/delta/standalone/actions/AddFile.html | 119 +++++- .../actions/CommitInfo.Builder.html | 17 +- .../delta/standalone/actions/CommitInfo.html | 147 +++++++- .../delta/standalone/actions/FileAction.html | 14 +- .../io/delta/standalone/actions/Format.html | 13 +- .../standalone/actions/JobInfo.Builder.html | 8 +- .../io/delta/standalone/actions/JobInfo.html | 8 +- .../standalone/actions/Metadata.Builder.html | 29 +- .../io/delta/standalone/actions/Metadata.html | 77 +++- .../standalone/actions/NotebookInfo.html | 4 +- .../io/delta/standalone/actions/Protocol.html | 6 +- .../actions/RemoveFile.Builder.html | 352 ------------------ .../delta/standalone/actions/RemoveFile.html | 121 +++--- .../standalone/actions/SetTransaction.html | 20 +- .../standalone/actions/package-frame.html | 5 +- .../standalone/actions/package-summary.html | 10 +- .../standalone/actions/package-tree.html | 5 +- .../standalone/data/CloseableIterator.html | 4 +- .../io/delta/standalone/data/RowRecord.html | 52 ++- .../delta/standalone/data/package-frame.html | 4 +- .../standalone/data/package-summary.html | 4 +- .../delta/standalone/data/package-tree.html | 4 +- .../exceptions/ConcurrentAppendException.html | 4 +- .../ConcurrentDeleteDeleteException.html | 4 +- .../ConcurrentDeleteReadException.html | 4 +- .../ConcurrentTransactionException.html | 4 +- .../DeltaConcurrentModificationException.html | 4 +- .../exceptions/DeltaStandaloneException.html | 4 +- .../exceptions/MetadataChangedException.html | 4 +- .../exceptions/ProtocolChangedException.html | 4 +- .../standalone/exceptions/package-frame.html | 4 +- .../exceptions/package-summary.html | 4 +- .../standalone/exceptions/package-tree.html | 4 +- .../io/delta/standalone/expressions/And.html | 4 +- .../expressions/BinaryComparison.html | 4 +- .../expressions/BinaryExpression.html | 8 +- .../expressions/BinaryOperator.html | 6 +- .../delta/standalone/expressions/Column.html | 14 +- .../delta/standalone/expressions/EqualTo.html | 4 +- .../standalone/expressions/Expression.html | 20 +- .../standalone/expressions/GreaterThan.html | 4 +- .../expressions/GreaterThanOrEqual.html | 4 +- .../io/delta/standalone/expressions/In.html | 10 +- .../standalone/expressions/IsNotNull.html | 8 +- .../delta/standalone/expressions/IsNull.html | 8 +- .../expressions/LeafExpression.html | 8 +- .../standalone/expressions/LessThan.html | 4 +- .../expressions/LessThanOrEqual.html | 4 +- .../delta/standalone/expressions/Literal.html | 12 +- .../io/delta/standalone/expressions/Not.html | 6 +- .../io/delta/standalone/expressions/Or.html | 4 +- .../standalone/expressions/Predicate.html | 6 +- .../expressions/UnaryExpression.html | 8 +- .../standalone/expressions/package-frame.html | 4 +- .../expressions/package-summary.html | 4 +- .../standalone/expressions/package-tree.html | 4 +- .../io/delta/standalone/package-frame.html | 4 +- .../io/delta/standalone/package-summary.html | 4 +- .../io/delta/standalone/package-tree.html | 4 +- .../io/delta/standalone/storage/LogStore.html | 17 +- .../standalone/storage/package-frame.html | 4 +- .../standalone/storage/package-summary.html | 4 +- .../standalone/storage/package-tree.html | 4 +- .../io/delta/standalone/types/ArrayType.html | 12 +- .../io/delta/standalone/types/BinaryType.html | 4 +- .../delta/standalone/types/BooleanType.html | 4 +- .../io/delta/standalone/types/ByteType.html | 6 +- .../io/delta/standalone/types/DataType.html | 24 +- .../io/delta/standalone/types/DateType.html | 4 +- .../delta/standalone/types/DecimalType.html | 12 +- .../io/delta/standalone/types/DoubleType.html | 4 +- .../types/FieldMetadata.Builder.html | 8 +- .../delta/standalone/types/FieldMetadata.html | 17 +- .../io/delta/standalone/types/FloatType.html | 4 +- .../delta/standalone/types/IntegerType.html | 6 +- .../io/delta/standalone/types/LongType.html | 6 +- .../io/delta/standalone/types/MapType.html | 16 +- .../io/delta/standalone/types/NullType.html | 4 +- .../io/delta/standalone/types/ShortType.html | 6 +- .../io/delta/standalone/types/StringType.html | 4 +- .../delta/standalone/types/StructField.html | 20 +- .../io/delta/standalone/types/StructType.html | 29 +- .../delta/standalone/types/TimestampType.html | 4 +- .../delta/standalone/types/package-frame.html | 4 +- .../standalone/types/package-summary.html | 4 +- .../delta/standalone/types/package-tree.html | 4 +- ...aConverter.ParquetOutputTimestampType.html | 10 +- .../util/ParquetSchemaConverter.html | 12 +- .../delta/standalone/util/package-frame.html | 4 +- .../standalone/util/package-summary.html | 4 +- .../delta/standalone/util/package-tree.html | 4 +- .../api/java/overview-frame.html | 4 +- .../api/java/overview-summary.html | 4 +- .../api/java/overview-tree.html | 5 +- .../api/java/serialized-form.html | 4 +- 115 files changed, 1071 insertions(+), 709 deletions(-) delete mode 100644 docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html diff --git a/build.sbt b/build.sbt index b82988ab29c..079b4270fa7 100644 --- a/build.sbt +++ b/build.sbt @@ -427,7 +427,11 @@ lazy val standalone = (project in file("standalone")) ), "org.scalatest" %% "scalatest" % scalaTestVersion % "test", "org.slf4j" % "slf4j-api" % "1.7.25", - "org.slf4j" % "slf4j-log4j12" % "1.7.25" + "org.slf4j" % "slf4j-log4j12" % "1.7.25", + + // Compiler plugins + // -- Bump up the genjavadoc version explicitly to 0.18 to work with Scala 2.12 + compilerPlugin("com.typesafe.genjavadoc" %% "genjavadoc-plugin" % "0.18" cross CrossVersion.full) ), sourceGenerators in Compile += Def.task { val file = (sourceManaged in Compile).value / "io" / "delta" / "standalone" / "package.scala" @@ -495,7 +499,6 @@ lazy val standalone = (project in file("standalone")) "-public", "-windowtitle", "Delta Standalone Reader " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", "-noqualifier", "java.lang", - "-tag", "return:X", // `doclint` is disabled on Circle CI. Need to enable it manually to test our javadoc. "-Xdoclint:all" ), diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-frame.html index a08fd15db92..4f840b63cdf 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-frame.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-frame.html @@ -2,9 +2,9 @@ - + All Classes (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -80,7 +80,6 @@

All Classes

  • Protocol
  • ProtocolChangedException
  • RemoveFile
  • -
  • RemoveFile.Builder
  • RowRecord
  • SetTransaction
  • ShortType
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html index f7c67cdedfd..e5d1db8cea5 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/allclasses-noframe.html @@ -2,9 +2,9 @@ - + All Classes (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -80,7 +80,6 @@

    All Classes

  • Protocol
  • ProtocolChangedException
  • RemoveFile
  • -
  • RemoveFile.Builder
  • RowRecord
  • SetTransaction
  • ShortType
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html index 74286369323..d8bd198c42b 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/constant-values.html @@ -2,9 +2,9 @@ - + Constant Field Values (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html index 556d32ddc7c..e893367fdfb 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/deprecated-list.html @@ -2,9 +2,9 @@ - + Deprecated List (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html index 02ce022ccdc..4a8ddef3777 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/help-doc.html @@ -2,9 +2,9 @@ - + API Help (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html index bb8cdff396f..571a92ce35c 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index-all.html @@ -2,9 +2,9 @@ - + Index (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -165,8 +165,6 @@

    B

     
    build() - Method in class io.delta.standalone.actions.Metadata.Builder
     
    -
    build() - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    build() - Method in class io.delta.standalone.types.FieldMetadata.Builder
     
    builder(String, Map<String, String>, long, long, boolean) - Static method in class io.delta.standalone.actions.AddFile
    @@ -185,10 +183,6 @@

    B

     
    Builder() - Constructor for class io.delta.standalone.actions.Metadata.Builder
     
    -
    builder(String) - Static method in class io.delta.standalone.actions.RemoveFile
    -
     
    -
    Builder(String) - Constructor for class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    builder() - Static method in class io.delta.standalone.types.FieldMetadata
     
    Builder() - Constructor for class io.delta.standalone.types.FieldMetadata.Builder
    @@ -290,8 +284,6 @@

    C

    D

    -
    dataChange(boolean) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    dataType() - Method in class io.delta.standalone.expressions.Column
     
    dataType() - Method in interface io.delta.standalone.expressions.Expression
    @@ -318,8 +310,6 @@

    D

    DecimalType(int, int) - Constructor for class io.delta.standalone.types.DecimalType
     
    -
    deletionTimestamp(Long) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    DeltaConcurrentModificationException - Exception in io.delta.standalone.exceptions
    The basic class for all Delta Standalone commit conflict exceptions.
    @@ -434,8 +424,6 @@

    E

    An expression in Delta Standalone.
    -
    extendedFileMetadata(boolean) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    @@ -1191,8 +1179,6 @@

    P

    partitionColumns(List<String>) - Method in class io.delta.standalone.actions.Metadata.Builder
     
    -
    partitionValues(Map<String, String>) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    Predicate - Interface in io.delta.standalone.expressions
     
    Protocol - Class in io.delta.standalone.actions
    @@ -1258,15 +1244,23 @@

    R

     
    references() - Method in class io.delta.standalone.expressions.LeafExpression
     
    +
    remove() - Method in class io.delta.standalone.actions.AddFile
    +
     
    +
    remove(long) - Method in class io.delta.standalone.actions.AddFile
    +
     
    +
    remove(boolean) - Method in class io.delta.standalone.actions.AddFile
    +
     
    +
    remove(long, boolean) - Method in class io.delta.standalone.actions.AddFile
    +
     
    RemoveFile - Class in io.delta.standalone.actions
    Logical removal of a given file from the reservoir.
    RemoveFile(String, Optional<Long>, boolean, boolean, Map<String, String>, long, Map<String, String>) - Constructor for class io.delta.standalone.actions.RemoveFile
    -
     
    -
    RemoveFile.Builder - Class in io.delta.standalone.actions
    -
    Builder class for RemoveFile.
    +
    Users should not construct RemoveFiles themselves, and should instead use one + of the various AddFile.remove() methods to instantiate the correct RemoveFile + for a given AddFile instance.
    resolvePathOnPhysicalStorage(Path, Configuration) - Method in class io.delta.standalone.storage.LogStore
    @@ -1310,8 +1304,6 @@

    S

    ShortType() - Constructor for class io.delta.standalone.types.ShortType
     
    -
    size(long) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    snapshot() - Method in interface io.delta.standalone.DeltaLog
     
    Snapshot - Interface in io.delta.standalone
    @@ -1377,8 +1369,6 @@

    T

    tags(Map<String, String>) - Method in class io.delta.standalone.actions.AddFile.Builder
     
    -
    tags(Map<String, String>) - Method in class io.delta.standalone.actions.RemoveFile.Builder
    -
     
    timestamp(Timestamp) - Method in class io.delta.standalone.actions.CommitInfo.Builder
     
    TimestampType - Class in io.delta.standalone.types
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html index 9858e9a615f..da75d26458e 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/index.html @@ -2,7 +2,7 @@ - + Delta Standalone Reader 0.2.1 JavaDoc @@ -196,6 +196,10 @@

    Method Detail

  • getVersion

    public long getVersion()
    +
    +
    Returns:
    +
    the table version that was committed.
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html index 808c8b49c67..8fdcab77e5d 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaLog.html @@ -2,9 +2,9 @@ - + DeltaLog (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -211,6 +211,11 @@

    Method Detail

  • snapshot

    Snapshot snapshot()
    +
    +
    Returns:
    +
    the current Snapshot of the Delta table. You may need to call + update() to access the latest snapshot if the current snapshot is stale.
    +
  • @@ -222,6 +227,10 @@

    update

    Snapshot update()
    Bring DeltaLog's current Snapshot to the latest state if there are any new transaction logs.
    +
    +
    Returns:
    +
    the latest snapshot after applying the new transaction logs.
    +
    @@ -235,6 +244,8 @@

    getSnapshotForVersionAsOf

    Parameters:
    version - the snapshot version to generate
    +
    Returns:
    +
    the snapshot at the provided version
    Throws:
    IllegalArgumentException - if the version is outside the range of available versions
    @@ -253,6 +264,8 @@

    getSnapshotForTimestampAsOf

    Parameters:
    timestamp - the number of milliseconds since midnight, January 1, 1970 UTC
    +
    Returns:
    +
    the snapshot nearest to, but not after, the provided timestamp
    Throws:
    RuntimeException - if the snapshot is unable to be recreated
    IllegalArgumentException - if the timestamp is before the earliest possible @@ -273,6 +286,10 @@

    startTransaction

    Note that all reads in a transaction must go through the returned transaction object, and not directly to the DeltaLog otherwise they will not be checked for conflicts. +

    +
    Returns:
    +
    a new OptimisticTransaction.
    +
    @@ -285,6 +302,8 @@

    getCommitInfoAt

    Parameters:
    version - the commit version to retrieve CommitInfo
    +
    Returns:
    +
    the CommitInfo of the commit at the provided version.
    @@ -295,6 +314,10 @@

    getCommitInfoAt

  • getPath

    org.apache.hadoop.fs.Path getPath()
    +
    +
    Returns:
    +
    the path of the Delta table.
    +
  • @@ -311,6 +334,8 @@

    getChanges

    Parameters:
    startVersion - the table version to begin retrieving actions from (inclusive)
    failOnDataLoss - whether to throw when data loss detected
    +
    Returns:
    +
    an Iterator of VersionLogs
    Throws:
    IllegalArgumentException - if startVersion is negative
    IllegalStateException - if data loss detected and failOnDataLoss is true
    @@ -331,6 +356,8 @@

    forTable

    Parameters:
    hadoopConf - Hadoop Configuration to use when accessing the Delta table
    path - the path to the Delta table
    +
    Returns:
    +
    the DeltaLog for the provided path
    @@ -348,6 +375,8 @@

    forTable

    Parameters:
    hadoopConf - Hadoop Configuration to use when accessing the Delta table
    path - the path to the Delta table
    +
    Returns:
    +
    the DeltaLog for the provided path
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html index f4f615ead6f..51472e74e5b 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/DeltaScan.html @@ -2,9 +2,9 @@ - + DeltaScan (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -168,6 +168,10 @@

    getFiles

    Files returned are guaranteed to satisfy the predicate, if any, returned by getPushedPredicate(). +

    +
    Returns:
    +
    a CloseableIterator to iterate over files.
    +
    @@ -177,6 +181,10 @@

    getFiles

  • getInputPredicate

    java.util.Optional<Expression> getInputPredicate()
    +
    +
    Returns:
    +
    the input predicate used to filter files.
    +
  • @@ -186,6 +194,12 @@

    getInputPredicate

  • getPushedPredicate

    java.util.Optional<Expression> getPushedPredicate()
    +
    +
    Returns:
    +
    portion of the input predicate that can be evaluated by Delta Standalone using only + metadata. Files returned by getFiles() are guaranteed to satisfy the pushed + predicate, and the caller doesn’t need to apply them again on the returned files.
    +
  • @@ -195,6 +209,12 @@

    getPushedPredicate

  • getResidualPredicate

    java.util.Optional<Expression> getResidualPredicate()
    +
    +
    Returns:
    +
    portion of the input predicate that may not be fully applied. Files returned by + getFiles() are not guaranteed to satisfy the residual predicate, and the + caller should still apply them on the returned files.
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html index bcc63653599..04dd0a7f513 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Metrics.html @@ -2,9 +2,9 @@ - + Operation.Metrics (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html index 12a380a2ab6..0c3229eed6c 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.Name.html @@ -2,9 +2,9 @@ - + Operation.Name (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -476,6 +476,10 @@

    values

    for (Operation.Name c : Operation.Name.values())   System.out.println(c);
    +
    +
    Returns:
    +
    an array containing the constants of this enum type, in the order they are declared
    +
    @@ -492,6 +496,8 @@

    valueOf

    Parameters:
    name - the name of the enum constant to be returned.
    +
    Returns:
    +
    the enum constant with the specified name
    Throws:
    IllegalArgumentException - if this enum type has no constant with the specified name
    NullPointerException - if the argument is null
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html index e61ea8948e8..574d9923725 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Operation.html @@ -2,9 +2,9 @@ - + Operation (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -322,6 +322,10 @@

    Method Detail

    getName

    @Nonnull
     public Operation.Name getName()
    +
    +
    Returns:
    +
    operation name
    +
    @@ -332,6 +336,10 @@

    getName

    getParameters

    @Nullable
     public java.util.Map<String,String> getParameters()
    +
    +
    Returns:
    +
    operation parameters
    +
    @@ -342,6 +350,10 @@

    getParameters

    getMetrics

    @Nullable
     public java.util.Map<String,String> getMetrics()
    +
    +
    Returns:
    +
    operation metrics
    +
    @@ -352,6 +364,10 @@

    getMetrics

    getUserMetadata

    @Nonnull
     public java.util.Optional<String> getUserMetadata()
    +
    +
    Returns:
    +
    user metadata for this operation
    +
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html index b8f7db626c1..bf0592a8c55 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/OptimisticTransaction.html @@ -2,9 +2,9 @@ - + OptimisticTransaction (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -197,6 +197,8 @@

    commit

    engineInfo - String used to identify the writer engine. It should resemble "{engineName}/{engineVersion}", with dashes in place of whitespace. For example, "Flink-Connector/1.1.0".
    +
    Returns:
    +
    a CommitResult, wrapping the table version that was committed.
    @@ -227,6 +229,9 @@

    markFilesAsRead

    Parameters:
    readPredicate - Predicate used to determine which files were read.
    +
    Returns:
    +
    a DeltaScan containing the list of files matching the push portion of the + readPredicate.
    @@ -269,6 +274,9 @@

    txnVersion

    Parameters:
    id - transaction id
    +
    Returns:
    +
    the latest version that has committed for the idempotent transaction with given + id.
    @@ -279,6 +287,11 @@

    txnVersion

  • metadata

    Metadata metadata()
    +
    +
    Returns:
    +
    the metadata for this transaction. The metadata refers to the metadata of the + snapshot at the transaction's read version unless updated during the transaction.
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html index 2d47ccd23a1..b4113305446 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/Snapshot.html @@ -2,9 +2,9 @@ - + Snapshot (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -172,6 +172,10 @@

    Method Detail

  • scan

    DeltaScan scan()
    +
    +
    Returns:
    +
    a DeltaScan of the files in this snapshot.
    +
  • @@ -184,6 +188,8 @@

    scan

    Parameters:
    predicate - the predicate to be used to filter the files in this snapshot.
    +
    Returns:
    +
    a DeltaScan of the files in this snapshot.
    @@ -194,6 +200,10 @@

    scan

  • getAllFiles

    java.util.List<AddFile> getAllFiles()
    +
    +
    Returns:
    +
    all of the files present in this snapshot
    +
  • @@ -203,6 +213,10 @@

    getAllFiles

  • getMetadata

    Metadata getMetadata()
    +
    +
    Returns:
    +
    the table metadata for this snapshot
    +
  • @@ -212,6 +226,10 @@

    getMetadata

  • getVersion

    long getVersion()
    +
    +
    Returns:
    +
    the version for this snapshot
    +
  • @@ -223,6 +241,10 @@

    open

    CloseableIterator<RowRecord> open()
    Creates a CloseableIterator which can iterate over data belonging to this snapshot. It provides no iteration ordering guarantee among data.
    +
    +
    Returns:
    +
    a CloseableIterator to iterate over data
    +
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html index d09fe6d1875..453ecde8626 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/VersionLog.html @@ -2,9 +2,9 @@ - + VersionLog (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -204,6 +204,10 @@

    Method Detail

  • getVersion

    public long getVersion()
    +
    +
    Returns:
    +
    the table version at which these actions occurred
    +
  • @@ -214,6 +218,10 @@

    getVersion

    getActions

    @Nonnull
     public java.util.List<Action> getActions()
    +
    +
    Returns:
    +
    an unmodifiable List of the actions for this table version
    +
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html index a5b87d2f29c..ed28bd28efa 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Action.html @@ -2,9 +2,9 @@ - + Action (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -105,7 +105,7 @@

    Interface Action

    public interface Action
    A marker interface for all Actions that can be applied to a Delta Table. Each action represents a single change to the state of a Delta table. - +

    You can use the following code to extract the concrete type of an Action.

    
        // {@link io.delta.standalone.DeltaLog.getChanges} is one way to get such actions
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html
    index a27d76e3d17..3e5b1ed1d1a 100644
    --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html
    +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddCDCFile.html
    @@ -2,9 +2,9 @@
     
     
     
    -
    +
     AddCDCFile (Delta Standalone Reader 0.2.1 JavaDoc)
    -
    +
     
     
     
    @@ -247,6 +247,10 @@ 

    getPath

    Specified by:
    getPath in interface FileAction
    +
    Returns:
    +
    the relative path or the absolute path that should be added to the table. If it's a + relative path, it's relative to the root of the table. Note: the path is encoded and + should be decoded by new java.net.URI(path) when using it.
    @@ -259,6 +263,10 @@

    getPartitionValues

    @Nonnull
     public java.util.Map<String,String> getPartitionValues()
    +
    Returns:
    +
    an unmodifiable Map from partition column to value for + this file. Partition values are stored as strings, using the following formats. + An empty string for any type translates to a null partition value.
    See Also:
    Delta Protocol Partition Value Serialization
    @@ -271,6 +279,10 @@

    getPartitionValues

  • getSize

    public long getSize()
    +
    +
    Returns:
    +
    the size of this file in bytes
    +
  • @@ -281,6 +293,10 @@

    getSize

    getTags

    @Nullable
     public java.util.Map<String,String> getTags()
    +
    +
    Returns:
    +
    an unmodifiable Map containing metadata about this file
    +
    @@ -293,6 +309,8 @@

    isDataChange

    Specified by:
    isDataChange in interface FileAction
    +
    Returns:
    +
    whether any data was changed as a result of this file being added or removed.
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html index 07cc85451e7..9fb32661a89 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.Builder.html @@ -2,9 +2,9 @@ - + AddFile.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -234,6 +234,10 @@

    tags

  • build

    public AddFile build()
    +
    +
    Returns:
    +
    a new AddFile with the same properties as this
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html index bb52032f472..34ac5e3ca78 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html @@ -2,9 +2,9 @@ - + AddFile (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -18,7 +18,7 @@ catch(err) { } //--> -var methods = {"i0":9,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10}; +var methods = {"i0":9,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10}; var tabs = {65535:["t0","All Methods"],1:["t1","Static Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]}; var altColor = "altColor"; var rowColor = "rowColor"; @@ -117,7 +117,7 @@

    Class AddFile

    implements FileAction
    Represents an action that adds a new file to the table. The path of a file acts as the primary key for the entry in the set of files. - +

    Note: since actions within a given Delta file are not guaranteed to be applied in order, it is not valid for multiple file operations with the same path to exist in a single version.

    @@ -245,6 +245,23 @@

    Method Summary

    boolean isDataChange()  + +RemoveFile +remove()  + + +RemoveFile +remove(boolean dataChange)  + + +RemoveFile +remove(long deletionTimestamp)  + + +RemoveFile +remove(long deletionTimestamp, + boolean dataChange)  +
    • @@ -294,6 +311,66 @@

      AddFile

      Method Detail

      + + + +
        +
      • +

        remove

        +
        @Nonnull
        +public RemoveFile remove()
        +
        +
        Returns:
        +
        the corresponding RemoveFile for this file
        +
        +
      • +
      + + + +
        +
      • +

        remove

        +
        @Nonnull
        +public RemoveFile remove(long deletionTimestamp)
        +
        +
        Returns:
        +
        the corresponding RemoveFile for this file, instantiated with the given + deletionTimestamp
        +
        +
      • +
      + + + +
        +
      • +

        remove

        +
        @Nonnull
        +public RemoveFile remove(boolean dataChange)
        +
        +
        Returns:
        +
        the corresponding RemoveFile for this file, instantiated with the given + dataChange flag
        +
        +
      • +
      + + + +
        +
      • +

        remove

        +
        @Nonnull
        +public RemoveFile remove(long deletionTimestamp,
        +                                  boolean dataChange)
        +
        +
        Returns:
        +
        the corresponding RemoveFile for this file, instantiated with the given + deletionTimestamp value and dataChange flag
        +
        +
      • +
      @@ -305,6 +382,10 @@

      getPath

      Specified by:
      getPath in interface FileAction
      +
      Returns:
      +
      the relative path or the absolute path that should be added to the table. If it's a + relative path, it's relative to the root of the table. Note: the path is encoded and + should be decoded by new java.net.URI(path) when using it.
    @@ -317,6 +398,10 @@

    getPartitionValues

    @Nonnull
     public java.util.Map<String,String> getPartitionValues()
    +
    Returns:
    +
    an unmodifiable Map from partition column to value for + this file. Partition values are stored as strings, using the following formats. + An empty string for any type translates to a null partition value.
    See Also:
    Delta Protocol Partition Value Serialization
    @@ -329,6 +414,10 @@

    getPartitionValues

  • getSize

    public long getSize()
    +
    +
    Returns:
    +
    the size of this file in bytes
    +
  • @@ -338,6 +427,11 @@

    getSize

  • getModificationTime

    public long getModificationTime()
    +
    +
    Returns:
    +
    the time that this file was last modified or created, as + milliseconds since the epoch
    +
  • @@ -350,6 +444,10 @@

    isDataChange

    Specified by:
    isDataChange in interface FileAction
    +
    Returns:
    +
    whether any data was changed as a result of this file being created. When + false the file must already be present in the table or the records in the + added file must be contained in one or more remove actions in the same version
    @@ -361,6 +459,11 @@

    isDataChange

    getStats

    @Nullable
     public String getStats()
    +
    +
    Returns:
    +
    statistics (for example: count, min/max values for columns) + about the data in this file as serialized JSON
    +
    @@ -371,6 +474,10 @@

    getStats

    getTags

    @Nullable
     public java.util.Map<String,String> getTags()
    +
    +
    Returns:
    +
    an unmodifiable Map containing metadata about this file
    +
    @@ -410,6 +517,10 @@

    builder

    long size, long modificationTime, boolean dataChange) +
    +
    Returns:
    +
    a new AddFile.Builder
    +
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html index 0b3fd6c163e..0e665b5b01a 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html @@ -2,9 +2,9 @@ - + CommitInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -268,7 +268,8 @@

    version

    @@ -277,7 +278,8 @@

    timestamp

    @@ -304,7 +306,8 @@

    operation

    @@ -395,6 +398,10 @@

    engineInfo

  • build

    public CommitInfo build()
    +
    +
    Returns:
    +
    a new CommitInfo with the same properties as this
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html index 6bd6c8fb8ef..07dfb459919 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html @@ -2,9 +2,9 @@ - + CommitInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -323,19 +323,33 @@

    Constructor Detail

    • CommitInfo

      -
      public CommitInfo(java.util.Optional<Long> version,
      +
      public CommitInfo(@Nonnull
      +                  java.util.Optional<Long> version,
      +                  @Nullable
                         java.sql.Timestamp timestamp,
      +                  @Nonnull
                         java.util.Optional<String> userId,
      +                  @Nonnull
                         java.util.Optional<String> userName,
      +                  @Nullable
                         String operation,
      +                  @Nullable
                         java.util.Map<String,String> operationParameters,
      +                  @Nonnull
                         java.util.Optional<JobInfo> jobInfo,
      +                  @Nonnull
                         java.util.Optional<NotebookInfo> notebookInfo,
      +                  @Nonnull
                         java.util.Optional<String> clusterId,
      +                  @Nonnull
                         java.util.Optional<Long> readVersion,
      +                  @Nonnull
                         java.util.Optional<String> isolationLevel,
      +                  @Nonnull
                         java.util.Optional<Boolean> isBlindAppend,
      +                  @Nonnull
                         java.util.Optional<java.util.Map<String,String>> operationMetrics,
      +                  @Nonnull
                         java.util.Optional<String> userMetadata)
    @@ -345,20 +359,35 @@

    CommitInfo

    • CommitInfo

      -
      public CommitInfo(java.util.Optional<Long> version,
      +
      public CommitInfo(@Nonnull
      +                  java.util.Optional<Long> version,
      +                  @Nullable
                         java.sql.Timestamp timestamp,
      +                  @Nonnull
                         java.util.Optional<String> userId,
      +                  @Nonnull
                         java.util.Optional<String> userName,
      +                  @Nullable
                         String operation,
      +                  @Nullable
                         java.util.Map<String,String> operationParameters,
      +                  @Nonnull
                         java.util.Optional<JobInfo> jobInfo,
      +                  @Nonnull
                         java.util.Optional<NotebookInfo> notebookInfo,
      +                  @Nonnull
                         java.util.Optional<String> clusterId,
      +                  @Nonnull
                         java.util.Optional<Long> readVersion,
      +                  @Nonnull
                         java.util.Optional<String> isolationLevel,
      +                  @Nonnull
                         java.util.Optional<Boolean> isBlindAppend,
      +                  @Nonnull
                         java.util.Optional<java.util.Map<String,String>> operationMetrics,
      +                  @Nonnull
                         java.util.Optional<String> userMetadata,
      +                  @Nonnull
                         java.util.Optional<String> engineInfo)
    @@ -376,7 +405,12 @@

    Method Detail

    • getVersion

      -
      public java.util.Optional<Long> getVersion()
      +
      @Nonnull
      +public java.util.Optional<Long> getVersion()
      +
      +
      Returns:
      +
      the log version for this commit
      +
    @@ -385,7 +419,12 @@

    getVersion

    • getTimestamp

      -
      public java.sql.Timestamp getTimestamp()
      +
      @Nullable
      +public java.sql.Timestamp getTimestamp()
      +
      +
      Returns:
      +
      the time the files in this commit were committed
      +
    @@ -394,7 +433,12 @@

    getTimestamp

    • getUserId

      -
      public java.util.Optional<String> getUserId()
      +
      @Nonnull
      +public java.util.Optional<String> getUserId()
      +
      +
      Returns:
      +
      the userId of the user who committed this file
      +
    @@ -403,7 +447,12 @@

    getUserId

    • getUserName

      -
      public java.util.Optional<String> getUserName()
      +
      @Nonnull
      +public java.util.Optional<String> getUserName()
      +
      +
      Returns:
      +
      the userName of the user who committed this file
      +
    @@ -412,7 +461,12 @@

    getUserName

    • getOperation

      -
      public String getOperation()
      +
      @Nullable
      +public String getOperation()
      +
      +
      Returns:
      +
      the type of operation for this commit. e.g. "WRITE"
      +
    @@ -421,7 +475,12 @@

    getOperation

    • getOperationParameters

      -
      public java.util.Map<String,String> getOperationParameters()
      +
      @Nullable
      +public java.util.Map<String,String> getOperationParameters()
      +
      +
      Returns:
      +
      any relevant operation parameters. e.g. "mode", "partitionBy"
      +
    @@ -430,7 +489,12 @@

    getOperationParameters

    @@ -439,7 +503,12 @@

    getJobInfo

    @@ -448,7 +517,12 @@

    getNotebookInfo

    • getClusterId

      -
      public java.util.Optional<String> getClusterId()
      +
      @Nonnull
      +public java.util.Optional<String> getClusterId()
      +
      +
      Returns:
      +
      the ID of the cluster used to generate this commit
      +
    @@ -457,7 +531,12 @@

    getClusterId

    • getReadVersion

      -
      public java.util.Optional<Long> getReadVersion()
      +
      @Nonnull
      +public java.util.Optional<Long> getReadVersion()
      +
      +
      Returns:
      +
      the version that the transaction used to generate this commit is reading from
      +
    @@ -466,7 +545,12 @@

    getReadVersion

    • getIsolationLevel

      -
      public java.util.Optional<String> getIsolationLevel()
      +
      @Nonnull
      +public java.util.Optional<String> getIsolationLevel()
      +
      +
      Returns:
      +
      the isolation level at which this commit was generated
      +
    @@ -475,7 +559,12 @@

    getIsolationLevel

    • getIsBlindAppend

      -
      public java.util.Optional<Boolean> getIsBlindAppend()
      +
      @Nonnull
      +public java.util.Optional<Boolean> getIsBlindAppend()
      +
      +
      Returns:
      +
      whether this commit has blindly appended without caring about existing files
      +
    @@ -484,7 +573,12 @@

    getIsBlindAppend

    • getOperationMetrics

      -
      public java.util.Optional<java.util.Map<String,String>> getOperationMetrics()
      +
      @Nonnull
      +public java.util.Optional<java.util.Map<String,String>> getOperationMetrics()
      +
      +
      Returns:
      +
      any operation metrics calculated
      +
    @@ -493,7 +587,12 @@

    getOperationMetrics

    • getUserMetadata

      -
      public java.util.Optional<String> getUserMetadata()
      +
      @Nonnull
      +public java.util.Optional<String> getUserMetadata()
      +
      +
      Returns:
      +
      any additional user metadata
      +
    @@ -502,7 +601,13 @@

    getUserMetadata

    • getEngineInfo

      -
      public java.util.Optional<String> getEngineInfo()
      +
      @Nonnull
      +public java.util.Optional<String> getEngineInfo()
      +
      +
      Returns:
      +
      the engineInfo of the operation that performed this commit. It should be of the form + "{engineName}/{engineVersion} Delta-Standalone/{deltaStandaloneVersion}"
      +
    @@ -538,6 +643,10 @@

    hashCode

  • builder

    public static CommitInfo.Builder builder()
    +
    +
    Returns:
    +
    a new CommitInfo.Builder
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html index e45950c8921..d1ccab9665d 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html @@ -2,9 +2,9 @@ - + FileAction (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -174,6 +174,12 @@

    Method Detail

  • getPath

    String getPath()
    +
    +
    Returns:
    +
    the relative path or the absolute path of the file being added or removed by this + action. If it's a relative path, it's relative to the root of the table. Note: the path + is encoded and should be decoded by new java.net.URI(path) when using it.
    +
  • @@ -183,6 +189,10 @@

    getPath

  • isDataChange

    boolean isDataChange()
    +
    +
    Returns:
    +
    whether any data was changed as a result of this file being added or removed.
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html index 9ee08e73d68..864975487ec 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html @@ -2,9 +2,9 @@ - + Format (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -226,6 +226,10 @@

    Method Detail

  • getProvider

    public String getProvider()
    +
    +
    Returns:
    +
    the name of the encoding for files in this table
    +
  • @@ -235,6 +239,11 @@

    getProvider

  • getOptions

    public java.util.Map<String,String> getOptions()
    +
    +
    Returns:
    +
    an unmodifiable Map containing configuration options for + the format
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html index 78c11bc5098..01868af9e98 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html @@ -2,9 +2,9 @@ - + JobInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -252,6 +252,10 @@

    triggerType

  • build

    public JobInfo build()
    +
    +
    Returns:
    +
    a new JobInfo with the same properties as this
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html index 13364abc72c..ddc6e727353 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html @@ -2,9 +2,9 @@ - + JobInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -324,6 +324,10 @@

    hashCode

  • builder

    public static JobInfo.Builder builder(String jobId)
    +
    +
    Returns:
    +
    a new JobInfo.Builder
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html index a33e6f8afeb..e9ee92a11af 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html @@ -2,9 +2,9 @@ - + Metadata.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -231,7 +231,8 @@

    Method Detail

    @@ -240,7 +241,8 @@

    id

    @@ -249,7 +251,8 @@

    name

    @@ -258,7 +261,8 @@

    description

    @@ -267,7 +271,8 @@

    format

    @@ -276,7 +281,8 @@

    partitionColumns

    @@ -294,7 +300,8 @@

    createdTime

    @@ -304,6 +311,10 @@

    schema

  • build

    public Metadata build()
    +
    +
    Returns:
    +
    a new Metadata with the same properties as this
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html index 0123d50b353..ae9e3c4f3ce 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html @@ -2,9 +2,9 @@ - + Metadata (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -275,13 +275,21 @@

    Constructor Detail

    • Metadata

      -
      public Metadata(String id,
      +
      public Metadata(@Nonnull
      +                String id,
      +                @Nullable
                       String name,
      +                @Nullable
                       String description,
      +                @Nonnull
                       Format format,
      +                @Nonnull
                       java.util.List<String> partitionColumns,
      +                @Nonnull
                       java.util.Map<String,String> configuration,
      +                @Nonnull
                       java.util.Optional<Long> createdTime,
      +                @Nullable
                       StructType schema)
    @@ -299,7 +307,12 @@

    Method Detail

    • getId

      -
      public String getId()
      +
      @Nonnull
      +public String getId()
      +
      +
      Returns:
      +
      the unique identifier for this table
      +
    @@ -308,7 +321,12 @@

    getId

    • getName

      -
      public String getName()
      +
      @Nullable
      +public String getName()
      +
      +
      Returns:
      +
      the user-provided identifier for this table
      +
    @@ -317,7 +335,12 @@

    getName

    • getDescription

      -
      public String getDescription()
      +
      @Nullable
      +public String getDescription()
      +
      +
      Returns:
      +
      the user-provided description for this table
      +
    @@ -326,7 +349,12 @@

    getDescription

    @@ -335,7 +363,13 @@

    getFormat

    • getPartitionColumns

      -
      public java.util.List<String> getPartitionColumns()
      +
      @Nonnull
      +public java.util.List<String> getPartitionColumns()
      +
      +
      Returns:
      +
      an unmodifiable java.util.List containing the names of + columns by which the data should be partitioned
      +
    @@ -344,7 +378,13 @@

    getPartitionColumns

    • getConfiguration

      -
      public java.util.Map<String,String> getConfiguration()
      +
      @Nonnull
      +public java.util.Map<String,String> getConfiguration()
      +
      +
      Returns:
      +
      an unmodifiable java.util.Map containing configuration + options for this metadata
      +
    @@ -353,7 +393,13 @@

    getConfiguration

    • getCreatedTime

      -
      public java.util.Optional<Long> getCreatedTime()
      +
      @Nonnull
      +public java.util.Optional<Long> getCreatedTime()
      +
      +
      Returns:
      +
      the time when this metadata action was created, in milliseconds + since the Unix epoch
      +
    @@ -362,7 +408,12 @@

    getCreatedTime

    @@ -398,6 +449,10 @@

    hashCode

  • builder

    public static Metadata.Builder builder()
    +
    +
    Returns:
    +
    a new Metadata.Builder
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html index b431ef39da0..f706f56128a 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html @@ -2,9 +2,9 @@ - + NotebookInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html index e01d2379cb5..999809fea9c 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html @@ -2,9 +2,9 @@ - + Protocol (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -119,7 +119,7 @@

    Class Protocol

    incompatible changes are made to the protocol. Readers and writers are responsible for checking that they meet the minimum versions before performing any other operations. - +

    Since this action allows us to explicitly block older clients in the case of a breaking change to the protocol, clients should be tolerant of messages and fields that they do not understand.

    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html deleted file mode 100644 index 5ff3d0a1a15..00000000000 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.Builder.html +++ /dev/null @@ -1,352 +0,0 @@ - - - - - -RemoveFile.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - - - - - - - - - - - - -
    -
    io.delta.standalone.actions
    -

    Class RemoveFile.Builder

    -
    -
    -
      -
    • Object
    • -
    • -
        -
      • io.delta.standalone.actions.RemoveFile.Builder
      • -
      -
    • -
    -
    -
      -
    • -
      -
      Enclosing class:
      -
      RemoveFile
      -
      -
      -
      -
      public static class RemoveFile.Builder
      -extends Object
      -
      Builder class for RemoveFile. Enables construction of RemoveFile object with default values.
      -
    • -
    -
    -
    - -
    -
    - -
    -
    - - - - - - - diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html index 0529ff791e5..f3761244494 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html @@ -2,9 +2,9 @@ - + RemoveFile (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -18,8 +18,8 @@ catch(err) { } //--> -var methods = {"i0":9,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":10}; -var tabs = {65535:["t0","All Methods"],1:["t1","Static Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]}; +var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10}; +var tabs = {65535:["t0","All Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]}; var altColor = "altColor"; var rowColor = "rowColor"; var tableTab = "tableTab"; @@ -49,7 +49,7 @@
    @@ -156,7 +157,9 @@

    Method Summary

    AddFile -build()  +build() +
    Builds an AddFile using the provided parameters.
    + AddFile.Builder @@ -234,9 +237,11 @@

    tags

  • build

    public AddFile build()
    +
    Builds an AddFile using the provided parameters. If a parameter is not provided + its default values is used.
    Returns:
    -
    a new AddFile with the same properties as this
    +
    a new AddFile with the properties added to the builder
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html index 34ac5e3ca78..8499fd73627 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/AddFile.html @@ -2,9 +2,9 @@ - + AddFile (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -122,7 +122,7 @@

    Class AddFile

    not valid for multiple file operations with the same path to exist in a single version.
    See Also:
    -
    Delta Transaction Log Protocol
    +
    Delta Transaction Log Protocol: Add File and Remove File
    @@ -145,7 +145,7 @@

    Nested Class Summary

    static class  AddFile.Builder -
    Builder class for AddFile.
    +
    Builder class for AddFile.
    @@ -321,7 +321,8 @@

    remove

    public RemoveFile remove()
    Returns:
    -
    the corresponding RemoveFile for this file
    +
    the corresponding RemoveFile for this file, instantiated with + deletionTimestamp = System.currentTimeMillis()
    @@ -519,7 +520,7 @@

    builder

    boolean dataChange)
    Returns:
    -
    a new AddFile.Builder
    +
    a new AddFile.Builder
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html index 0e665b5b01a..e66be8cf9f5 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.Builder.html @@ -2,9 +2,9 @@ - + CommitInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -114,7 +114,8 @@

    Class CommitInfo.Builder


    public static class CommitInfo.Builder
     extends Object
    -
    Builder class for CommitInfo. Enables construction of CommitInfo object with default values.
    +
    Builder class for CommitInfo. Enables construction of CommitInfos with + default values.
    @@ -152,7 +153,9 @@

    Method Summary

    CommitInfo -build()  +build() +
    Builds a CommitInfo using the provided parameters.
    + CommitInfo.Builder @@ -398,9 +401,11 @@

    engineInfo

  • build

    public CommitInfo build()
    +
    Builds a CommitInfo using the provided parameters. If a parameter is not provided + its default values is used.
    Returns:
    -
    a new CommitInfo with the same properties as this
    +
    a new CommitInfo with the properties added to the builder
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html index 07dfb459919..856094f4cae 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/CommitInfo.html @@ -2,9 +2,9 @@ - + CommitInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -120,7 +120,7 @@

    Class CommitInfo

    Information stored in it is best effort (i.e. can be falsified by a writer).
    See Also:
    -
    Delta Transaction Log Protocol
    +
    Delta Transaction Log Protocol: Commit Provenance Information
    @@ -143,7 +143,7 @@

    Nested Class Summary

    static class  CommitInfo.Builder -
    Builder class for CommitInfo.
    +
    Builder class for CommitInfo.
    @@ -605,7 +605,7 @@

    getEngineInfo

    public java.util.Optional<String> getEngineInfo()
    Returns:
    -
    the engineInfo of the operation that performed this commit. It should be of the form +
    the engineInfo of the engine that performed this commit. It should be of the form "{engineName}/{engineVersion} Delta-Standalone/{deltaStandaloneVersion}"
    @@ -645,7 +645,7 @@

    builder

    public static CommitInfo.Builder builder()
    Returns:
    -
    a new CommitInfo.Builder
    +
    a new CommitInfo.Builder
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html index d1ccab9665d..e4707a13406 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/FileAction.html @@ -2,9 +2,9 @@ - + FileAction (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html index 864975487ec..8f01a7f1d42 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Format.html @@ -2,9 +2,9 @@ - + Format (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -113,7 +113,7 @@

    Class Format

    A specification of the encoding for the files stored in a table.
    See Also:
    -
    Delta Transaction Log Protocol
    +
    Delta Transaction Log Protocol: Format Specification
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html index 01868af9e98..56a21f26430 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.Builder.html @@ -2,9 +2,9 @@ - + JobInfo.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -114,7 +114,8 @@

    Class JobInfo.Builder


    public static class JobInfo.Builder
     extends Object
    -
    Builder class for JobInfo. Enables construction of JobInfo object with default values.
    +
    Builder class for JobInfo. Enables construction of JobInfos with default + values.
    @@ -152,7 +153,9 @@

    Method Summary

    JobInfo -build()  +build() +
    Builds a JobInfo using the provided parameters.
    + JobInfo.Builder @@ -252,9 +255,11 @@

    triggerType

  • build

    public JobInfo build()
    +
    Builds a JobInfo using the provided parameters. If a parameter is not provided + its default values is used.
    Returns:
    -
    a new JobInfo with the same properties as this
    +
    a new JobInfo with the properties added to the builder
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html index ddc6e727353..481bd275e33 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/JobInfo.html @@ -2,9 +2,9 @@ - + JobInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -132,7 +132,7 @@

    Nested Class Summary

    static class  JobInfo.Builder -
    Builder class for JobInfo.
    +
    Builder class for JobInfo.
    @@ -326,7 +326,7 @@

    builder

    public static JobInfo.Builder builder(String jobId)
    Returns:
    -
    a new JobInfo.Builder
    +
    a new JobInfo.Builder
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html index e9ee92a11af..0e1a884be03 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.Builder.html @@ -2,9 +2,9 @@ - + Metadata.Builder (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -114,7 +114,8 @@

    Class Metadata.Builder


    public static class Metadata.Builder
     extends Object
    -
    Builder class for Metadata. Enables construction of Metadata object with default values.
    +
    Builder class for Metadata. Enables construction of Metadatas with default + values.
    @@ -152,7 +153,9 @@

    Method Summary

    Metadata -build()  +build() +
    Builds a Metadata using the provided parameters.
    + Metadata.Builder @@ -311,9 +314,11 @@

    schema

  • build

    public Metadata build()
    +
    Builds a Metadata using the provided parameters. If a parameter is not provided + its default values is used.
    Returns:
    -
    a new Metadata with the same properties as this
    +
    a new Metadata with the properties added to the builder
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html index ae9e3c4f3ce..53c1c01c803 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Metadata.html @@ -2,9 +2,9 @@ - + Metadata (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -123,7 +123,7 @@

    Class Metadata

    given version of the table.
    See Also:
    -
    Delta Transaction Log Protocol
    +
    Delta Transaction Log Protocol: Change Metadata
    @@ -146,7 +146,7 @@

    Nested Class Summary

    static class  Metadata.Builder -
    Builder class for Metadata.
    +
    Builder class for Metadata.
    @@ -451,7 +451,7 @@

    builder

    public static Metadata.Builder builder()
    Returns:
    -
    a new Metadata.Builder
    +
    a new Metadata.Builder
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html index f706f56128a..4c4f947d3bf 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/NotebookInfo.html @@ -2,9 +2,9 @@ - + NotebookInfo (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html index 999809fea9c..5bca11282e5 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/Protocol.html @@ -2,9 +2,9 @@ - + Protocol (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -123,6 +123,10 @@

    Class Protocol

    Since this action allows us to explicitly block older clients in the case of a breaking change to the protocol, clients should be tolerant of messages and fields that they do not understand. +
    +
    See Also:
    +
    Delta Transaction Log Protocol: Protocol Evolution
    +
    @@ -249,6 +253,11 @@

    Method Detail

  • getMinReaderVersion

    public int getMinReaderVersion()
    +
    +
    Returns:
    +
    the minimum version of the Delta read protocol that a client must implement in order + to correctly read this table
    +
  • @@ -258,6 +267,11 @@

    getMinReaderVersion

  • getMinWriterVersion

    public int getMinWriterVersion()
    +
    +
    Returns:
    +
    the minimum version of the Delta write protocol that a client must implement in order + to correctly write this table
    +
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html index f3761244494..6adacccdfe5 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/RemoveFile.html @@ -2,9 +2,9 @@ - + RemoveFile (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -118,13 +118,17 @@

    Class RemoveFile

    Logical removal of a given file from the reservoir. Acts as a tombstone before a file is deleted permanently.

    - Note that users should onlu instantiate RemoveFile instances using one of the various + Users should only instantiate RemoveFile instances using one of the various AddFile.remove() methods.

    - As well, note that for protocol compatibility reasons, the fields partitionValues, - size, and tags are only present when the extendedFileMetadata flag is true. New - writers should generally be setting this flag, but old writers (and FSCK) won't, so readers must - check this flag before attempting to consume those values.

    + Note that for protocol compatibility reasons, the fields partitionValues, + size, and tags are only present when the extendedFileMetadata flag is + true. New writers should generally be setting this flag, but old writers (and FSCK) won't, so + readers must check this flag before attempting to consume those values. +
    +
    See Also:
    +
    Delta Transaction Log Protocol: Add File and Remove File
    +
    @@ -314,9 +318,9 @@

    isDataChange

    Specified by:
    isDataChange in interface FileAction
    Returns:
    -
    whether any data was changed as a result of this file being created. When - false the file must already be present in the table or the records in the - added file must be contained in one or more remove actions in the same version
    +
    whether any data was changed as a result of this file being removed. When + false the records in the removed file must be contained in one or more add + actions in the same version
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html index c7d01837bb6..f15c3296847 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/SetTransaction.html @@ -2,9 +2,9 @@ - + SetTransaction (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -115,8 +115,12 @@

    Class SetTransaction

    public final class SetTransaction
     extends Object
     implements Action
    -
    Sets the committed version for a given application. Used to make operations like streaming append - idempotent.
    +
    Sets the committed version for a given application. Used to make operations like + Operation.Name.STREAMING_UPDATE idempotent.
    +
    +
    See Also:
    +
    Delta Transaction Log Protocol: Transaction Identifiers
    +
    @@ -234,7 +238,7 @@

    getAppId

    public String getAppId()
    Returns:
    -
    the application ID
    +
    the unique identifier for the application performing the transaction
    @@ -247,7 +251,7 @@

    getVersion

    public long getVersion()
    Returns:
    -
    the committed version for the application ID
    +
    the application-specific numeric identifier for this transaction
    @@ -261,7 +265,8 @@

    getLastUpdated

    public java.util.Optional<Long> getLastUpdated()
    Returns:
    -
    the last updated timestamp of this transaction (milliseconds since the epoch)
    +
    the time when this transaction action was created, in milliseconds since the Unix + epoch
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html index 06d40efcdb1..7ffd958ff40 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-frame.html @@ -2,9 +2,9 @@ - + io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html index 753f89553a5..4973912d6ad 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-summary.html @@ -2,9 +2,9 @@ - + io.delta.standalone.actions (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -84,7 +84,7 @@

    Package io.delta.standalone.actions

    Action -
    A marker interface for all Actions that can be applied to a Delta Table.
    +
    A marker interface for all actions that can be applied to a Delta table.
    @@ -119,7 +119,7 @@

    Package io.delta.standalone.actions

    AddFile.Builder -
    Builder class for AddFile.
    +
    Builder class for AddFile.
    @@ -131,7 +131,7 @@

    Package io.delta.standalone.actions

    CommitInfo.Builder -
    Builder class for CommitInfo.
    +
    Builder class for CommitInfo.
    @@ -149,7 +149,7 @@

    Package io.delta.standalone.actions

    JobInfo.Builder -
    Builder class for JobInfo.
    +
    Builder class for JobInfo.
    @@ -161,7 +161,7 @@

    Package io.delta.standalone.actions

    Metadata.Builder -
    Builder class for Metadata.
    +
    Builder class for Metadata.
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html index 9b8099e0a32..948efb28e66 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/actions/package-tree.html @@ -2,9 +2,9 @@ - + io.delta.standalone.actions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html index b6596d96b88..7a761788d03 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/CloseableIterator.html @@ -2,9 +2,9 @@ - + CloseableIterator (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -100,7 +100,7 @@

    Interface CloseableIterato
    public interface CloseableIterator<T>
     extends java.util.Iterator<T>, java.io.Closeable
    -
    An Iterator that also need to implement the Closeable interface. The caller +
    An Iterator that also implements the Closeable interface. The caller should call Closeable.close() method to free all resources properly after using the iterator.
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html index 558f8dceeca..196e0beb7dc 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/RowRecord.html @@ -2,9 +2,9 @@ - + RowRecord (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html index 3465c4f3ee0..c0fab2f8cc3 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-frame.html @@ -2,9 +2,9 @@ - + io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html index dac0745f600..baa540ba136 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-summary.html @@ -2,9 +2,9 @@ - + io.delta.standalone.data (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -84,7 +84,7 @@

    Package io.delta.standalone.data

    CloseableIterator<T> -
    An Iterator that also need to implement the Closeable interface.
    +
    An Iterator that also implements the Closeable interface.
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html index c2263ebec36..26da978d83c 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/data/package-tree.html @@ -2,9 +2,9 @@ - + io.delta.standalone.data Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html index de518b9b301..bfe75fb1c62 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentAppendException.html @@ -2,9 +2,9 @@ - + ConcurrentAppendException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html index d955cd76cd3..340e3e8760b 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteDeleteException.html @@ -2,9 +2,9 @@ - + ConcurrentDeleteDeleteException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html index bb73c77d7f7..9e53f21891b 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentDeleteReadException.html @@ -2,9 +2,9 @@ - + ConcurrentDeleteReadException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html index 0e0eb89ec0f..8708b4703a7 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ConcurrentTransactionException.html @@ -2,9 +2,9 @@ - + ConcurrentTransactionException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html index 14f950ecf36..6811377b636 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaConcurrentModificationException.html @@ -2,9 +2,9 @@ - + DeltaConcurrentModificationException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html index 641c37df0a2..10e6501fef2 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/DeltaStandaloneException.html @@ -2,9 +2,9 @@ - + DeltaStandaloneException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html index 18f0880825f..8e72468a25c 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/MetadataChangedException.html @@ -2,9 +2,9 @@ - + MetadataChangedException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html index 28fede9c571..470b9c9532b 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/ProtocolChangedException.html @@ -2,9 +2,9 @@ - + ProtocolChangedException (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html index 8c4b6deba2d..dbce6ab7ed9 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-frame.html @@ -2,9 +2,9 @@ - + io.delta.standalone.exceptions (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html index 6ae55cff4fe..a485ae53b71 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-summary.html @@ -2,9 +2,9 @@ - + io.delta.standalone.exceptions (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html index f29e80bcbb0..41e6ee1dcdb 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/exceptions/package-tree.html @@ -2,9 +2,9 @@ - + io.delta.standalone.exceptions Class Hierarchy (Delta Standalone Reader 0.2.1 JavaDoc) - + diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html index 93329cb0175..13cebd042eb 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/And.html @@ -2,9 +2,9 @@ - + And (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -18,12 +18,6 @@ catch(err) { } //--> -var methods = {"i0":10}; -var tabs = {65535:["t0","All Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]}; -var altColor = "altColor"; -var rowColor = "rowColor"; -var tableTab = "tableTab"; -var activeTableTab = "activeTableTab";
    @@ -125,7 +119,9 @@

    Class And

    public final class And
     extends BinaryOperator
     implements Predicate
    -
    Usage: new And(expr1, expr2) - Logical AND
    +
    Evaluates logical expr1 AND expr2 for new And(expr1, expr2). +

    + Requires both left and right input expressions evaluate to booleans.

    @@ -156,18 +152,6 @@

    Constructor Summary

    Method Summary

    - - - - - - - - - - -
    All Methods Instance Methods Concrete Methods 
    Modifier and TypeMethod and Description
    ObjectnullSafeEval(Object leftResult, - Object rightResult) 
    - -
      -
    • - - -

      Method Detail

      - - - -
        -
      • -

        nullSafeEval

        -
        public Object nullSafeEval(Object leftResult,
        -                           Object rightResult)
        -
      • -
      -
    • -
    @@ -300,13 +266,13 @@

    nullSafeEval

  • Nested | 
  • Field | 
  • Constr | 
  • -
  • Method
  • +
  • Method
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html index a9d5f5dcea7..f73a2c8fe35 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryComparison.html @@ -2,9 +2,9 @@ - + BinaryComparison (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -69,13 +69,13 @@
  • Summary: 
  • Nested | 
  • Field | 
  • -
  • Constr | 
  • +
  • Constr | 
  • Method
  • @@ -123,7 +123,7 @@

    Class BinaryComparison

    public abstract class BinaryComparison
     extends BinaryOperator
     implements Predicate
    -
    A BinaryOperator that compares the left and right Expressions and returns a +
    A BinaryOperator that compares the left and right Expressions and evaluates to a boolean value.
    @@ -131,25 +131,6 @@

    Class BinaryComparison

    -
    - -
    @@ -270,13 +226,13 @@

    BinaryComparison

  • Summary: 
  • Nested | 
  • Field | 
  • -
  • Constr | 
  • +
  • Constr | 
  • Method
  • diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html index 340efbc2cec..70b8e7d3d36 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryExpression.html @@ -2,9 +2,9 @@ - + BinaryExpression (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -75,13 +75,13 @@
  • Summary: 
  • Nested | 
  • Field | 
  • -
  • Constr | 
  • +
  • Constr | 
  • Method
  • @@ -119,32 +119,14 @@

    Class BinaryExpression

    public abstract class BinaryExpression
     extends Object
     implements Expression
    -
    An expression with two inputs and one output. The output is by default evaluated to null - if any input is evaluated to null.
    +
    An Expression with two inputs and one output. The output is by default evaluated to null + if either input is evaluated to null.
    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html index eaee63ffaaf..4edd3879432 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/BinaryOperator.html @@ -2,9 +2,9 @@ - + BinaryOperator (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -75,13 +75,13 @@
  • Summary: 
  • Nested | 
  • Field | 
  • -
  • Constr | 
  • +
  • Constr | 
  • Method
  • @@ -123,37 +123,16 @@

    Class BinaryOperator


    public abstract class BinaryOperator
     extends BinaryExpression
    -
    A BinaryExpression that is an operator, with two properties: -
      -
    1. The string representation is x symbol y, rather than funcName(x, y).
    2. -
    3. Two inputs are expected to be of the same type. If the two inputs have different types, an - IllegalArgumentException will be thrown.
    4. -
    +
    A BinaryExpression that is an operator, meaning the string representation is + x symbol y, rather than funcName(x, y). +

    + Requires both inputs to be of the same data type.

    diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html index 05ac6359db7..c56febb0c58 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/Column.html @@ -2,9 +2,9 @@ - + Column (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -121,10 +121,11 @@

    Class Column

    extends LeafExpression
    A column whose row-value will be computed based on the data in a RowRecord.

    - Usage: new Column(columnName, columnDataType). + It is recommended that you instantiate using an existing table schema StructType + with StructType.column(String). For example, mySchema.column(columnName).

    - It is recommended that you instantiate using an existing table schema (StructType). - e.g. mySchema.column(columnName)

    + Only supports primitive data types, see + Delta Transaction Log Protocol: Primitive Types. diff --git a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html index 636edb6c02c..a097f0f26ed 100644 --- a/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html +++ b/docs/0.3.0-SNAPSHOT/delta-standalone/api/java/io/delta/standalone/expressions/EqualTo.html @@ -2,9 +2,9 @@ - + EqualTo (Delta Standalone Reader 0.2.1 JavaDoc) - + @@ -18,12 +18,6 @@ catch(err) { } //--> -var methods = {"i0":10}; -var tabs = {65535:["t0","All Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]}; -var altColor = "altColor"; -var rowColor = "rowColor"; -var tableTab = "tableTab"; -var activeTableTab = "activeTableTab";