diff --git a/.github/ISSUE_TEMPLATE/SUPPORT_REQUEST.md b/.github/ISSUE_TEMPLATE/SUPPORT_REQUEST.md index adf727399355..9ce26e9d2bd3 100644 --- a/.github/ISSUE_TEMPLATE/SUPPORT_REQUEST.md +++ b/.github/ISSUE_TEMPLATE/SUPPORT_REQUEST.md @@ -8,7 +8,7 @@ labels: question **_Tips before filing an issue_** -- Have you gone through our [FAQs](https://cwiki.apache.org/confluence/display/HUDI/FAQ)? +- Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)? - Join the mailing list to engage in conversations and get faster support at dev-subscribe@hudi.apache.org. diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 0652de980e16..fc79fefd2065 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -18,8 +18,20 @@ jobs: include: - scala: "scala-2.11" spark: "spark2" + - scala: "scala-2.11" + spark: "spark2,spark-shade-unbundle-avro" + - scala: "scala-2.12" + spark: "spark3,spark3.0.x" + - scala: "scala-2.12" + spark: "spark3,spark3.0.x,spark-shade-unbundle-avro" + - scala: "scala-2.12" + spark: "spark3,spark3.1.x" + - scala: "scala-2.12" + spark: "spark3,spark3.1.x,spark-shade-unbundle-avro" - scala: "scala-2.12" spark: "spark3" + - scala: "scala-2.12" + spark: "spark3,spark-shade-unbundle-avro" steps: - uses: actions/checkout@v2 - name: Set up JDK 8 diff --git a/.gitignore b/.gitignore index 413e0a0c4c0d..f465a1992bb1 100644 --- a/.gitignore +++ b/.gitignore @@ -61,7 +61,8 @@ local.properties # IntelliJ specific files/directories # ####################################### .out -.idea +.idea/* +!.idea/vcs.xml *.ipr *.iws *.iml diff --git a/.idea/vcs.xml b/.idea/vcs.xml new file mode 100644 index 000000000000..10142b516e46 --- /dev/null +++ b/.idea/vcs.xml @@ -0,0 +1,36 @@ + + + + + + + + + + diff --git a/NOTICE b/NOTICE index 9b249331f343..437b974ac217 100644 --- a/NOTICE +++ b/NOTICE @@ -159,3 +159,9 @@ its NOTICE file: This product includes software developed at StreamSets (http://www.streamsets.com/). +-------------------------------------------------------------------------------- + +This product includes code from hilbert-curve project + * Copyright https://github.com/davidmoten/hilbert-curve + * Licensed under the Apache-2.0 License + diff --git a/README.md b/README.md index 764ba0d48bcc..4f48fc6c7f9b 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Prerequisites for building Apache Hudi: * Unix-like system (like Linux, Mac OS X) * Java 8 (Java 9 or 10 may work) * Git -* Maven +* Maven (>=3.3.1) ``` # Checkout code and build @@ -78,12 +78,19 @@ The default Scala version supported is 2.11. To build for Scala 2.12 version, bu mvn clean package -DskipTests -Dscala-2.12 ``` -### Build with Spark 3.0.0 +### Build with Spark 3 -The default Spark version supported is 2.4.4. To build for Spark 3.0.0 version, build using `spark3` profile +The default Spark version supported is 2.4.4. To build for different Spark 3 versions, use the corresponding profile ``` +# Build against Spark 3.2.0 (default build shipped with the public jars) mvn clean package -DskipTests -Dspark3 + +# Build against Spark 3.1.2 +mvn clean package -DskipTests -Dspark3.1.x + +# Build against Spark 3.0.3 +mvn clean package -DskipTests -Dspark3.0.x ``` ### Build without spark-avro module diff --git a/azure-pipelines.yml b/azure-pipelines.yml index ef68cd52e468..cd75e28dae24 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -26,13 +26,14 @@ variables: MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) -Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true' SPARK_VERSION: '2.4.4' HADOOP_VERSION: '2.7' - SPARK_HOME: $(Pipeline.Workspace)/spark-$(SPARK_VERSION)-bin-hadoop$(HADOOP_VERSION) + SPARK_ARCHIVE: spark-$(SPARK_VERSION)-bin-hadoop$(HADOOP_VERSION) stages: - stage: test jobs: - job: UT_FT_1 displayName: UT FT common & flink & UT client/spark-client + timeoutInMinutes: '90' steps: - task: Cache@2 displayName: set cache @@ -47,7 +48,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'install' - options: -DskipTests + options: -T 2.5C -DskipTests publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -71,6 +72,7 @@ stages: mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: UT_FT_2 displayName: FT client/spark-client + timeoutInMinutes: '90' steps: - task: Cache@2 displayName: set cache @@ -85,7 +87,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'install' - options: -DskipTests + options: -T 2.5C -DskipTests publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -99,7 +101,8 @@ stages: jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: UT_FT_3 - displayName: UT FT cli & utilities & sync/hive-sync + displayName: UT FT clients & cli & utilities & sync/hive-sync + timeoutInMinutes: '90' steps: - task: Cache@2 displayName: set cache @@ -114,30 +117,31 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'install' - options: -DskipTests + options: -T 2.5C -DskipTests publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - task: Maven@3 - displayName: UT cli & utilities & sync/hive-sync + displayName: UT clients & cli & utilities & sync/hive-sync inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Punit-tests -pl hudi-cli,hudi-utilities,hudi-sync/hudi-hive-sync + options: -Punit-tests -pl hudi-client/hudi-client-common,hudi-client/hudi-flink-client,hudi-client/hudi-java-client,hudi-cli,hudi-utilities,hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - task: Maven@3 - displayName: FT cli & utilities & sync/hive-sync + displayName: FT clients & cli & utilities & sync/hive-sync inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Pfunctional-tests -pl hudi-cli,hudi-utilities,hudi-sync/hudi-hive-sync + options: -Pfunctional-tests -pl hudi-client/hudi-client-common,hudi-client/hudi-flink-client,hudi-client/hudi-java-client,hudi-cli,hudi-utilities,hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: UT_FT_4 displayName: UT FT other modules + timeoutInMinutes: '90' steps: - task: Cache@2 displayName: set cache @@ -152,7 +156,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'install' - options: -DskipTests + options: -T 2.5C -DskipTests publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -161,7 +165,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Punit-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync + options: -Punit-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -170,16 +174,23 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Pfunctional-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync + options: -Pfunctional-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: IT steps: + - task: AzureCLI@2 + displayName: Prepare for IT + inputs: + azureSubscription: apachehudici-service-connection + scriptType: bash + scriptLocation: inlineScript + inlineScript: | + echo 'Downloading $(SPARK_ARCHIVE)' + az storage blob download -c ci-caches -n $(SPARK_ARCHIVE).tgz -f $(Pipeline.Workspace)/$(SPARK_ARCHIVE).tgz --account-name apachehudici + tar -xvf $(Pipeline.Workspace)/$(SPARK_ARCHIVE).tgz -C $(Pipeline.Workspace)/ + mkdir /tmp/spark-events/ - script: | - echo 'Downloading spark-$(SPARK_VERSION)-bin-hadoop$(HADOOP_VERSION)' - wget https://archive.apache.org/dist/spark/spark-$(SPARK_VERSION)/spark-$(SPARK_VERSION)-bin-hadoop$(HADOOP_VERSION).tgz -O $(Pipeline.Workspace)/spark-$(SPARK_VERSION).tgz - tar -xvf $(Pipeline.Workspace)/spark-$(SPARK_VERSION).tgz -C $(Pipeline.Workspace)/ - mkdir /tmp/spark-events/ mvn $(MAVEN_OPTS) -Pintegration-tests verify displayName: IT diff --git a/conf/hudi-defaults.conf.template b/conf/hudi-defaults.conf.template new file mode 100644 index 000000000000..175dbaf23d73 --- /dev/null +++ b/conf/hudi-defaults.conf.template @@ -0,0 +1,26 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +# hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 +# hoodie.datasource.hive_sync.use_jdbc true +# hoodie.datasource.hive_sync.support_timestamp false +# hoodie.index.type BLOOM +# hoodie.metadata.enable false diff --git a/doap_HUDI.rdf b/doap_HUDI.rdf index fc474d767fba..33f64ecf82ec 100644 --- a/doap_HUDI.rdf +++ b/doap_HUDI.rdf @@ -76,6 +76,11 @@ 2021-08-26 0.9.0 + + Apache Hudi 0.10.0 + 2021-12-08 + 0.10.0 + diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index 5361e90684b9..bb0ec788e0f3 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index 8f3f90f0559c..e9b4e4fbac6a 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index fdf41e15a537..dbdb7182f682 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index c7a828e5c3f1..e267e8487e79 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index a0e6af4fad3b..71998198f53c 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index 443badc11161..db47f3924e4b 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../../pom.xml 4.0.0 @@ -54,7 +54,7 @@ 2.3.3 2.8.4 0.217 - 1.4.3 + 1.4.13 true ${project.parent.basedir} diff --git a/docker/hoodie/hadoop/prestobase/pom.xml b/docker/hoodie/hadoop/prestobase/pom.xml index 3d53485654dd..98502aa0fe2f 100644 --- a/docker/hoodie/hadoop/prestobase/pom.xml +++ b/docker/hoodie/hadoop/prestobase/pom.xml @@ -20,7 +20,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index b7c49e34a516..8cd1ee2dea3c 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index 2da40edf1fa4..20a9cab164c3 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index b000eb675282..3a8dabc4afc3 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index cd44580c7ea1..6a79f8debc16 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 pom diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml new file mode 100644 index 000000000000..d44a389a61f6 --- /dev/null +++ b/hudi-aws/pom.xml @@ -0,0 +1,208 @@ + + + + + hudi + org.apache.hudi + 0.11.0-SNAPSHOT + + 4.0.0 + + hudi-aws + 0.11.0-SNAPSHOT + + hudi-aws + jar + + + 1.15.0 + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + + + log4j + log4j + + + + + org.apache.hadoop + hadoop-common + tests + test + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + + + + com.amazonaws + dynamodb-lock-client + ${dynamodb.lockclient.version} + + + + + com.amazonaws + aws-java-sdk-cloudwatch + ${aws.sdk.version} + + + + com.amazonaws + aws-java-sdk-dynamodb + ${aws.sdk.version} + + + io.netty + * + + + + + com.amazonaws + aws-java-sdk-core + ${aws.sdk.version} + + + + io.dropwizard.metrics + metrics-core + + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.mockito + mockito-junit-jupiter + test + + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + io.fabric8 + docker-maven-plugin + + + prepare-it-database + pre-integration-test + + start + + + + + amazon/dynamodb-local:${dynamodb-local.version} + it-database + + + ${dynamodb-local.port}:${dynamodb-local.port} + + + + ${dynamodb-local.endpoint}/shell/ + + + + + + + + + + remove-it-database + post-integration-test + + stop + + + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java b/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java new file mode 100644 index 000000000000..e4bc598ce293 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/cloudwatch/CloudWatchReporter.java @@ -0,0 +1,315 @@ +/* + * 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 org.apache.hudi.aws.cloudwatch; + +import org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory; +import org.apache.hudi.common.util.Option; + +import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsync; +import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsyncClientBuilder; +import com.amazonaws.services.cloudwatch.model.Dimension; +import com.amazonaws.services.cloudwatch.model.MetricDatum; +import com.amazonaws.services.cloudwatch.model.PutMetricDataRequest; +import com.amazonaws.services.cloudwatch.model.PutMetricDataResult; +import com.amazonaws.services.cloudwatch.model.StandardUnit; +import com.codahale.metrics.Clock; +import com.codahale.metrics.Counter; +import com.codahale.metrics.Counting; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.ScheduledReporter; +import com.codahale.metrics.Timer; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.SortedMap; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * A reporter for publishing metrics to Amazon CloudWatch. It is responsible for collecting, converting DropWizard + * metrics to CloudWatch metrics and composing metrics payload. + */ +public class CloudWatchReporter extends ScheduledReporter { + + static final String DIMENSION_TABLE_NAME_KEY = "Table"; + static final String DIMENSION_METRIC_TYPE_KEY = "Metric Type"; + static final String DIMENSION_GAUGE_TYPE_VALUE = "gauge"; + static final String DIMENSION_COUNT_TYPE_VALUE = "count"; + + private static final Logger LOG = LogManager.getLogger(CloudWatchReporter.class); + + private final AmazonCloudWatchAsync cloudWatchClientAsync; + private final Clock clock; + private final String prefix; + private final String namespace; + private final int maxDatumsPerRequest; + + public static Builder forRegistry(MetricRegistry registry) { + return new Builder(registry); + } + + public static class Builder { + private MetricRegistry registry; + private Clock clock; + private String prefix; + private TimeUnit rateUnit; + private TimeUnit durationUnit; + private MetricFilter filter; + private String namespace; + private int maxDatumsPerRequest; + + private Builder(MetricRegistry registry) { + this.registry = registry; + this.clock = Clock.defaultClock(); + this.rateUnit = TimeUnit.SECONDS; + this.durationUnit = TimeUnit.MILLISECONDS; + this.filter = MetricFilter.ALL; + this.maxDatumsPerRequest = 20; + } + + public Builder withClock(Clock clock) { + this.clock = clock; + return this; + } + + public Builder prefixedWith(String prefix) { + this.prefix = prefix; + return this; + } + + public Builder convertRatesTo(TimeUnit rateUnit) { + this.rateUnit = rateUnit; + return this; + } + + public Builder convertDurationsTo(TimeUnit durationUnit) { + this.durationUnit = durationUnit; + return this; + } + + public Builder filter(MetricFilter filter) { + this.filter = filter; + return this; + } + + public Builder namespace(String namespace) { + this.namespace = namespace; + return this; + } + + public Builder maxDatumsPerRequest(int maxDatumsPerRequest) { + this.maxDatumsPerRequest = maxDatumsPerRequest; + return this; + } + + public CloudWatchReporter build(Properties props) { + return new CloudWatchReporter(registry, + getAmazonCloudWatchClient(props), + clock, + prefix, + namespace, + maxDatumsPerRequest, + filter, + rateUnit, + durationUnit); + } + + CloudWatchReporter build(AmazonCloudWatchAsync amazonCloudWatchAsync) { + return new CloudWatchReporter(registry, + amazonCloudWatchAsync, + clock, + prefix, + namespace, + maxDatumsPerRequest, + filter, + rateUnit, + durationUnit); + } + } + + protected CloudWatchReporter(MetricRegistry registry, + AmazonCloudWatchAsync cloudWatchClientAsync, + Clock clock, + String prefix, + String namespace, + int maxDatumsPerRequest, + MetricFilter filter, + TimeUnit rateUnit, + TimeUnit durationUnit) { + super(registry, "hudi-cloudWatch-reporter", filter, rateUnit, durationUnit); + this.cloudWatchClientAsync = cloudWatchClientAsync; + this.clock = clock; + this.prefix = prefix; + this.namespace = namespace; + this.maxDatumsPerRequest = maxDatumsPerRequest; + } + + private static AmazonCloudWatchAsync getAmazonCloudWatchClient(Properties props) { + return AmazonCloudWatchAsyncClientBuilder.standard() + .withCredentials(HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(props)) + .build(); + } + + @Override + public void report(SortedMap gauges, + SortedMap counters, + SortedMap histograms, + SortedMap meters, + SortedMap timers) { + LOG.info("Reporting Metrics to CloudWatch."); + + final long timestampMilliSec = clock.getTime(); + List metricsData = new ArrayList<>(); + + for (Map.Entry entry : gauges.entrySet()) { + processGauge(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData); + } + + for (Map.Entry entry : counters.entrySet()) { + processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData); + } + + for (Map.Entry entry : histograms.entrySet()) { + processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData); + //TODO: Publish other Histogram metrics to cloud watch + } + + for (Map.Entry entry : meters.entrySet()) { + processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData); + //TODO: Publish other Meter metrics to cloud watch + } + + for (Map.Entry entry : timers.entrySet()) { + processCounter(entry.getKey(), entry.getValue(), timestampMilliSec, metricsData); + //TODO: Publish other Timer metrics to cloud watch + } + + report(metricsData); + } + + private void report(List metricsData) { + List> cloudWatchFutures = new ArrayList<>(metricsData.size()); + List> partitions = new ArrayList<>(); + + for (int i = 0; i < metricsData.size(); i += maxDatumsPerRequest) { + int end = Math.min(metricsData.size(), i + maxDatumsPerRequest); + partitions.add(metricsData.subList(i, end)); + } + + for (List partition : partitions) { + PutMetricDataRequest request = new PutMetricDataRequest() + .withNamespace(namespace) + .withMetricData(partition); + + cloudWatchFutures.add(cloudWatchClientAsync.putMetricDataAsync(request)); + } + + for (final Future cloudWatchFuture : cloudWatchFutures) { + try { + cloudWatchFuture.get(30, TimeUnit.SECONDS); + } catch (final Exception ex) { + LOG.error("Error reporting metrics to CloudWatch. The data in this CloudWatch request " + + "may have been discarded, and not made it to CloudWatch.", ex); + } + } + } + + private void processGauge(final String metricName, + final Gauge gauge, + final long timestampMilliSec, + final List metricData) { + Option.ofNullable(gauge.getValue()) + .toJavaOptional() + .filter(value -> value instanceof Number) + .map(value -> (Number) value) + .ifPresent(value -> stageMetricDatum(metricName, + value.doubleValue(), + DIMENSION_GAUGE_TYPE_VALUE, + StandardUnit.None, + timestampMilliSec, + metricData)); + } + + private void processCounter(final String metricName, + final Counting counter, + final long timestampMilliSec, + final List metricData) { + stageMetricDatum(metricName, + counter.getCount(), + DIMENSION_COUNT_TYPE_VALUE, + StandardUnit.Count, + timestampMilliSec, + metricData); + } + + private void stageMetricDatum(String metricName, + double metricValue, + String metricType, + StandardUnit standardUnit, + long timestampMilliSec, + List metricData) { + String[] metricNameParts = metricName.split("\\.", 2); + String tableName = metricNameParts[0]; + + + metricData.add(new MetricDatum() + .withTimestamp(new Date(timestampMilliSec)) + .withMetricName(prefix(metricNameParts[1])) + .withValue(metricValue) + .withDimensions(getDimensions(tableName, metricType)) + .withUnit(standardUnit)); + } + + private List getDimensions(String tableName, String metricType) { + List dimensions = new ArrayList<>(); + dimensions.add(new Dimension() + .withName(DIMENSION_TABLE_NAME_KEY) + .withValue(tableName)); + dimensions.add(new Dimension() + .withName(DIMENSION_METRIC_TYPE_KEY) + .withValue(metricType)); + return dimensions; + } + + private String prefix(String... components) { + return MetricRegistry.name(prefix, components); + } + + @Override + public void stop() { + try { + super.stop(); + } finally { + try { + cloudWatchClientAsync.shutdown(); + } catch (Exception ex) { + LOG.warn("Exception while shutting down CloudWatch client.", ex); + } + } + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java new file mode 100644 index 000000000000..631b0fa8d534 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieAWSCredentialsProviderFactory.java @@ -0,0 +1,45 @@ +/* + * 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 org.apache.hudi.aws.credentials; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProviderChain; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Factory class for Hoodie AWSCredentialsProvider. + */ +public class HoodieAWSCredentialsProviderFactory { + public static AWSCredentialsProvider getAwsCredentialsProvider(Properties props) { + return getAwsCredentialsProviderChain(props); + } + + private static AWSCredentialsProvider getAwsCredentialsProviderChain(Properties props) { + List providers = new ArrayList<>(); + providers.add(new HoodieConfigAWSCredentialsProvider(props)); + providers.add(new DefaultAWSCredentialsProviderChain()); + AWSCredentialsProviderChain providerChain = new AWSCredentialsProviderChain(providers); + providerChain.setReuseLastProvider(true); + return providerChain; + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java new file mode 100644 index 000000000000..4e9cf383906a --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/credentials/HoodieConfigAWSCredentialsProvider.java @@ -0,0 +1,70 @@ +/* + * 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 org.apache.hudi.aws.credentials; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.BasicSessionCredentials; +import org.apache.hudi.config.HoodieAWSConfig; +import org.apache.hudi.common.util.StringUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Properties; + +/** + * Credentials provider which fetches AWS access key from Hoodie config. + */ +public class HoodieConfigAWSCredentialsProvider implements AWSCredentialsProvider { + + private static final Logger LOG = LogManager.getLogger(HoodieConfigAWSCredentialsProvider.class); + + private AWSCredentials awsCredentials; + + public HoodieConfigAWSCredentialsProvider(Properties props) { + String accessKey = props.getProperty(HoodieAWSConfig.AWS_ACCESS_KEY.key()); + String secretKey = props.getProperty(HoodieAWSConfig.AWS_SECRET_KEY.key()); + String sessionToken = props.getProperty(HoodieAWSConfig.AWS_SESSION_TOKEN.key()); + + if (StringUtils.isNullOrEmpty(accessKey) || StringUtils.isNullOrEmpty(secretKey)) { + LOG.debug("AWS access key or secret key not found in the Hudi configuration. " + + "Use default AWS credentials"); + } else { + this.awsCredentials = createCredentials(accessKey, secretKey, sessionToken); + } + } + + private static AWSCredentials createCredentials(String accessKey, String secretKey, + String sessionToken) { + return (sessionToken == null) + ? new BasicAWSCredentials(accessKey, secretKey) + : new BasicSessionCredentials(accessKey, secretKey, sessionToken); + } + + @Override + public AWSCredentials getCredentials() { + return this.awsCredentials; + } + + @Override + public void refresh() { + + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java b/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java new file mode 100644 index 000000000000..57348bea426c --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/transaction/lock/DynamoDBBasedLockProvider.java @@ -0,0 +1,220 @@ +/* + * 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 org.apache.hudi.aws.transaction.lock; + +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.regions.RegionUtils; +import com.amazonaws.services.dynamodbv2.AcquireLockOptions; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBLockClient; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBLockClientOptions; +import com.amazonaws.services.dynamodbv2.LockItem; +import com.amazonaws.services.dynamodbv2.model.AttributeDefinition; +import com.amazonaws.services.dynamodbv2.model.BillingMode; +import com.amazonaws.services.dynamodbv2.model.CreateTableRequest; +import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; +import com.amazonaws.services.dynamodbv2.model.KeyType; +import com.amazonaws.services.dynamodbv2.model.LockNotGrantedException; +import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput; +import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType; +import com.amazonaws.services.dynamodbv2.util.TableUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.DynamoDbBasedLockConfig; +import org.apache.hudi.exception.HoodieLockException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.NotThreadSafe; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY; + +/** + * A DynamoDB based lock. This {@link LockProvider} implementation allows to lock table operations + * using DynamoDB. Users need to have access to AWS DynamoDB to be able to use this lock. + */ +@NotThreadSafe +public class DynamoDBBasedLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(DynamoDBBasedLockProvider.class); + + private static final String DYNAMODB_ATTRIBUTE_NAME = "key"; + + private final AmazonDynamoDBLockClient client; + private final String tableName; + private final String dynamoDBPartitionKey; + protected LockConfiguration lockConfiguration; + private volatile LockItem lock; + + public DynamoDBBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + this(lockConfiguration, conf, null); + } + + public DynamoDBBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf, AmazonDynamoDB dynamoDB) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.tableName = lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key()); + this.dynamoDBPartitionKey = lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_PARTITION_KEY.key()); + long leaseDuration = Long.parseLong(lockConfiguration.getConfig().getString(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY)); + if (dynamoDB == null) { + dynamoDB = getDynamoDBClient(); + } + // build the dynamoDb lock client + this.client = new AmazonDynamoDBLockClient( + AmazonDynamoDBLockClientOptions.builder(dynamoDB, tableName) + .withTimeUnit(TimeUnit.MILLISECONDS) + .withLeaseDuration(leaseDuration) + .withHeartbeatPeriod(leaseDuration / 3) + .withCreateHeartbeatBackgroundThread(true) + .build()); + + if (!this.client.lockTableExists()) { + createLockTableInDynamoDB(dynamoDB, tableName); + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + lock = client.acquireLock(AcquireLockOptions.builder(dynamoDBPartitionKey) + .withAdditionalTimeToWaitForLock(time) + .withTimeUnit(TimeUnit.MILLISECONDS) + .build()); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (InterruptedException e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } catch (LockNotGrantedException e) { + return false; + } + return lock != null && !lock.isExpired(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null) { + return; + } + if (!client.releaseLock(lock)) { + LOG.warn("The lock has already been stolen"); + } + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + if (!client.releaseLock(lock)) { + LOG.warn("The lock has already been stolen"); + } + lock = null; + } + this.client.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public LockItem getLock() { + return lock; + } + + private AmazonDynamoDB getDynamoDBClient() { + String region = this.lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_REGION.key()); + String endpointURL = this.lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_ENDPOINT_URL.key()) == null + ? RegionUtils.getRegion(region).getServiceEndpoint(AmazonDynamoDB.ENDPOINT_PREFIX) + : this.lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_ENDPOINT_URL.key()); + AwsClientBuilder.EndpointConfiguration dynamodbEndpoint = + new AwsClientBuilder.EndpointConfiguration(endpointURL, region); + return AmazonDynamoDBClientBuilder.standard() + .withEndpointConfiguration(dynamodbEndpoint) + .withCredentials(HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(lockConfiguration.getConfig())) + .build(); + } + + private void createLockTableInDynamoDB(AmazonDynamoDB dynamoDB, String tableName) { + String billingMode = lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_BILLING_MODE.key()); + KeySchemaElement partitionKeyElement = new KeySchemaElement(); + partitionKeyElement.setAttributeName(DYNAMODB_ATTRIBUTE_NAME); + partitionKeyElement.setKeyType(KeyType.HASH); + + List keySchema = new ArrayList<>(); + keySchema.add(partitionKeyElement); + + Collection attributeDefinitions = new ArrayList<>(); + attributeDefinitions.add(new AttributeDefinition().withAttributeName(DYNAMODB_ATTRIBUTE_NAME).withAttributeType(ScalarAttributeType.S)); + + CreateTableRequest createTableRequest = new CreateTableRequest(tableName, keySchema); + createTableRequest.setAttributeDefinitions(attributeDefinitions); + createTableRequest.setBillingMode(billingMode); + if (billingMode.equals(BillingMode.PROVISIONED.name())) { + createTableRequest.setProvisionedThroughput(new ProvisionedThroughput() + .withReadCapacityUnits(Long.parseLong(lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY.key()))) + .withWriteCapacityUnits(Long.parseLong(lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY.key())))); + } + dynamoDB.createTable(createTableRequest); + + LOG.info("Creating dynamoDB table " + tableName + ", waiting for table to be active"); + try { + TableUtils.waitUntilActive(dynamoDB, tableName, Integer.parseInt(lockConfiguration.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT.key())), 20 * 1000); + } catch (TableUtils.TableNeverTransitionedToStateException e) { + throw new HoodieLockException("Created dynamoDB table never transits to active", e); + } catch (InterruptedException e) { + throw new HoodieLockException("Thread interrupted while waiting for dynamoDB table to turn active", e); + } + LOG.info("Created dynamoDB table " + tableName); + } + + private void checkRequiredProps(final LockConfiguration config) { + ValidationUtils.checkArgument(config.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key()) != null); + ValidationUtils.checkArgument(config.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_REGION.key()) != null); + ValidationUtils.checkArgument(config.getConfig().getString(DynamoDbBasedLockConfig.DYNAMODB_LOCK_PARTITION_KEY.key()) != null); + config.getConfig().putIfAbsent(DynamoDbBasedLockConfig.DYNAMODB_LOCK_BILLING_MODE.key(), BillingMode.PAY_PER_REQUEST.name()); + config.getConfig().putIfAbsent(DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY.key(), "20"); + config.getConfig().putIfAbsent(DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY.key(), "10"); + config.getConfig().putIfAbsent(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT.key(), "600000"); + } + + private String generateLogSuffixString() { + return StringUtils.join("DynamoDb table = ", tableName, ", partition key = ", dynamoDBPartitionKey); + } + + protected String generateLogStatement(LockState state, String suffix) { + return StringUtils.join(state.name(), " lock at ", suffix); + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java new file mode 100644 index 000000000000..19c63ea78655 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java @@ -0,0 +1,110 @@ +/* + * 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 org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; + +import com.amazonaws.regions.RegionUtils; +import com.amazonaws.services.dynamodbv2.model.BillingMode; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX; + +/** + * Hoodie Configs for Locks. + */ +@ConfigClassProperty(name = "DynamoDB based Locks Configurations", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Configs that control DynamoDB based locking mechanisms required for concurrency control " + + " between writers to a Hudi table. Concurrency between Hudi's own table services " + + " are auto managed internally.") +public class DynamoDbBasedLockConfig extends HoodieConfig { + + // configs for DynamoDb based locks + public static final String DYNAMODB_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "dynamodb."; + + public static final ConfigProperty DYNAMODB_LOCK_TABLE_NAME = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table") + .noDefaultValue() + .sinceVersion("0.10.0") + .withDocumentation("For DynamoDB based lock provider, the name of the DynamoDB table acting as lock table"); + + public static final ConfigProperty DYNAMODB_LOCK_PARTITION_KEY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "partition_key") + .noDefaultValue() + .sinceVersion("0.10.0") + .withInferFunction(cfg -> { + if (cfg.contains(HoodieTableConfig.NAME)) { + return Option.of(cfg.getString(HoodieTableConfig.NAME)); + } + return Option.empty(); + }) + .withDocumentation("For DynamoDB based lock provider, the partition key for the DynamoDB lock table. " + + "Each Hudi dataset should has it's unique key so concurrent writers could refer to the same partition key." + + " By default we use the Hudi table name specified to be the partition key"); + + public static final ConfigProperty DYNAMODB_LOCK_REGION = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "region") + .defaultValue("us-east-1") + .sinceVersion("0.10.0") + .withInferFunction(cfg -> { + String regionFromEnv = System.getenv("AWS_REGION"); + if (regionFromEnv != null) { + return Option.of(RegionUtils.getRegion(regionFromEnv).getName()); + } + return Option.empty(); + }) + .withDocumentation("For DynamoDB based lock provider, the region used in endpoint for Amazon DynamoDB service." + + " Would try to first get it from AWS_REGION environment variable. If not find, by default use us-east-1"); + + public static final ConfigProperty DYNAMODB_LOCK_BILLING_MODE = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "billing_mode") + .defaultValue(BillingMode.PAY_PER_REQUEST.name()) + .sinceVersion("0.10.0") + .withDocumentation("For DynamoDB based lock provider, by default it is PAY_PER_REQUEST mode"); + + public static final ConfigProperty DYNAMODB_LOCK_READ_CAPACITY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "read_capacity") + .defaultValue("20") + .sinceVersion("0.10.0") + .withDocumentation("For DynamoDB based lock provider, read capacity units when using PROVISIONED billing mode"); + + public static final ConfigProperty DYNAMODB_LOCK_WRITE_CAPACITY = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "write_capacity") + .defaultValue("10") + .sinceVersion("0.10.0") + .withDocumentation("For DynamoDB based lock provider, write capacity units when using PROVISIONED billing mode"); + + public static final ConfigProperty DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "table_creation_timeout") + .defaultValue(String.valueOf(10 * 60 * 1000)) + .sinceVersion("0.10.0") + .withDocumentation("For DynamoDB based lock provider, the maximum number of milliseconds to wait for creating DynamoDB table"); + + public static final ConfigProperty DYNAMODB_ENDPOINT_URL = ConfigProperty + .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "endpoint_url") + .defaultValue("us-east-1") + .sinceVersion("0.11.0") + .withDocumentation("For DynamoDB based lock provider, the url endpoint used for Amazon DynamoDB service." + + " Useful for development with a local dynamodb instance."); +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java new file mode 100644 index 000000000000..623704232e41 --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java @@ -0,0 +1,152 @@ +/* + * 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 org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; +import javax.annotation.concurrent.Immutable; + +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_BILLING_MODE; +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_PARTITION_KEY; +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY; +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_REGION; +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME; +import static org.apache.hudi.config.DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY; + +/** + * Configurations used by the AWS credentials and AWS DynamoDB based lock. + */ +@Immutable +@ConfigClassProperty(name = "Amazon Web Services Configs", + groupName = ConfigGroups.Names.AWS, + description = "Amazon Web Services configurations to access resources like Amazon DynamoDB (for locks)," + + " Amazon CloudWatch (metrics).") +public class HoodieAWSConfig extends HoodieConfig { + public static final ConfigProperty AWS_ACCESS_KEY = ConfigProperty + .key("hoodie.aws.access.key") + .noDefaultValue() + .sinceVersion("0.10.0") + .withDocumentation("AWS access key id"); + + public static final ConfigProperty AWS_SECRET_KEY = ConfigProperty + .key("hoodie.aws.secret.key") + .noDefaultValue() + .sinceVersion("0.10.0") + .withDocumentation("AWS secret key"); + + public static final ConfigProperty AWS_SESSION_TOKEN = ConfigProperty + .key("hoodie.aws.session.token") + .noDefaultValue() + .sinceVersion("0.10.0") + .withDocumentation("AWS session token"); + + private HoodieAWSConfig() { + super(); + } + + public static HoodieAWSConfig.Builder newBuilder() { + return new HoodieAWSConfig.Builder(); + } + + public String getAWSAccessKey() { + return getString(AWS_ACCESS_KEY); + } + + public String getAWSSecretKey() { + return getString(AWS_SECRET_KEY); + } + + public String getAWSSessionToken() { + return getString(AWS_SESSION_TOKEN); + } + + public static class Builder { + + private final HoodieAWSConfig awsConfig = new HoodieAWSConfig(); + + public HoodieAWSConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.awsConfig.getProps().load(reader); + return this; + } + } + + public HoodieAWSConfig.Builder fromProperties(Properties props) { + this.awsConfig.getProps().putAll(props); + return this; + } + + public HoodieAWSConfig.Builder withAccessKey(String accessKey) { + awsConfig.setValue(AWS_ACCESS_KEY, accessKey); + return this; + } + + public HoodieAWSConfig.Builder withSecretKey(String secretKey) { + awsConfig.setValue(AWS_SECRET_KEY, secretKey); + return this; + } + + public HoodieAWSConfig.Builder withSessionToken(String sessionToken) { + awsConfig.setValue(AWS_SESSION_TOKEN, sessionToken); + return this; + } + + public Builder withDynamoDBTable(String dynamoDbTableName) { + awsConfig.setValue(DYNAMODB_LOCK_TABLE_NAME, dynamoDbTableName); + return this; + } + + public Builder withDynamoDBPartitionKey(String partitionKey) { + awsConfig.setValue(DYNAMODB_LOCK_PARTITION_KEY, partitionKey); + return this; + } + + public Builder withDynamoDBRegion(String region) { + awsConfig.setValue(DYNAMODB_LOCK_REGION, region); + return this; + } + + public Builder withDynamoDBBillingMode(String mode) { + awsConfig.setValue(DYNAMODB_LOCK_BILLING_MODE, mode); + return this; + } + + public Builder withDynamoDBReadCapacity(String capacity) { + awsConfig.setValue(DYNAMODB_LOCK_READ_CAPACITY, capacity); + return this; + } + + public Builder withDynamoDBWriteCapacity(String capacity) { + awsConfig.setValue(DYNAMODB_LOCK_WRITE_CAPACITY, capacity); + return this; + } + + public HoodieAWSConfig build() { + awsConfig.setDefaults(HoodieAWSConfig.class.getName()); + return awsConfig; + } + } +} diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java new file mode 100644 index 000000000000..e4e46d5a1f7b --- /dev/null +++ b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieMetricsCloudWatchConfig.java @@ -0,0 +1,87 @@ +/* + * 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 org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import java.util.Properties; + +@ConfigClassProperty( + name = "Metrics Configurations for Amazon CloudWatch", + groupName = ConfigGroups.Names.METRICS, + description = + "Enables reporting on Hudi metrics using Amazon CloudWatch. " + + " Hudi publishes metrics on every commit, clean, rollback etc.") +public class HoodieMetricsCloudWatchConfig extends HoodieConfig { + + public static final String CLOUDWATCH_PREFIX = "hoodie.metrics.cloudwatch"; + + public static final ConfigProperty REPORT_PERIOD_SECONDS = ConfigProperty + .key(CLOUDWATCH_PREFIX + ".report.period.seconds") + .defaultValue(60) + .sinceVersion("0.10.0") + .withDocumentation("Reporting interval in seconds"); + + public static final ConfigProperty METRIC_PREFIX = ConfigProperty + .key(CLOUDWATCH_PREFIX + ".metric.prefix") + .defaultValue("") + .sinceVersion("0.10.0") + .withDocumentation("Metric prefix of reporter"); + + public static final ConfigProperty METRIC_NAMESPACE = ConfigProperty + .key(CLOUDWATCH_PREFIX + ".namespace") + .defaultValue("Hudi") + .sinceVersion("0.10.0") + .withDocumentation("Namespace of reporter"); + /* + Amazon CloudWatch allows a maximum of 20 metrics per request. Choosing this as the default maximum. + Reference: https://docs.aws.amazon.com/AmazonCloudWatch/latest/APIReference/API_PutMetricData.html + */ + public static final ConfigProperty MAX_DATUMS_PER_REQUEST = + ConfigProperty.key(CLOUDWATCH_PREFIX + ".maxDatumsPerRequest") + .defaultValue(20) + .sinceVersion("0.10.0") + .withDocumentation("Max number of Datums per request"); + + public HoodieMetricsCloudWatchConfig() { + super(); + } + + public static HoodieMetricsCloudWatchConfig.Builder newBuilder() { + return new HoodieMetricsCloudWatchConfig.Builder(); + } + + public static class Builder { + + private HoodieMetricsCloudWatchConfig hoodieMetricsCloudWatchConfig = new HoodieMetricsCloudWatchConfig(); + + public HoodieMetricsCloudWatchConfig.Builder fromProperties(Properties props) { + this.hoodieMetricsCloudWatchConfig.getProps().putAll(props); + return this; + } + + public HoodieMetricsCloudWatchConfig build() { + hoodieMetricsCloudWatchConfig.setDefaults(HoodieMetricsCloudWatchConfig.class.getName()); + return hoodieMetricsCloudWatchConfig; + } + } +} diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java new file mode 100644 index 000000000000..051fe81e8b0f --- /dev/null +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/TestHoodieAWSCredentialsProviderFactory.java @@ -0,0 +1,42 @@ +/* + * 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 org.apache.hudi.aws; + +import com.amazonaws.auth.BasicSessionCredentials; +import org.apache.hudi.config.HoodieAWSConfig; +import org.apache.hudi.common.config.HoodieConfig; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestHoodieAWSCredentialsProviderFactory { + + @Test + public void testGetAWSCredentials() { + HoodieConfig cfg = new HoodieConfig(); + cfg.setValue(HoodieAWSConfig.AWS_ACCESS_KEY, "random-access-key"); + cfg.setValue(HoodieAWSConfig.AWS_SECRET_KEY, "random-secret-key"); + cfg.setValue(HoodieAWSConfig.AWS_SESSION_TOKEN, "random-session-token"); + BasicSessionCredentials credentials = (BasicSessionCredentials) org.apache.hudi.aws.credentials.HoodieAWSCredentialsProviderFactory.getAwsCredentialsProvider(cfg.getProps()).getCredentials(); + assertEquals("random-access-key", credentials.getAWSAccessKeyId()); + assertEquals("random-secret-key", credentials.getAWSSecretKey()); + assertEquals("random-session-token", credentials.getSessionToken()); + } +} diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java b/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java new file mode 100644 index 000000000000..85f551e6fda8 --- /dev/null +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/cloudwatch/TestCloudWatchReporter.java @@ -0,0 +1,183 @@ +/* + * 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 org.apache.hudi.aws.cloudwatch; + +import com.amazonaws.services.cloudwatch.AmazonCloudWatchAsync; +import com.amazonaws.services.cloudwatch.model.Dimension; +import com.amazonaws.services.cloudwatch.model.MetricDatum; +import com.amazonaws.services.cloudwatch.model.PutMetricDataRequest; +import com.amazonaws.services.cloudwatch.model.PutMetricDataResult; +import com.codahale.metrics.Clock; +import com.codahale.metrics.Counter; +import com.codahale.metrics.ExponentiallyDecayingReservoir; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Timer; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_COUNT_TYPE_VALUE; +import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_GAUGE_TYPE_VALUE; +import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_METRIC_TYPE_KEY; +import static org.apache.hudi.aws.cloudwatch.CloudWatchReporter.DIMENSION_TABLE_NAME_KEY; +import static org.junit.jupiter.api.Assertions.assertEquals; + +@ExtendWith(MockitoExtension.class) +public class TestCloudWatchReporter { + + private static final String NAMESPACE = "Hudi Test"; + private static final String PREFIX = "testPrefix"; + private static final String TABLE_NAME = "testTable"; + private static final int MAX_DATUMS_PER_REQUEST = 2; + + @Mock + MetricRegistry metricRegistry; + + @Mock + AmazonCloudWatchAsync cloudWatchAsync; + + @Mock + CompletableFuture cloudWatchFuture; + + @Captor + ArgumentCaptor putMetricDataRequestCaptor; + + CloudWatchReporter reporter; + + @BeforeEach + public void setup() { + reporter = CloudWatchReporter.forRegistry(metricRegistry) + .namespace(NAMESPACE) + .prefixedWith(PREFIX) + .maxDatumsPerRequest(MAX_DATUMS_PER_REQUEST) + .withClock(Clock.defaultClock()) + .filter(MetricFilter.ALL) + .convertRatesTo(TimeUnit.SECONDS) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .build(cloudWatchAsync); + + Mockito.when(cloudWatchAsync.putMetricDataAsync(ArgumentMatchers.any())).thenReturn(cloudWatchFuture); + } + + @Test + public void testReporter() { + SortedMap gauges = new TreeMap<>(); + Gauge gauge1 = () -> 100L; + Gauge gauge2 = () -> 100.1; + gauges.put(TABLE_NAME + ".gauge1", gauge1); + gauges.put(TABLE_NAME + ".gauge2", gauge2); + + SortedMap counters = new TreeMap<>(); + Counter counter1 = new Counter(); + counter1.inc(200); + counters.put(TABLE_NAME + ".counter1", counter1); + + SortedMap histograms = new TreeMap<>(); + Histogram histogram1 = new Histogram(new ExponentiallyDecayingReservoir()); + histogram1.update(300); + histograms.put(TABLE_NAME + ".histogram1", histogram1); + + SortedMap meters = new TreeMap<>(); + Meter meter1 = new Meter(); + meter1.mark(400); + meters.put(TABLE_NAME + ".meter1", meter1); + + SortedMap timers = new TreeMap<>(); + Timer timer1 = new Timer(); + timer1.update(100, TimeUnit.SECONDS); + timers.put(TABLE_NAME + ".timer1", timer1); + + Mockito.when(metricRegistry.getGauges(MetricFilter.ALL)).thenReturn(gauges); + Mockito.when(metricRegistry.getCounters(MetricFilter.ALL)).thenReturn(counters); + Mockito.when(metricRegistry.getHistograms(MetricFilter.ALL)).thenReturn(histograms); + Mockito.when(metricRegistry.getMeters(MetricFilter.ALL)).thenReturn(meters); + Mockito.when(metricRegistry.getTimers(MetricFilter.ALL)).thenReturn(timers); + + reporter.report(); + + // Since there are 6 metrics in total, and max datums per request is 2 we would expect 3 calls to CloudWatch + // with 2 datums in each + Mockito.verify(cloudWatchAsync, Mockito.times(3)).putMetricDataAsync(putMetricDataRequestCaptor.capture()); + Assertions.assertEquals(NAMESPACE, putMetricDataRequestCaptor.getValue().getNamespace()); + + List putMetricDataRequests = putMetricDataRequestCaptor.getAllValues(); + putMetricDataRequests.forEach(request -> assertEquals(2, request.getMetricData().size())); + + List metricDataBatch1 = putMetricDataRequests.get(0).getMetricData(); + assertEquals(PREFIX + ".gauge1", metricDataBatch1.get(0).getMetricName()); + assertEquals(Double.valueOf(gauge1.getValue()), metricDataBatch1.get(0).getValue()); + assertDimensions(metricDataBatch1.get(0).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE); + + assertEquals(PREFIX + ".gauge2", metricDataBatch1.get(1).getMetricName()); + assertEquals(gauge2.getValue(), metricDataBatch1.get(1).getValue()); + assertDimensions(metricDataBatch1.get(1).getDimensions(), DIMENSION_GAUGE_TYPE_VALUE); + + List metricDataBatch2 = putMetricDataRequests.get(1).getMetricData(); + assertEquals(PREFIX + ".counter1", metricDataBatch2.get(0).getMetricName()); + assertEquals(counter1.getCount(), metricDataBatch2.get(0).getValue().longValue()); + assertDimensions(metricDataBatch2.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE); + + assertEquals(PREFIX + ".histogram1", metricDataBatch2.get(1).getMetricName()); + assertEquals(histogram1.getCount(), metricDataBatch2.get(1).getValue().longValue()); + assertDimensions(metricDataBatch2.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE); + + List metricDataBatch3 = putMetricDataRequests.get(2).getMetricData(); + assertEquals(PREFIX + ".meter1", metricDataBatch3.get(0).getMetricName()); + assertEquals(meter1.getCount(), metricDataBatch3.get(0).getValue().longValue()); + assertDimensions(metricDataBatch3.get(0).getDimensions(), DIMENSION_COUNT_TYPE_VALUE); + + assertEquals(PREFIX + ".timer1", metricDataBatch3.get(1).getMetricName()); + assertEquals(timer1.getCount(), metricDataBatch3.get(1).getValue().longValue()); + assertDimensions(metricDataBatch3.get(1).getDimensions(), DIMENSION_COUNT_TYPE_VALUE); + + reporter.stop(); + Mockito.verify(cloudWatchAsync).shutdown(); + } + + private void assertDimensions(List actualDimensions, String metricTypeDimensionVal) { + assertEquals(2, actualDimensions.size()); + + Dimension expectedTableNameDimension = new Dimension() + .withName(DIMENSION_TABLE_NAME_KEY) + .withValue(TABLE_NAME); + Dimension expectedMetricTypeDimension = new Dimension() + .withName(DIMENSION_METRIC_TYPE_KEY) + .withValue(metricTypeDimensionVal); + + assertEquals(expectedTableNameDimension, actualDimensions.get(0)); + assertEquals(expectedMetricTypeDimension, actualDimensions.get(1)); + } +} \ No newline at end of file diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java new file mode 100644 index 000000000000..d2ab0375e050 --- /dev/null +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/transaction/integ/ITTestDynamoDBBasedLockProvider.java @@ -0,0 +1,120 @@ +/* + * 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 org.apache.hudi.aws.transaction.integ; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder; +import com.amazonaws.services.dynamodbv2.model.BillingMode; +import org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.config.DynamoDbBasedLockConfig; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.UUID; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY; + +/** + * Test for {@link DynamoDBBasedLockProvider}. + * Set it as integration test because it requires setting up docker environment. + */ +public class ITTestDynamoDBBasedLockProvider { + + private static LockConfiguration lockConfiguration; + private static AmazonDynamoDB dynamoDb; + + private static final String TABLE_NAME_PREFIX = "testDDBTable-"; + private static final String REGION = "us-east-2"; + + @BeforeAll + public static void setup() throws InterruptedException { + Properties properties = new Properties(); + properties.setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_BILLING_MODE.key(), BillingMode.PAY_PER_REQUEST.name()); + // properties.setProperty(AWSLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX); + properties.setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_PARTITION_KEY.key(), "testKey"); + properties.setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_REGION.key(), REGION); + properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000"); + properties.setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_READ_CAPACITY.key(), "0"); + properties.setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_WRITE_CAPACITY.key(), "0"); + lockConfiguration = new LockConfiguration(properties); + dynamoDb = getDynamoClientWithLocalEndpoint(); + } + + @Test + public void testAcquireLock() { + lockConfiguration.getConfig().setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + } + + @Test + public void testUnlock() { + lockConfiguration.getConfig().setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + } + + @Test + public void testReentrantLock() { + lockConfiguration.getConfig().setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + Assertions.assertTrue(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + Assertions.assertFalse(dynamoDbBasedLockProvider.tryLock(lockConfiguration.getConfig() + .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS)); + dynamoDbBasedLockProvider.unlock(); + } + + @Test + public void testUnlockWithoutLock() { + lockConfiguration.getConfig().setProperty(DynamoDbBasedLockConfig.DYNAMODB_LOCK_TABLE_NAME.key(), TABLE_NAME_PREFIX + UUID.randomUUID()); + DynamoDBBasedLockProvider dynamoDbBasedLockProvider = new DynamoDBBasedLockProvider(lockConfiguration, null, dynamoDb); + dynamoDbBasedLockProvider.unlock(); + } + + private static AmazonDynamoDB getDynamoClientWithLocalEndpoint() { + String endpoint = System.getProperty("dynamodb-local.endpoint"); + if (endpoint == null || endpoint.isEmpty()) { + throw new IllegalStateException("dynamodb-local.endpoint system property not set"); + } + return AmazonDynamoDBClientBuilder.standard() + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(endpoint, REGION)) + .withCredentials(getCredentials()) + .build(); + } + + private static AWSCredentialsProvider getCredentials() { + return new AWSStaticCredentialsProvider(new BasicAWSCredentials("random-access-key", "random-secret-key")); + } +} diff --git a/hudi-aws/src/test/resources/log4j-surefire.properties b/hudi-aws/src/test/resources/log4j-surefire.properties new file mode 100644 index 000000000000..a59d4ebe2b19 --- /dev/null +++ b/hudi-aws/src/test/resources/log4j-surefire.properties @@ -0,0 +1,25 @@ +### +# 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. +### +log4j.rootLogger=WARN, A1 +log4j.category.org.apache=INFO +log4j.category.org.apache.parquet.hadoop=WARN +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index 26c0e685a931..29bdf85ab08c 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 @@ -234,6 +234,12 @@ org.springframework.shell spring-shell ${spring.shell.version} + + + com.google.guava + guava + + diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java index 092f9270b964..9adae1daa533 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java @@ -40,14 +40,21 @@ public class ClusteringCommand implements CommandMarker { private static final Logger LOG = LogManager.getLogger(ClusteringCommand.class); + /** + * Schedule clustering table service. + *

+ * Example: + * > connect --path {path to hudi table} + * > clustering schedule --sparkMaster local --sparkMemory 2g + */ @CliCommand(value = "clustering schedule", help = "Schedule Clustering") public String scheduleClustering( - @CliOption(key = "sparkMemory", help = "Spark executor memory", - unspecifiedDefaultValue = "1G") final String sparkMemory, - @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for clustering", - unspecifiedDefaultValue = "") final String propsFilePath, - @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array", - unspecifiedDefaultValue = "") final String[] configs) throws Exception { + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master, + @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1g", help = "Spark executor memory") final String sparkMemory, + @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations " + + "for hoodie client for clustering", unspecifiedDefaultValue = "") final String propsFilePath, + @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can " + + "be passed here in the form of an array", unspecifiedDefaultValue = "") final String[] configs) throws Exception { HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); @@ -59,8 +66,8 @@ public String scheduleClustering( // First get a clustering instant time and pass it to spark launcher for scheduling clustering String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); - sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_SCHEDULE.toString(), client.getBasePath(), - client.getTableConfig().getTableName(), clusteringInstantTime, sparkMemory, propsFilePath); + sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_SCHEDULE.toString(), master, sparkMemory, + client.getBasePath(), client.getTableConfig().getTableName(), clusteringInstantTime, propsFilePath); UtilHelpers.validateAndAddProperties(configs, sparkLauncher); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); @@ -71,21 +78,25 @@ public String scheduleClustering( return "Succeeded to schedule clustering for " + clusteringInstantTime; } + /** + * Run clustering table service. + *

+ * Example: + * > connect --path {path to hudi table} + * > clustering schedule --sparkMaster local --sparkMemory 2g + * > clustering run --sparkMaster local --sparkMemory 2g --clusteringInstant 20211124005208 + */ @CliCommand(value = "clustering run", help = "Run Clustering") public String runClustering( - @CliOption(key = "parallelism", help = "Parallelism for hoodie clustering", - unspecifiedDefaultValue = "1") final String parallelism, - @CliOption(key = "sparkMemory", help = "Spark executor memory", - unspecifiedDefaultValue = "4G") final String sparkMemory, - @CliOption(key = "retry", help = "Number of retries", - unspecifiedDefaultValue = "1") final String retry, - @CliOption(key = "clusteringInstant", help = "Clustering instant time", - mandatory = true) final String clusteringInstantTime, - @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting", - unspecifiedDefaultValue = "") final String propsFilePath, - @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array", - unspecifiedDefaultValue = "") final String[] configs - ) throws Exception { + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master, + @CliOption(key = "sparkMemory", help = "Spark executor memory", unspecifiedDefaultValue = "4g") final String sparkMemory, + @CliOption(key = "parallelism", help = "Parallelism for hoodie clustering", unspecifiedDefaultValue = "1") final String parallelism, + @CliOption(key = "retry", help = "Number of retries", unspecifiedDefaultValue = "1") final String retry, + @CliOption(key = "clusteringInstant", help = "Clustering instant time", mandatory = true) final String clusteringInstantTime, + @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for " + + "hoodie client for compacting", unspecifiedDefaultValue = "") final String propsFilePath, + @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be " + + "passed here in the form of an array", unspecifiedDefaultValue = "") final String[] configs) throws Exception { HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); @@ -93,8 +104,9 @@ public String runClustering( String sparkPropertiesPath = Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala()); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_RUN.toString(), client.getBasePath(), - client.getTableConfig().getTableName(), clusteringInstantTime, parallelism, sparkMemory, retry, propsFilePath); + sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_RUN.toString(), master, sparkMemory, + client.getBasePath(), client.getTableConfig().getTableName(), clusteringInstantTime, + parallelism, retry, propsFilePath); UtilHelpers.validateAndAddProperties(configs, sparkLauncher); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index ac898a8e644f..94e56102dbd5 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -18,20 +18,25 @@ package org.apache.hudi.cli.commands; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.hudi.cli.HoodieCLI; +import org.apache.hudi.cli.HoodiePrintHelper; +import org.apache.hudi.cli.TableHeader; import org.apache.hudi.cli.utils.SparkUtil; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.annotation.CliCommand; @@ -40,25 +45,44 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; /** * CLI commands to operate on the Metadata Table. + *

+ *

+ * Example: + * The default spark.master conf is set to yarn. If you are running on a local deployment, + * we can set the spark master to local using set conf command. + * > set --conf SPARK_MASTER=local[2] + *

+ * Connect to the table + * > connect --path {path to hudi table} + *

+ * Run metadata commands + * > metadata list-partitions */ @Component public class MetadataCommand implements CommandMarker { - private JavaSparkContext jsc; + private static final Logger LOG = LogManager.getLogger(MetadataCommand.class); private static String metadataBaseDirectory; + private JavaSparkContext jsc; /** * Sets the directory to store/read Metadata Table. - * + *

* This can be used to store the metadata table away from the dataset directory. - * - Useful for testing as well as for using via the HUDI CLI so that the actual dataset is not written to. - * - Useful for testing Metadata Table performance and operations on existing datasets before enabling. + * - Useful for testing as well as for using via the HUDI CLI so that the actual dataset is not written to. + * - Useful for testing Metadata Table performance and operations on existing datasets before enabling. */ public static void setMetadataBaseDirectory(String metadataDir) { ValidationUtils.checkState(metadataBaseDirectory == null, @@ -75,8 +99,7 @@ public static String getMetadataTableBasePath(String tableBasePath) { @CliCommand(value = "metadata set", help = "Set options for Metadata Table") public String set(@CliOption(key = {"metadataDir"}, - help = "Directory to read/write metadata table (can be different from dataset)", unspecifiedDefaultValue = "") - final String metadataDir) { + help = "Directory to read/write metadata table (can be different from dataset)", unspecifiedDefaultValue = "") final String metadataDir) { if (!metadataDir.isEmpty()) { setMetadataBaseDirectory(metadataDir); } @@ -152,16 +175,22 @@ public String stats() throws IOException { config, HoodieCLI.basePath, "/tmp"); Map stats = metadata.stats(); - StringBuffer out = new StringBuffer("\n"); - out.append(String.format("Base path: %s\n", getMetadataTableBasePath(HoodieCLI.basePath))); + final List rows = new ArrayList<>(); for (Map.Entry entry : stats.entrySet()) { - out.append(String.format("%s: %s\n", entry.getKey(), entry.getValue())); + Comparable[] row = new Comparable[2]; + row[0] = entry.getKey(); + row[1] = entry.getValue(); + rows.add(row); } - return out.toString(); + TableHeader header = new TableHeader() + .addTableHeaderField("stat key") + .addTableHeaderField("stat value"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", + false, Integer.MAX_VALUE, false, rows); } - @CliCommand(value = "metadata list-partitions", help = "Print a list of all partitions from the metadata") + @CliCommand(value = "metadata list-partitions", help = "List all partitions from metadata") public String listPartitions() throws IOException { HoodieCLI.getTableMetaClient(); initJavaSparkContext(); @@ -169,55 +198,158 @@ public String listPartitions() throws IOException { HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, HoodieCLI.basePath, "/tmp"); - StringBuffer out = new StringBuffer("\n"); if (!metadata.enabled()) { - out.append("=== Metadata Table not initilized. Using file listing to get list of partitions. ===\n\n"); + return "[ERROR] Metadata Table not enabled/initialized\n\n"; } - long t1 = System.currentTimeMillis(); + HoodieTimer timer = new HoodieTimer().startTimer(); List partitions = metadata.getAllPartitionPaths(); - long t2 = System.currentTimeMillis(); - - int[] count = {0}; - partitions.stream().sorted((p1, p2) -> p2.compareTo(p1)).forEach(p -> { - out.append(p); - if (++count[0] % 15 == 0) { - out.append("\n"); - } else { - out.append(", "); - } - }); + LOG.debug("Took " + timer.endTimer() + " ms"); - out.append(String.format("\n\n=== List of partitions retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + final List rows = new ArrayList<>(); + partitions.stream().sorted(Comparator.reverseOrder()).forEach(p -> { + Comparable[] row = new Comparable[1]; + row[0] = p; + rows.add(row); + }); - return out.toString(); + TableHeader header = new TableHeader().addTableHeaderField("partition"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", + false, Integer.MAX_VALUE, false, rows); } @CliCommand(value = "metadata list-files", help = "Print a list of all files in a partition from the metadata") public String listFiles( - @CliOption(key = {"partition"}, help = "Name of the partition to list files", mandatory = true) - final String partition) throws IOException { + @CliOption(key = {"partition"}, help = "Name of the partition to list files", mandatory = true) final String partition) throws IOException { HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); - HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); - StringBuffer out = new StringBuffer("\n"); if (!metaReader.enabled()) { - out.append("=== Metadata Table not initialized. Using file listing to get list of files in partition. ===\n\n"); + return "[ERROR] Metadata Table not enabled/initialized\n\n"; } - long t1 = System.currentTimeMillis(); + HoodieTimer timer = new HoodieTimer().startTimer(); FileStatus[] statuses = metaReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition)); - long t2 = System.currentTimeMillis(); + LOG.debug("Took " + timer.endTimer() + " ms"); - Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(p -> { - out.append("\t" + p.getPath().getName()); - out.append("\n"); + final List rows = new ArrayList<>(); + Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(f -> { + Comparable[] row = new Comparable[1]; + row[0] = f; + rows.add(row); }); - out.append(String.format("\n=== Files in partition retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + TableHeader header = new TableHeader().addTableHeaderField("file path"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", + false, Integer.MAX_VALUE, false, rows); + } + + @CliCommand(value = "metadata validate-files", help = "Validate all files in all partitions from the metadata") + public String validateFiles( + @CliOption(key = {"verbose"}, help = "Print all file details", unspecifiedDefaultValue = "false") final boolean verbose) throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); + HoodieBackedTableMetadata metadataReader = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + + if (!metadataReader.enabled()) { + return "[ERROR] Metadata Table not enabled/initialized\n\n"; + } + + HoodieMetadataConfig fsConfig = HoodieMetadataConfig.newBuilder().enable(false).build(); + HoodieBackedTableMetadata fsMetaReader = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath, "/tmp"); + + HoodieTimer timer = new HoodieTimer().startTimer(); + List metadataPartitions = metadataReader.getAllPartitionPaths(); + LOG.debug("Listing partitions Took " + timer.endTimer() + " ms"); + List fsPartitions = fsMetaReader.getAllPartitionPaths(); + Collections.sort(fsPartitions); + Collections.sort(metadataPartitions); + + Set allPartitions = new HashSet<>(); + allPartitions.addAll(fsPartitions); + allPartitions.addAll(metadataPartitions); + + if (!fsPartitions.equals(metadataPartitions)) { + LOG.error("FS partition listing is not matching with metadata partition listing!"); + LOG.error("All FS partitions: " + Arrays.toString(fsPartitions.toArray())); + LOG.error("All Metadata partitions: " + Arrays.toString(metadataPartitions.toArray())); + } + + final List rows = new ArrayList<>(); + for (String partition : allPartitions) { + Map fileStatusMap = new HashMap<>(); + Map metadataFileStatusMap = new HashMap<>(); + FileStatus[] metadataStatuses = metadataReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition)); + Arrays.stream(metadataStatuses).forEach(entry -> metadataFileStatusMap.put(entry.getPath().getName(), entry)); + FileStatus[] fsStatuses = fsMetaReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition)); + Arrays.stream(fsStatuses).forEach(entry -> fileStatusMap.put(entry.getPath().getName(), entry)); + + Set allFiles = new HashSet<>(); + allFiles.addAll(fileStatusMap.keySet()); + allFiles.addAll(metadataFileStatusMap.keySet()); + + for (String file : allFiles) { + Comparable[] row = new Comparable[6]; + row[0] = partition; + FileStatus fsFileStatus = fileStatusMap.get(file); + FileStatus metaFileStatus = metadataFileStatusMap.get(file); + boolean doesFsFileExists = fsFileStatus != null; + boolean doesMetadataFileExists = metaFileStatus != null; + long fsFileLength = doesFsFileExists ? fsFileStatus.getLen() : 0; + long metadataFileLength = doesMetadataFileExists ? metaFileStatus.getLen() : 0; + row[1] = file; + row[2] = doesFsFileExists; + row[3] = doesMetadataFileExists; + row[4] = fsFileLength; + row[5] = metadataFileLength; + if (verbose) { // if verbose print all files + rows.add(row); + } else if ((doesFsFileExists != doesMetadataFileExists) || (fsFileLength != metadataFileLength)) { // if non verbose, print only non matching files + rows.add(row); + } + } + + if (metadataStatuses.length != fsStatuses.length) { + LOG.error(" FS and metadata files count not matching for " + partition + ". FS files count " + fsStatuses.length + ", metadata base files count " + + metadataStatuses.length); + } - return out.toString(); + for (Map.Entry entry : fileStatusMap.entrySet()) { + if (!metadataFileStatusMap.containsKey(entry.getKey())) { + LOG.error("FS file not found in metadata " + entry.getKey()); + } else { + if (entry.getValue().getLen() != metadataFileStatusMap.get(entry.getKey()).getLen()) { + LOG.error(" FS file size mismatch " + entry.getKey() + ", size equality " + + (entry.getValue().getLen() == metadataFileStatusMap.get(entry.getKey()).getLen()) + + ". FS size " + entry.getValue().getLen() + ", metadata size " + + metadataFileStatusMap.get(entry.getKey()).getLen()); + } + } + } + for (Map.Entry entry : metadataFileStatusMap.entrySet()) { + if (!fileStatusMap.containsKey(entry.getKey())) { + LOG.error("Metadata file not found in FS " + entry.getKey()); + } else { + if (entry.getValue().getLen() != fileStatusMap.get(entry.getKey()).getLen()) { + LOG.error(" Metadata file size mismatch " + entry.getKey() + ", size equality " + + (entry.getValue().getLen() == fileStatusMap.get(entry.getKey()).getLen()) + + ". Metadata size " + entry.getValue().getLen() + ", FS size " + + metadataFileStatusMap.get(entry.getKey()).getLen()); + } + } + } + } + TableHeader header = new TableHeader().addTableHeaderField("Partition") + .addTableHeaderField("File Name") + .addTableHeaderField(" Is Present in FS ") + .addTableHeaderField(" Is Present in Metadata") + .addTableHeaderField(" FS size") + .addTableHeaderField(" Metadata size"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, Integer.MAX_VALUE, false, rows); } private HoodieWriteConfig getWriteConfig() { @@ -227,7 +359,7 @@ private HoodieWriteConfig getWriteConfig() { private void initJavaSparkContext() { if (jsc == null) { - jsc = SparkUtil.initJavaSparkConf("HoodieClI"); + jsc = SparkUtil.initJavaSparkConf(SparkUtil.getDefaultConf("HoodieCLI", Option.empty())); } } -} +} \ No newline at end of file diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index 127cb28ad010..2533562d8206 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -156,7 +156,7 @@ public String overwriteHoodieProperties( newProps.load(new FileInputStream(new File(overwriteFilePath))); Map oldProps = client.getTableConfig().propsMap(); Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); - HoodieTableConfig.createHoodieProperties(client.getFs(), metaPathDir, newProps); + HoodieTableConfig.create(client.getFs(), metaPathDir, newProps); TreeSet allPropKeys = new TreeSet<>(); allPropKeys.addAll(newProps.keySet().stream().map(Object::toString).collect(Collectors.toSet())); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java index 7969808e2983..a3d78d126096 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java @@ -43,6 +43,7 @@ public void setEnv(@CliOption(key = {"conf"}, help = "Env config to be set") fin throw new IllegalArgumentException("Illegal set parameter, please use like [set --conf SPARK_HOME=/usr/etc/spark]"); } env.put(map[0].trim(), map[1].trim()); + System.setProperty(map[0].trim(), map[1].trim()); } @CliCommand(value = "show envs all", help = "Show spark launcher envs") diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 82688fecc366..d1ee109f5904 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieBootstrapConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -79,12 +80,14 @@ enum SparkCommand { } public static void main(String[] args) throws Exception { - String command = args[0]; - LOG.info("Invoking SparkMain:" + command); + ValidationUtils.checkArgument(args.length >= 4); + final String commandString = args[0]; + LOG.info("Invoking SparkMain: " + commandString); + final SparkCommand cmd = SparkCommand.valueOf(commandString); - SparkCommand cmd = SparkCommand.valueOf(command); + JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + commandString, + Option.of(args[1]), Option.of(args[2])); - JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command, Option.of(args[1]), Option.of(args[2])); int returnCode = 0; try { switch (cmd) { @@ -111,8 +114,8 @@ public static void main(String[] args) throws Exception { if (args.length > 13) { configs.addAll(Arrays.asList(args).subList(13, args.length)); } - returnCode = dataLoad(jsc, command, args[3], args[4], args[5], args[6], args[7], args[8], - Integer.parseInt(args[9]), args[10], Integer.parseInt(args[11]), propsFilePath, configs); + returnCode = dataLoad(jsc, commandString, args[3], args[4], args[5], args[6], args[7], args[8], + Integer.parseInt(args[9]), args[10], Integer.parseInt(args[11]), propsFilePath, configs); break; case COMPACT_RUN: assert (args.length >= 10); @@ -159,33 +162,34 @@ public static void main(String[] args) throws Exception { case COMPACT_UNSCHEDULE_PLAN: assert (args.length == 9); doCompactUnschedule(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]), - Boolean.parseBoolean(args[7]), Boolean.parseBoolean(args[8])); + Boolean.parseBoolean(args[7]), Boolean.parseBoolean(args[8])); returnCode = 0; break; case CLUSTERING_RUN: - assert (args.length >= 8); + assert (args.length >= 9); propsFilePath = null; - if (!StringUtils.isNullOrEmpty(args[7])) { - propsFilePath = args[7]; + if (!StringUtils.isNullOrEmpty(args[8])) { + propsFilePath = args[8]; } configs = new ArrayList<>(); - if (args.length > 8) { - configs.addAll(Arrays.asList(args).subList(8, args.length)); + if (args.length > 9) { + configs.addAll(Arrays.asList(args).subList(9, args.length)); } - returnCode = cluster(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], - Integer.parseInt(args[6]), false, propsFilePath, configs); + returnCode = cluster(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]), args[2], + Integer.parseInt(args[7]), false, propsFilePath, configs); break; case CLUSTERING_SCHEDULE: - assert (args.length >= 6); + assert (args.length >= 7); propsFilePath = null; - if (!StringUtils.isNullOrEmpty(args[5])) { - propsFilePath = args[5]; + if (!StringUtils.isNullOrEmpty(args[6])) { + propsFilePath = args[6]; } configs = new ArrayList<>(); - if (args.length > 6) { - configs.addAll(Arrays.asList(args).subList(6, args.length)); + if (args.length > 7) { + configs.addAll(Arrays.asList(args).subList(7, args.length)); } - returnCode = cluster(jsc, args[1], args[2], args[3], 1, args[4], 0, true, propsFilePath, configs); + returnCode = cluster(jsc, args[3], args[4], args[5], 1, args[2], + 0, true, propsFilePath, configs); break; case CLEAN: assert (args.length >= 5); @@ -229,7 +233,7 @@ public static void main(String[] args) throws Exception { break; } } catch (Throwable throwable) { - LOG.error("Fail to execute command", throwable); + LOG.error("Fail to execute commandString", throwable); returnCode = -1; } finally { jsc.stop(); @@ -360,7 +364,7 @@ private static int doBootstrap(JavaSparkContext jsc, String tableName, String ta String payloadClassName, String enableHiveSync, String propsFilePath, List configs) throws IOException { TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs) - : UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getConfig(); + : UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(propsFilePath), configs).getProps(true); properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java index 03c76375e599..41cdfb88f844 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java @@ -20,13 +20,16 @@ import org.apache.hudi.cli.HoodieCLI; import org.apache.hudi.cli.HoodiePrintHelper; +import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.TableHeader; import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.exception.TableNotFoundException; import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.springframework.shell.core.CommandMarker; @@ -35,12 +38,21 @@ import org.springframework.stereotype.Component; import java.io.File; +import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; /** * CLI command to display hudi table options. @@ -170,6 +182,67 @@ public String fetchTableSchema( } } + @CliCommand(value = "table recover-configs", help = "Recover table configs, from update/delete that failed midway.") + public String recoverTableConfig() throws IOException { + HoodieCLI.refreshTableMetadata(); + HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); + Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); + HoodieTableConfig.recover(client.getFs(), metaPathDir); + return descTable(); + } + + @CliCommand(value = "table update-configs", help = "Update the table configs with configs with provided file.") + public String updateTableConfig( + @CliOption(key = {"props-file"}, mandatory = true, help = "Path to a properties file on local filesystem") + final String updatePropsFilePath) throws IOException { + HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); + Map oldProps = client.getTableConfig().propsMap(); + + Properties updatedProps = new Properties(); + updatedProps.load(new FileInputStream(updatePropsFilePath)); + Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); + HoodieTableConfig.update(client.getFs(), metaPathDir, updatedProps); + + HoodieCLI.refreshTableMetadata(); + Map newProps = HoodieCLI.getTableMetaClient().getTableConfig().propsMap(); + return renderOldNewProps(newProps, oldProps); + } + + @CliCommand(value = "table delete-configs", help = "Delete the supplied table configs from the table.") + public String deleteTableConfig( + @CliOption(key = {"comma-separated-configs"}, mandatory = true, help = "Comma separated list of configs to delete.") + final String csConfigs) { + HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); + Map oldProps = client.getTableConfig().propsMap(); + + Set deleteConfigs = Arrays.stream(csConfigs.split(",")).collect(Collectors.toSet()); + Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); + HoodieTableConfig.delete(client.getFs(), metaPathDir, deleteConfigs); + + HoodieCLI.refreshTableMetadata(); + Map newProps = HoodieCLI.getTableMetaClient().getTableConfig().propsMap(); + return renderOldNewProps(newProps, oldProps); + } + + private static String renderOldNewProps(Map newProps, Map oldProps) { + TreeSet allPropKeys = new TreeSet<>(); + allPropKeys.addAll(newProps.keySet().stream().map(Object::toString).collect(Collectors.toSet())); + allPropKeys.addAll(oldProps.keySet()); + + String[][] rows = new String[allPropKeys.size()][]; + int ind = 0; + for (String propKey : allPropKeys) { + String[] row = new String[]{ + propKey, + oldProps.getOrDefault(propKey, "null"), + newProps.getOrDefault(propKey, "null") + }; + rows[ind++] = row; + } + return HoodiePrintHelper.print(new String[] {HoodieTableHeaderFields.HEADER_HOODIE_PROPERTY, + HoodieTableHeaderFields.HEADER_OLD_VALUE, HoodieTableHeaderFields.HEADER_NEW_VALUE}, rows); + } + /** * Use Streams when you are dealing with raw data. * @param filePath output file path. diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java index 5f08f0097a45..a95cc53df329 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java @@ -51,7 +51,7 @@ public static long countNewRecords(HoodieTableMetaClient target, List co public static String getTimeDaysAgo(int numberOfDays) { Date date = Date.from(ZonedDateTime.now().minusDays(numberOfDays).toInstant()); - return HoodieActiveTimeline.formatInstantTime(date); + return HoodieActiveTimeline.formatDate(date); } /** @@ -61,8 +61,8 @@ public static String getTimeDaysAgo(int numberOfDays) { * b) hours: -1, returns 20200202010000 */ public static String addHours(String compactionCommitTime, int hours) throws ParseException { - Instant instant = HoodieActiveTimeline.parseInstantTime(compactionCommitTime).toInstant(); + Instant instant = HoodieActiveTimeline.parseDateFromInstantTime(compactionCommitTime).toInstant(); ZonedDateTime commitDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); - return HoodieActiveTimeline.formatInstantTime(Date.from(commitDateTime.plusHours(hours).toInstant())); + return HoodieActiveTimeline.formatDate(Date.from(commitDateTime.plusHours(hours).toInstant())); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index 7127631775de..45e048755166 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -34,13 +34,14 @@ import java.net.URISyntaxException; import java.util.Map; import java.util.Objects; +import java.util.Properties; /** * Utility functions dealing with Spark. */ public class SparkUtil { - private static final String DEFAULT_SPARK_MASTER = "yarn"; + public static final String DEFAULT_SPARK_MASTER = "yarn"; /** * TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro. @@ -62,28 +63,30 @@ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntax return sparkLauncher; } - public static JavaSparkContext initJavaSparkConf(String name) { - return initJavaSparkConf(name, Option.empty(), Option.empty()); - } - - public static JavaSparkContext initJavaSparkConf(String name, Option master, - Option executorMemory) { - SparkConf sparkConf = new SparkConf().setAppName(name); + /** + * Get the default spark configuration. + * + * @param appName - Spark application name + * @param sparkMaster - Spark master node name + * @return Spark configuration + */ + public static SparkConf getDefaultConf(final String appName, final Option sparkMaster) { + final Properties properties = System.getProperties(); + SparkConf sparkConf = new SparkConf().setAppName(appName); - String defMaster = master.orElse(sparkConf.getenv(HoodieCliSparkConfig.CLI_SPARK_MASTER)); - if ((null == defMaster) || (defMaster.isEmpty())) { - sparkConf.setMaster(DEFAULT_SPARK_MASTER); - } else { - sparkConf.setMaster(defMaster); + // Configure the sparkMaster + String sparkMasterNode = DEFAULT_SPARK_MASTER; + if (properties.getProperty(HoodieCliSparkConfig.CLI_SPARK_MASTER) != null) { + sparkMasterNode = properties.getProperty(HoodieCliSparkConfig.CLI_SPARK_MASTER); } + sparkMasterNode = sparkMaster.orElse(sparkMasterNode); + sparkConf.setMaster(sparkMasterNode); - sparkConf.set(HoodieCliSparkConfig.CLI_SERIALIZER, "org.apache.spark.serializer.KryoSerializer"); + // Configure driver sparkConf.set(HoodieCliSparkConfig.CLI_DRIVER_MAX_RESULT_SIZE, "2g"); sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_OVERWRITE, "true"); - sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_ENABLED, "true"); - if (executorMemory.isPresent()) { - sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get()); - } + sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_ENABLED, "false"); + sparkConf.set(HoodieCliSparkConfig.CLI_SERIALIZER, "org.apache.spark.serializer.KryoSerializer"); // Configure hadoop conf sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESS, "true"); @@ -91,10 +94,28 @@ public static JavaSparkContext initJavaSparkConf(String name, Option mas sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_CODEC, "org.apache.hadoop.io.compress.GzipCodec"); sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_TYPE, "BLOCK"); + return sparkConf; + } + + public static JavaSparkContext initJavaSparkConf(String name) { + return initJavaSparkConf(name, Option.empty(), Option.empty()); + } + + public static JavaSparkContext initJavaSparkConf(String name, Option master, Option executorMemory) { + SparkConf sparkConf = getDefaultConf(name, master); + if (executorMemory.isPresent()) { + sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get()); + } + + return initJavaSparkConf(sparkConf); + } + + public static JavaSparkContext initJavaSparkConf(SparkConf sparkConf) { SparkRDDWriteClient.registerClasses(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false); FSUtils.prepareHadoopConf(jsc.hadoopConfiguration()); return jsc; } + } diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala index 04c5f93e7218..e471ed925865 100644 --- a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala +++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala @@ -233,7 +233,7 @@ class DedupeSparkJob(basePath: String, } println("No duplicates found & counts are in check!!!! ") - // 4. Prepare to copy the fixed files back. + // 5. Prepare to copy the fixed files back. fileNameToPathMap.foreach { case (_, filePath) => val srcPath = new Path(s"$repairOutputPath/${filePath.getName}") val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}") diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index 9732ce72b913..091c4b4173ae 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; @@ -72,6 +73,8 @@ public void init() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index d71e7ec8d987..959346e15007 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.NumericUtils; @@ -209,6 +210,8 @@ public void testShowArchivedCommits() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index de305f404455..21841a576945 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.CompactionTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; @@ -159,6 +160,8 @@ private void generateArchive() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); // archive diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java index ceaabddec7d2..17bc48f66f0c 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java @@ -90,6 +90,7 @@ public void init() throws Exception { .withBaseFilesInPartitions(partitionAndFileId); // generate two rollback HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withRollbackUsingMarkers(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); try (AbstractHoodieWriteClient client = new SparkRDDWriteClient(context(), config)) { diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java index a39329e018bf..cba6d901b956 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java @@ -28,8 +28,10 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -62,6 +64,8 @@ public void init() throws Exception { new TableCommand().createTable( tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + timelineService = HoodieClientTestUtils.initTimelineService( + context, basePath(), FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue()); metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()); //Create some commits files and base files HoodieTestTable.of(metaClient) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java b/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java index b2b9f86b0207..7a12a6692a2b 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/functional/CLIFunctionalTestHarness.java @@ -22,7 +22,10 @@ import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.providers.SparkProvider; +import org.apache.hudi.timeline.service.TimelineService; import org.apache.hadoop.conf.Configuration; import org.apache.spark.SparkConf; @@ -39,10 +42,13 @@ public class CLIFunctionalTestHarness implements SparkProvider { + protected static int timelineServicePort = + FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue(); + protected static transient TimelineService timelineService; + protected static transient HoodieSparkEngineContext context; private static transient SparkSession spark; private static transient SQLContext sqlContext; private static transient JavaSparkContext jsc; - private static transient HoodieSparkEngineContext context; private static transient JLineShellComponent shell; /** * An indicator of the initialization status. @@ -107,6 +113,9 @@ public synchronized void runBeforeEach() { jsc = new JavaSparkContext(spark.sparkContext()); context = new HoodieSparkEngineContext(jsc); shell = new Bootstrap().getJLineShellComponent(); + timelineService = HoodieClientTestUtils.initTimelineService( + context, basePath(), incrementTimelineServicePortToUse()); + timelineServicePort = timelineService.getServerPort(); } } @@ -120,14 +129,25 @@ public static synchronized void cleanUpAfterAll() { shell.stop(); shell = null; } + if (timelineService != null) { + timelineService.close(); + } } /** * Helper to prepare string for matching. + * * @param str Input string. * @return pruned string with non word characters removed. */ protected static String removeNonWordAndStripSpace(String str) { return str.replaceAll("[\\s]+", ",").replaceAll("[\\W]+", ","); } + + protected int incrementTimelineServicePortToUse() { + // Increment the timeline service port for each individual test + // to avoid port reuse causing failures + timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024; + return timelineServicePort; + } } diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index c67621fbb9a3..a9209f5534df 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-client-common - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-client-common jar @@ -36,6 +36,11 @@ hudi-common ${project.version} + + org.apache.hudi + hudi-aws + ${project.version} + org.apache.hudi hudi-timeline-service @@ -59,6 +64,13 @@ parquet-avro + + + com.github.davidmoten + hilbert-curve + 0.2.2 + + io.dropwizard.metrics @@ -94,7 +106,6 @@ io.prometheus simpleclient_pushgateway - org.apache.hudi diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java index 85e008199adc..f57484d886c9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java @@ -29,7 +29,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; @@ -130,7 +129,7 @@ public void start(Function onShutdownCallback) { future = res.getKey(); executor = res.getValue(); started = true; - monitorThreads(onShutdownCallback); + shutdownCallback(onShutdownCallback); } /** @@ -141,34 +140,15 @@ public void start(Function onShutdownCallback) { protected abstract Pair startService(); /** - * A monitor thread is started which would trigger a callback if the service is shutdown. + * Add shutdown callback for the completable future. * - * @param onShutdownCallback + * @param callback The callback */ - private void monitorThreads(Function onShutdownCallback) { - LOG.info("Submitting monitor thread !!"); - Executors.newSingleThreadExecutor(r -> { - Thread t = new Thread(r, "Monitor Thread"); - t.setDaemon(isRunInDaemonMode()); - return t; - }).submit(() -> { - boolean error = false; - try { - LOG.info("Monitoring thread(s) !!"); - future.get(); - } catch (ExecutionException ex) { - LOG.error("Monitor noticed one or more threads failed. Requesting graceful shutdown of other threads", ex); - error = true; - } catch (InterruptedException ie) { - LOG.error("Got interrupted Monitoring threads", ie); - error = true; - } finally { - // Mark as shutdown - shutdown = true; - if (null != onShutdownCallback) { - onShutdownCallback.apply(error); - } - shutdown(false); + @SuppressWarnings("unchecked") + private void shutdownCallback(Function callback) { + future.whenComplete((resp, error) -> { + if (null != callback) { + callback.apply(null != error); } }); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 444eae62b2ec..76b10fddd438 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -32,6 +32,7 @@ import org.apache.hudi.client.heartbeat.HeartbeatUtils; import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -67,6 +68,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTimelineArchiveLog; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.rollback.RollbackUtils; import org.apache.hudi.table.action.savepoint.SavepointHelpers; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -80,6 +82,8 @@ import java.text.ParseException; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -197,7 +201,7 @@ public boolean commitStats(String instantTime, List stats, Opti } catch (IOException e) { throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e); } finally { - this.txnManager.endTransaction(); + this.txnManager.endTransaction(Option.of(inflightInstant)); } // do this outside of lock since compaction, clustering can be time taking and we don't need a lock for the entire execution period runTableServicesInline(table, metadata, extraMetadata); @@ -218,6 +222,8 @@ protected void commit(HoodieTable table, String commitActionType, String instant HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); // Finalize write finalizeWrite(table, instantTime, stats); + // update Metadata table + writeTableMetadata(table, instantTime, commitActionType, metadata); activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } @@ -233,7 +239,7 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String if (writeTimer != null) { long durationInMs = metrics.getDurationInMs(writeTimer.stop()); - metrics.updateCommitMetrics(HoodieActiveTimeline.parseInstantTime(instantTime).getTime(), durationInMs, + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(), durationInMs, metadata, actionType); writeTimer = null; } @@ -244,16 +250,25 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String } /** - * Any pre-commit actions like conflict resolution or updating metadata table goes here. + * Any pre-commit actions like conflict resolution goes here. * @param inflightInstant instant of inflight operation. * @param metadata commit metadata for which pre commit is being invoked. */ protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) { - // Create a Hoodie table after starting the transaction which encapsulated the commits and files visible. - // Important to create this after the lock to ensure latest commits show up in the timeline without need for reload - HoodieTable table = createTable(config, hadoopConf); - table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, inflightInstant.getTimestamp(), - table.isTableServiceAction(inflightInstant.getAction()))); + // To be overridden by specific engines to perform conflict resolution if any. + } + + /** + * Write the HoodieCommitMetadata to metadata table if available. + * @param table {@link HoodieTable} of interest. + * @param instantTime instant time of the commit. + * @param actionType action type of the commit. + * @param metadata instance of {@link HoodieCommitMetadata}. + */ + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { + context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table"); + table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime, + table.isTableServiceAction(actionType))); } /** @@ -410,7 +425,11 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); + if (null == this.asyncCleanerService) { + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); + } else { + this.asyncCleanerService.start(null); + } } /** @@ -436,11 +455,9 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); autoCleanOnCommit(); - // We cannot have unbounded commit files. Archive commits if we have to archive - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); - archiveLog.archiveIfRequired(context); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); + if (config.isAutoArchive()) { + archive(table); + } } finally { this.heartbeatClient.stop(instantTime); } @@ -508,6 +525,13 @@ protected void autoCleanOnCommit() { } } + /** + * Run any pending compactions. + */ + public void runAnyPendingCompactions() { + runAnyPendingCompactions(createTable(config, hadoopConf, config.isMetadataTableEnabled())); + } + /** * Create a savepoint based on the latest commit action on the timeline. * @@ -574,22 +598,39 @@ public void restoreToSavepoint(String savepointTime) { @Deprecated public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { - return rollback(commitInstantTime, false); + HoodieTable table = createTable(config, hadoopConf); + Option pendingRollbackInfo = getPendingRollbackInfo(table.getMetaClient(), commitInstantTime); + return rollback(commitInstantTime, pendingRollbackInfo, false); } /** * @Deprecated * Rollback the inflight record changes with the given commit time. This * will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)} - * + * Adding this api for backwards compatability. * @param commitInstantTime Instant time of the commit * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. * @throws HoodieRollbackException if rollback cannot be performed successfully */ @Deprecated public boolean rollback(final String commitInstantTime, boolean skipLocking) throws HoodieRollbackException { + return rollback(commitInstantTime, Option.empty(), skipLocking); + } + + /** + * @Deprecated + * Rollback the inflight record changes with the given commit time. This + * will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)} + * + * @param commitInstantTime Instant time of the commit + * @param pendingRollbackInfo pending rollback instant and plan if rollback failed from previous attempt. + * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. + * @throws HoodieRollbackException if rollback cannot be performed successfully + */ + @Deprecated + public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException { LOG.info("Begin rollback of instant " + commitInstantTime); - final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime(); + final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); final Timer.Context timerContext = this.metrics.getRollbackCtx(); try { HoodieTable table = createTable(config, hadoopConf); @@ -598,8 +639,8 @@ public boolean rollback(final String commitInstantTime, boolean skipLocking) thr .findFirst()); if (commitInstantOpt.isPresent()) { LOG.info("Scheduling Rollback at instant time :" + rollbackInstantTime); - Option rollbackPlanOption = table.scheduleRollback(context, rollbackInstantTime, - commitInstantOpt.get(), false); + Option rollbackPlanOption = pendingRollbackInfo.map(entry -> Option.of(entry.getRollbackPlan())).orElse(table.scheduleRollback(context, rollbackInstantTime, + commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers())); if (rollbackPlanOption.isPresent()) { // execute rollback HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true, @@ -713,6 +754,31 @@ public HoodieCleanMetadata clean(boolean skipLocking) { return clean(HoodieActiveTimeline.createNewInstantTime(), skipLocking); } + /** + * Trigger archival for the table. This ensures that the number of commits do not explode + * and keep increasing unbounded over time. + * @param table table to commit on. + */ + protected void archive(HoodieTable table) { + try { + // We cannot have unbounded commit files. Archive commits if we have to archive + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); + archiveLog.archiveIfRequired(context); + } catch (IOException ioe) { + throw new HoodieIOException("Failed to archive", ioe); + } + } + + /** + * Trigger archival for the table. This ensures that the number of commits do not explode + * and keep increasing unbounded over time. + */ + public void archive() { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = createTable(config, hadoopConf); + archive(table); + } + /** * Provides a new commit time for a write operation (insert/update/delete). */ @@ -828,6 +894,33 @@ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieT return inflightTimelineExcludeClusteringCommit; } + private Option getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) { + return getPendingRollbackInfos(metaClient).getOrDefault(commitToRollback, Option.empty()); + } + + /** + * Fetch map of pending commits to be rolled-back to {@link HoodiePendingRollbackInfo}. + * @param metaClient instance of {@link HoodieTableMetaClient} to use. + * @return map of pending commits to be rolled-back instants to Rollback Instant and Rollback plan Pair. + */ + protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient) { + List instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().collect(Collectors.toList()); + Map> infoMap = new HashMap<>(); + HoodieTimeline pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline(); + for (HoodieInstant instant : instants) { + try { + HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, instant); + String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime(); + if (!pendingCompactionTimeline.containsInstant(instantToRollback)) { + infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan))); + } + } catch (IOException e) { + LOG.warn("Fetching rollback plan failed for " + infoMap + ", skip the plan", e); + } + } + return infoMap; + } + /** * Rollback all failed writes. */ @@ -841,29 +934,30 @@ public Boolean rollbackFailedWrites() { */ public Boolean rollbackFailedWrites(boolean skipLocking) { HoodieTable table = createTable(config, hadoopConf); - List instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), - Option.empty()); - rollbackFailedWrites(instantsToRollback, skipLocking); + List instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty()); + Map> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient()); + instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); + rollbackFailedWrites(pendingRollbacks, skipLocking); return true; } - protected void rollbackFailedWrites(List instantsToRollback, boolean skipLocking) { - for (String instant : instantsToRollback) { - if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, + protected void rollbackFailedWrites(Map> instantsToRollback, boolean skipLocking) { + // sort in reverse order of commit times + LinkedHashMap> reverseSortedRollbackInstants = instantsToRollback.entrySet() + .stream().sorted((i1, i2) -> i2.getKey().compareTo(i1.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e1, LinkedHashMap::new)); + for (Map.Entry> entry : reverseSortedRollbackInstants.entrySet()) { + if (HoodieTimeline.compareTimestamps(entry.getKey(), HoodieTimeline.LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { + // do we need to handle failed rollback of a bootstrap rollbackFailedBootstrap(); + HeartbeatUtils.deleteHeartbeatFile(fs, basePath, entry.getKey(), config); break; } else { - rollback(instant, skipLocking); + rollback(entry.getKey(), entry.getValue(), skipLocking); + HeartbeatUtils.deleteHeartbeatFile(fs, basePath, entry.getKey(), config); } } - // Delete any heartbeat files for already rolled back commits - try { - HeartbeatUtils.cleanExpiredHeartbeats(this.heartbeatClient.getAllExistingHeartbeatInstants(), - createMetaClient(true), basePath); - } catch (IOException io) { - LOG.error("Unable to delete heartbeat files", io); - } } protected List getInstantsToRollback(HoodieTableMetaClient metaClient, HoodieFailedWritesCleaningPolicy cleaningPolicy, Option curInstantTime) { @@ -977,13 +1071,14 @@ public Option scheduleTableService(Option> extraMeta public Option scheduleTableService(String instantTime, Option> extraMetadata, TableServiceType tableServiceType) { // A lock is required to guard against race conditions between an on-going writer and scheduling a table service. + final Option inflightInstant = Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, + tableServiceType.getAction(), instantTime)); try { - this.txnManager.beginTransaction(Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, - tableServiceType.getAction(), instantTime)), Option.empty()); + this.txnManager.beginTransaction(inflightInstant, Option.empty()); LOG.info("Scheduling table service " + tableServiceType); return scheduleTableServiceInternal(instantTime, extraMetadata, tableServiceType); } finally { - this.txnManager.endTransaction(); + this.txnManager.endTransaction(inflightInstant); } } @@ -1024,7 +1119,7 @@ protected Option inlineCluster(Option> extraMetadata protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { String commitTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, commitTime, inflightInstant, false); + table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); table.rollback(context, commitTime, inflightInstant, false, false); table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java index 2fd4251d2f9f..a5a38f2cc594 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java @@ -37,28 +37,26 @@ class AsyncCleanerService extends HoodieAsyncService { private static final Logger LOG = LogManager.getLogger(AsyncCleanerService.class); private final AbstractHoodieWriteClient writeClient; - private final String cleanInstantTime; private final transient ExecutorService executor = Executors.newSingleThreadExecutor(); - protected AsyncCleanerService(AbstractHoodieWriteClient writeClient, String cleanInstantTime) { + protected AsyncCleanerService(AbstractHoodieWriteClient writeClient) { this.writeClient = writeClient; - this.cleanInstantTime = cleanInstantTime; } @Override protected Pair startService() { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + LOG.info("Auto cleaning is enabled. Running cleaner async to write operation at instant time " + instantTime); return Pair.of(CompletableFuture.supplyAsync(() -> { - writeClient.clean(cleanInstantTime); + writeClient.clean(instantTime); return true; - }), executor); + }, executor), executor); } public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient) { AsyncCleanerService asyncCleanerService = null; if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) { - String instantTime = HoodieActiveTimeline.createNewInstantTime(); - LOG.info("Auto cleaning is enabled. Running cleaner async to write operation at instant time " + instantTime); - asyncCleanerService = new AsyncCleanerService(writeClient, instantTime); + asyncCleanerService = new AsyncCleanerService(writeClient); asyncCleanerService.start(null); } else { LOG.info("Async auto cleaning is not enabled. Not running cleaner now"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java index 80191d4c3cdc..a20469429030 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -30,9 +29,6 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; /** * Helper class to delete heartbeat for completed or failed instants with expired heartbeats. @@ -55,6 +51,8 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String deleted = fs.delete(new Path(heartbeatFolderPath + Path.SEPARATOR + instantTime), false); if (!deleted) { LOG.error("Failed to delete heartbeat for instant " + instantTime); + } else { + LOG.info("Deleted the heartbeat for instant " + instantTime); } } catch (IOException io) { LOG.error("Unable to delete heartbeat for instant " + instantTime, io); @@ -63,20 +61,19 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String } /** - * Deletes the heartbeat files for instants with expired heartbeats without any active instant. - * @param allExistingHeartbeatInstants - * @param metaClient - * @param basePath + * Deletes the heartbeat file for the specified instant. + * @param fs Hadoop FileSystem instance + * @param basePath Hoodie table base path + * @param instantTime Commit instant time + * @param config HoodieWriteConfig instance + * @return Boolean indicating whether heartbeat file was deleted or not */ - public static void cleanExpiredHeartbeats(List allExistingHeartbeatInstants, - HoodieTableMetaClient metaClient, String basePath) { - Set nonExpiredHeartbeatInstants = metaClient.getActiveTimeline() - .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - allExistingHeartbeatInstants.stream().forEach(instant -> { - if (!nonExpiredHeartbeatInstants.contains(instant)) { - deleteHeartbeatFile(metaClient.getFs(), basePath, instant); - } - }); + public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String instantTime, HoodieWriteConfig config) { + if (config.getFailedWritesCleanPolicy().isLazy()) { + return deleteHeartbeatFile(fs, basePath, instantTime); + } + + return false; } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java index a6753aaa3a2e..d9b9d3d269bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java @@ -35,49 +35,64 @@ public class TransactionManager implements Serializable { private static final Logger LOG = LogManager.getLogger(TransactionManager.class); - private final LockManager lockManager; - private Option currentTxnOwnerInstant; - private Option lastCompletedTxnOwnerInstant; - private boolean supportsOptimisticConcurrency; + private final boolean isOptimisticConcurrencyControlEnabled; + private Option currentTxnOwnerInstant = Option.empty(); + private Option lastCompletedTxnOwnerInstant = Option.empty(); public TransactionManager(HoodieWriteConfig config, FileSystem fs) { this.lockManager = new LockManager(config, fs); - this.supportsOptimisticConcurrency = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + this.isOptimisticConcurrencyControlEnabled = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); } - public synchronized void beginTransaction() { - if (supportsOptimisticConcurrency) { + public void beginTransaction() { + if (isOptimisticConcurrencyControlEnabled) { LOG.info("Transaction starting without a transaction owner"); lockManager.lock(); - LOG.info("Transaction started"); + LOG.info("Transaction started without a transaction owner"); } } - public synchronized void beginTransaction(Option currentTxnOwnerInstant, Option lastCompletedTxnOwnerInstant) { - if (supportsOptimisticConcurrency) { - this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant; - lockManager.setLatestCompletedWriteInstant(lastCompletedTxnOwnerInstant); - LOG.info("Latest completed transaction instant " + lastCompletedTxnOwnerInstant); - this.currentTxnOwnerInstant = currentTxnOwnerInstant; - LOG.info("Transaction starting with transaction owner " + currentTxnOwnerInstant); + public void beginTransaction(Option newTxnOwnerInstant, + Option lastCompletedTxnOwnerInstant) { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction starting for " + newTxnOwnerInstant + + " with latest completed transaction instant " + lastCompletedTxnOwnerInstant); lockManager.lock(); - LOG.info("Transaction started"); + reset(currentTxnOwnerInstant, newTxnOwnerInstant, lastCompletedTxnOwnerInstant); + LOG.info("Transaction started for " + newTxnOwnerInstant + + " with latest completed transaction instant " + lastCompletedTxnOwnerInstant); + } + } + + public void endTransaction() { + if (isOptimisticConcurrencyControlEnabled) { + LOG.info("Transaction ending without a transaction owner"); + lockManager.unlock(); + LOG.info("Transaction ended without a transaction owner"); } } - public synchronized void endTransaction() { - if (supportsOptimisticConcurrency) { + public void endTransaction(Option currentTxnOwnerInstant) { + if (isOptimisticConcurrencyControlEnabled) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); + reset(currentTxnOwnerInstant, Option.empty(), Option.empty()); lockManager.unlock(); - LOG.info("Transaction ended"); - this.lastCompletedTxnOwnerInstant = Option.empty(); - lockManager.resetLatestCompletedWriteInstant(); + LOG.info("Transaction ended with transaction owner " + currentTxnOwnerInstant); + } + } + + private synchronized void reset(Option callerInstant, + Option newTxnOwnerInstant, + Option lastCompletedTxnOwnerInstant) { + if (!this.currentTxnOwnerInstant.isPresent() || this.currentTxnOwnerInstant.get().equals(callerInstant.get())) { + this.currentTxnOwnerInstant = newTxnOwnerInstant; + this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant; } } public void close() { - if (supportsOptimisticConcurrency) { + if (isOptimisticConcurrencyControlEnabled) { lockManager.close(); LOG.info("Transaction manager closed"); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java new file mode 100644 index 000000000000..cab9d95df4be --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java @@ -0,0 +1,117 @@ +/* + * 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 org.apache.hudi.client.transaction.lock; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.jetbrains.annotations.NotNull; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * InProcess level lock. This {@link LockProvider} implementation is to + * guard table from concurrent operations happening in the local JVM process. + *

+ * Note: This Lock provider implementation doesn't allow lock reentrancy. + * Attempting to reacquire the lock from the same thread will throw + * HoodieLockException. Threads other than the current lock owner, will + * block on lock() and return false on tryLock(). + */ +public class InProcessLockProvider implements LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + private static final ReentrantReadWriteLock LOCK = new ReentrantReadWriteLock(); + private final long maxWaitTimeMillis; + + public InProcessLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + TypedProperties typedProperties = lockConfiguration.getConfig(); + maxWaitTimeMillis = typedProperties.getLong(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, + LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS); + } + + @Override + public void lock() { + LOG.info(getLogMessage(LockState.ACQUIRING)); + if (LOCK.isWriteLockedByCurrentThread()) { + throw new HoodieLockException(getLogMessage(LockState.ALREADY_ACQUIRED)); + } + LOCK.writeLock().lock(); + LOG.info(getLogMessage(LockState.ACQUIRED)); + } + + @Override + public boolean tryLock() { + return tryLock(maxWaitTimeMillis, TimeUnit.MILLISECONDS); + } + + @Override + public boolean tryLock(long time, @NotNull TimeUnit unit) { + LOG.info(getLogMessage(LockState.ACQUIRING)); + if (LOCK.isWriteLockedByCurrentThread()) { + throw new HoodieLockException(getLogMessage(LockState.ALREADY_ACQUIRED)); + } + + boolean isLockAcquired; + try { + isLockAcquired = LOCK.writeLock().tryLock(time, unit); + } catch (InterruptedException e) { + throw new HoodieLockException(getLogMessage(LockState.FAILED_TO_ACQUIRE)); + } + + LOG.info(getLogMessage(isLockAcquired ? LockState.ACQUIRED : LockState.FAILED_TO_ACQUIRE)); + return isLockAcquired; + } + + @Override + public void unlock() { + LOG.info(getLogMessage(LockState.RELEASING)); + try { + LOCK.writeLock().unlock(); + } catch (Exception e) { + throw new HoodieLockException(getLogMessage(LockState.FAILED_TO_RELEASE), e); + } + LOG.info(getLogMessage(LockState.RELEASED)); + } + + @Override + public ReentrantReadWriteLock getLock() { + return LOCK; + } + + @Override + public void close() { + if (LOCK.isWriteLockedByCurrentThread()) { + LOCK.writeLock().unlock(); + } + } + + private String getLogMessage(LockState state) { + return StringUtils.join("Thread ", String.valueOf(Thread.currentThread().getName()), " ", + state.name(), " in-process lock."); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java index 33519b9adc0d..976205f3592e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java @@ -20,13 +20,10 @@ import java.io.Serializable; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.lock.LockProvider; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieLockException; @@ -46,11 +43,8 @@ public class LockManager implements Serializable, AutoCloseable { private final LockConfiguration lockConfiguration; private final SerializableConfiguration hadoopConf; private volatile LockProvider lockProvider; - // Holds the latest completed write instant to know which ones to check conflict against - private final AtomicReference> latestCompletedWriteInstant; public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) { - this.latestCompletedWriteInstant = new AtomicReference<>(Option.empty()); this.writeConfig = writeConfig; this.hadoopConf = new SerializableConfiguration(fs.getConf()); this.lockConfiguration = new LockConfiguration(writeConfig.getProps()); @@ -72,7 +66,7 @@ public void lock() { LOG.info("Retrying to acquire lock..."); Thread.sleep(waitTimeInMs); retryCount++; - } catch (InterruptedException e) { + } catch (HoodieLockException | InterruptedException e) { if (retryCount >= retries) { throw new HoodieLockException("Unable to acquire lock, lock object ", e); } @@ -100,22 +94,6 @@ public synchronized LockProvider getLockProvider() { return lockProvider; } - public void setLatestCompletedWriteInstant(Option instant) { - this.latestCompletedWriteInstant.set(instant); - } - - public void compareAndSetLatestCompletedWriteInstant(Option expected, Option newValue) { - this.latestCompletedWriteInstant.compareAndSet(expected, newValue); - } - - public AtomicReference> getLatestCompletedWriteInstant() { - return latestCompletedWriteInstant; - } - - public void resetLatestCompletedWriteInstant() { - this.latestCompletedWriteInstant.set(Option.empty()); - } - @Override public void close() { closeQuietly(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java index 8e59478da05f..ed2ea457764f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -18,11 +18,8 @@ package org.apache.hudi.client.utils; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.client.transaction.ConcurrentOperation; +import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -34,9 +31,12 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.HoodieTable; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; + import java.io.IOException; +import java.util.Map; import java.util.stream.Stream; public class TransactionUtils { @@ -45,6 +45,7 @@ public class TransactionUtils { /** * Resolve any write conflicts when committing data. + * * @param table * @param currentTxnOwnerInstant * @param thisCommitMetadata @@ -54,11 +55,11 @@ public class TransactionUtils { * @throws HoodieWriteConflictException */ public static Option resolveWriteConflictIfAny( - final HoodieTable table, - final Option currentTxnOwnerInstant, - final Option thisCommitMetadata, - final HoodieWriteConfig config, - Option lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException { + final HoodieTable table, + final Option currentTxnOwnerInstant, + final Option thisCommitMetadata, + final HoodieWriteConfig config, + Option lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException { if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy(); Stream instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant); @@ -68,7 +69,7 @@ public static Option resolveWriteConflictIfAny( ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient()); if (resolutionStrategy.hasConflict(thisOperation, otherOperation)) { LOG.info("Conflict encountered between current instant = " + thisOperation + " and instant = " - + otherOperation + ", attempting to resolve it..."); + + otherOperation + ", attempting to resolve it..."); resolutionStrategy.resolveConflict(table, thisOperation, otherOperation); } } catch (IOException io) { @@ -77,10 +78,6 @@ public static Option resolveWriteConflictIfAny( }); LOG.info("Successfully resolved conflicts, if any"); - if (config.mergeDeltastreamerStateFromPreviousCommit()) { - mergeCheckpointStateFromPreviousCommit(table.getMetaClient(), thisOperation.getCommitMetadataOption()); - } - return thisOperation.getCommitMetadataOption(); } return thisCommitMetadata; @@ -88,6 +85,7 @@ public static Option resolveWriteConflictIfAny( /** * Get the last completed transaction hoodie instant and {@link HoodieCommitMetadata#getExtraMetadata()}. + * * @param metaClient * @return */ @@ -117,30 +115,4 @@ public static Option>> getLastCompletedT throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io); } } - - protected static void mergeCheckpointStateFromPreviousCommit(HoodieTableMetaClient metaClient, Option thisMetadata) { - overrideWithLatestCommitMetadata(metaClient, thisMetadata, Collections.singletonList(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY)); - } - - /** - * Generic method allowing us to override the current metadata with the metadata from - * the latest instant for the specified key prefixes. - * @param metaClient - * @param thisMetadata - * @param keyPrefixes The key prefixes to merge from the previous commit - */ - private static void overrideWithLatestCommitMetadata(HoodieTableMetaClient metaClient, - Option thisMetadata, - List keyPrefixes) { - if (keyPrefixes.size() == 1 && keyPrefixes.get(0).length() < 1) { - return; - } - Option>> lastInstant = getLastCompletedTxnInstantAndMetadata(metaClient); - if (lastInstant.isPresent() && thisMetadata.isPresent()) { - Stream lastCommitMetadataKeys = lastInstant.get().getRight().keySet().stream(); - keyPrefixes.stream().forEach(keyPrefix -> lastCommitMetadataKeys - .filter(key -> key.startsWith(keyPrefix)) - .forEach(key -> thisMetadata.get().getExtraMetadata().put(key, lastInstant.get().getRight().get(key)))); - } - } } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index 73c66d8038e5..9486ad43de60 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -22,12 +22,18 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.util.TypeUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; +import javax.annotation.Nonnull; import java.io.File; import java.io.FileReader; import java.io.IOException; -import java.util.Locale; +import java.util.Map; import java.util.Properties; /** @@ -41,6 +47,16 @@ public class HoodieClusteringConfig extends HoodieConfig { // Any strategy specific params can be saved with this prefix public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; + public static final String SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy"; + public static final String JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy"; + public static final String SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY = + "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy"; + public static final String JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY = + "org.apache.hudi.client.clustering.run.strategy.JavaSortAndSizeExecutionStrategy"; + public static final String PLAN_PARTITION_FILTER_MODE = + "hoodie.clustering.plan.partition.filter.mode"; // Any Space-filling curves optimize(z-order/hilbert) params can be saved with this prefix public static final String LAYOUT_OPTIMIZE_PARAM_PREFIX = "hoodie.layout.optimize."; @@ -51,6 +67,20 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Number of partitions to list to create ClusteringPlan"); + public static final ConfigProperty PARTITION_FILTER_BEGIN_PARTITION = ConfigProperty + .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.begin.partition") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("Begin partition used to filter partition (inclusive), only effective when the filter mode '" + + PLAN_PARTITION_FILTER_MODE + "' is " + ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); + + public static final ConfigProperty PARTITION_FILTER_END_PARTITION = ConfigProperty + .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.end.partition") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("End partition used to filter partition (inclusive), only effective when the filter mode '" + + PLAN_PARTITION_FILTER_MODE + "' is " + ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); + public static final ConfigProperty PLAN_STRATEGY_SMALL_FILE_LIMIT = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit") .defaultValue(String.valueOf(600 * 1024 * 1024L)) @@ -59,7 +89,7 @@ public class HoodieClusteringConfig extends HoodieConfig { public static final ConfigProperty PLAN_STRATEGY_CLASS_NAME = ConfigProperty .key("hoodie.clustering.plan.strategy.class") - .defaultValue("org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy") + .defaultValue(SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY) .sinceVersion("0.7.0") .withDocumentation("Config to provide a strategy class (subclass of ClusteringPlanStrategy) to create clustering plan " + "i.e select what file groups are being clustered. Default strategy, looks at the clustering small file size limit (determined by " @@ -67,7 +97,7 @@ public class HoodieClusteringConfig extends HoodieConfig { public static final ConfigProperty EXECUTION_STRATEGY_CLASS_NAME = ConfigProperty .key("hoodie.clustering.execution.strategy.class") - .defaultValue("org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy") + .defaultValue(SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY) .sinceVersion("0.7.0") .withDocumentation("Config to provide a strategy class (subclass of RunClusteringStrategy) to define how the " + " clustering plan is executed. By default, we sort the file groups in th plan by the specified columns, while " @@ -97,6 +127,17 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.9.0") .withDocumentation("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan"); + public static final ConfigProperty PLAN_PARTITION_FILTER_MODE_NAME = ConfigProperty + .key(PLAN_PARTITION_FILTER_MODE) + .defaultValue(ClusteringPlanPartitionFilterMode.NONE) + .sinceVersion("0.11.0") + .withDocumentation("Partition filter mode used in the creation of clustering plan. Available values are - " + + "NONE: do not filter table partition and thus the clustering plan will include all partitions that have clustering candidate." + + "RECENT_DAYS: keep a continuous range of partitions, worked together with configs '" + DAYBASED_LOOKBACK_PARTITIONS.key() + "' and '" + + PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST.key() + "." + + "SELECTED_PARTITIONS: keep partitions that are in the specified range ['" + PARTITION_FILTER_BEGIN_PARTITION.key() + "', '" + + PARTITION_FILTER_END_PARTITION.key() + "']."); + public static final ConfigProperty PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group") .defaultValue(String.valueOf(2 * 1024 * 1024 * 1024L)) @@ -142,6 +183,17 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.9.0") .withDocumentation("When rewriting data, preserves existing hoodie_commit_time"); + /** + * Using space-filling curves to optimize the layout of table to boost query performance. + * The table data which sorted by space-filling curve has better aggregation; + * combine with min-max filtering, it can achieve good performance improvement. + * + * Notice: + * when we use this feature, we need specify the sort columns. + * The more columns involved in sorting, the worse the aggregation, and the smaller the query performance improvement. + * Choose the filter columns which commonly used in query sql as sort columns. + * It is recommend that 2 ~ 4 columns participate in sorting. + */ public static final ConfigProperty LAYOUT_OPTIMIZE_ENABLE = ConfigProperty .key(LAYOUT_OPTIMIZE_PARAM_PREFIX + "enable") .defaultValue(false) @@ -191,6 +243,15 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.10.0") .withDocumentation("Enable data skipping by collecting statistics once layout optimization is complete."); + public static final ConfigProperty ROLLBACK_PENDING_CLUSTERING_ON_CONFLICT = ConfigProperty + .key("hoodie.clustering.rollback.pending.replacecommit.on.conflict") + .defaultValue(false) + .sinceVersion("0.10.0") + .withDocumentation("If updates are allowed to file groups pending clustering, then set this config to rollback failed or pending clustering instants. " + + "Pending clustering will be rolled back ONLY IF there is conflict between incoming upsert and filegroup to be clustered. " + + "Please exercise caution while setting this config, especially when clustering is done very frequently. This could lead to race condition in " + + "rare scenarios, for example, when the clustering completes after instants are fetched but before rollback completed."); + /** * @deprecated Use {@link #PLAN_STRATEGY_CLASS_NAME} and its methods instead */ @@ -304,11 +365,24 @@ public class HoodieClusteringConfig extends HoodieConfig { /** @deprecated Use {@link #ASYNC_CLUSTERING_ENABLE} and its methods instead */ @Deprecated public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = ASYNC_CLUSTERING_ENABLE.defaultValue(); - + + // NOTE: This ctor is required for appropriate deserialization public HoodieClusteringConfig() { super(); } + public boolean isAsyncClusteringEnabled() { + return getBooleanOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE); + } + + public boolean isInlineClusteringEnabled() { + return getBooleanOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING); + } + + public static HoodieClusteringConfig from(TypedProperties props) { + return HoodieClusteringConfig.newBuilder().fromProperties(props).build(); + } + public static Builder newBuilder() { return new Builder(); } @@ -316,6 +390,12 @@ public static Builder newBuilder() { public static class Builder { private final HoodieClusteringConfig clusteringConfig = new HoodieClusteringConfig(); + private EngineType engineType = EngineType.SPARK; + + public Builder withEngineType(EngineType engineType) { + this.engineType = engineType; + return this; + } public Builder fromFile(File propertiesFile) throws IOException { try (FileReader reader = new FileReader(propertiesFile)) { @@ -329,6 +409,11 @@ public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) { return this; } + public Builder withClusteringPlanPartitionFilterMode(ClusteringPlanPartitionFilterMode mode) { + clusteringConfig.setValue(PLAN_PARTITION_FILTER_MODE_NAME.key(), mode.toString()); + return this; + } + public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) { clusteringConfig.setValue(EXECUTION_STRATEGY_CLASS_NAME, runClusteringStrategyClass); return this; @@ -344,6 +429,16 @@ public Builder withClusteringSkipPartitionsFromLatest(int clusteringSkipPartitio return this; } + public Builder withClusteringPartitionFilterBeginPartition(String begin) { + clusteringConfig.setValue(PARTITION_FILTER_BEGIN_PARTITION, begin); + return this; + } + + public Builder withClusteringPartitionFilterEndPartition(String end) { + clusteringConfig.setValue(PARTITION_FILTER_END_PARTITION, end); + return this; + } + public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) { clusteringConfig.setValue(PLAN_STRATEGY_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit)); return this; @@ -385,6 +480,7 @@ public Builder withAsyncClusteringMaxCommits(int numCommits) { } public Builder fromProperties(Properties props) { + // TODO this should cherry-pick only clustering properties this.clusteringConfig.getProps().putAll(props); return this; } @@ -404,6 +500,11 @@ public Builder withPreserveHoodieCommitMetadata(Boolean preserveHoodieCommitMeta return this; } + public Builder withRollbackPendingClustering(Boolean rollbackPendingClustering) { + clusteringConfig.setValue(ROLLBACK_PENDING_CLUSTERING_ON_CONFLICT, String.valueOf(rollbackPendingClustering)); + return this; + } + public Builder withSpaceFillingCurveDataOptimizeEnable(Boolean enable) { clusteringConfig.setValue(LAYOUT_OPTIMIZE_ENABLE, String.valueOf(enable)); return this; @@ -430,17 +531,49 @@ public Builder withDataOptimizeDataSkippingEnable(boolean dataSkipping) { } public HoodieClusteringConfig build() { + clusteringConfig.setDefaultValue( + PLAN_STRATEGY_CLASS_NAME, getDefaultPlanStrategyClassName(engineType)); + clusteringConfig.setDefaultValue( + EXECUTION_STRATEGY_CLASS_NAME, getDefaultExecutionStrategyClassName(engineType)); clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName()); return clusteringConfig; } + + private String getDefaultPlanStrategyClassName(EngineType engineType) { + switch (engineType) { + case SPARK: + return SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; + case FLINK: + case JAVA: + return JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } + + private String getDefaultExecutionStrategyClassName(EngineType engineType) { + switch (engineType) { + case SPARK: + return SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY; + case FLINK: + case JAVA: + return JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY; + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } } /** - * strategy types for build z-ordering/space-filling curves. + * Type of a strategy for building Z-order/Hilbert space-filling curves. */ public enum BuildCurveStrategyType { DIRECT("direct"), SAMPLE("sample"); + + private static final Map VALUE_TO_ENUM_MAP = + TypeUtils.getValueToEnumMap(BuildCurveStrategyType.class, e -> e.value); + private final String value; BuildCurveStrategyType(String value) { @@ -448,14 +581,39 @@ public enum BuildCurveStrategyType { } public static BuildCurveStrategyType fromValue(String value) { - switch (value.toLowerCase(Locale.ROOT)) { - case "direct": - return DIRECT; - case "sample": - return SAMPLE; - default: - throw new HoodieException("Invalid value of Type."); + BuildCurveStrategyType enumValue = VALUE_TO_ENUM_MAP.get(value); + if (enumValue == null) { + throw new HoodieException(String.format("Invalid value (%s)", value)); } + + return enumValue; + } + } + + /** + * Layout optimization strategies such as Z-order/Hilbert space-curves, etc + */ + public enum LayoutOptimizationStrategy { + ZORDER("z-order"), + HILBERT("hilbert"); + + private static final Map VALUE_TO_ENUM_MAP = + TypeUtils.getValueToEnumMap(LayoutOptimizationStrategy.class, e -> e.value); + + private final String value; + + LayoutOptimizationStrategy(String value) { + this.value = value; + } + + @Nonnull + public static LayoutOptimizationStrategy fromValue(String value) { + LayoutOptimizationStrategy enumValue = VALUE_TO_ENUM_MAP.get(value); + if (enumValue == null) { + throw new HoodieException(String.format("Invalid value (%s)", value)); + } + + return enumValue; } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index ce74aad6b098..640f0cb1826a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -57,6 +57,13 @@ public class HoodieCompactionConfig extends HoodieConfig { + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage" + " growth is bounded."); + public static final ConfigProperty AUTO_ARCHIVE = ConfigProperty + .key("hoodie.archive.automatic") + .defaultValue("true") + .withDocumentation("When enabled, the archival table service is invoked immediately after each commit," + + " to archive commits if we cross a maximum value of commits." + + " It's recommended to enable this, to ensure number of active commits is bounded."); + public static final ConfigProperty ASYNC_CLEAN = ConfigProperty .key("hoodie.clean.async") .defaultValue("false") @@ -119,6 +126,11 @@ public class HoodieCompactionConfig extends HoodieConfig { + " keep the metadata overhead constant, even as the table size grows." + "This config controls the maximum number of instants to retain in the active timeline. "); + public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.archive.delete.parallelism") + .defaultValue(100) + .withDocumentation("Parallelism for deleting archived hoodie commits."); + public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty .key("hoodie.keep.min.commits") .defaultValue("20") @@ -166,6 +178,12 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("Amount of MBs to spend during compaction run for the LogFileSizeBasedCompactionStrategy. " + "This value helps bound ingestion latency while compaction is run inline mode."); + public static final ConfigProperty COMPACTION_LOG_FILE_SIZE_THRESHOLD = ConfigProperty + .key("hoodie.compaction.logfile.size.threshold") + .defaultValue(0L) + .withDocumentation("Only if the log file size is greater than the threshold in bytes," + + " the file group will be compacted."); + public static final ConfigProperty COMPACTION_STRATEGY = ConfigProperty .key("hoodie.compaction.strategy") .defaultValue(LogFileSizeBasedCompactionStrategy.class.getName()) @@ -482,6 +500,11 @@ public Builder withAsyncClean(Boolean asyncClean) { return this; } + public Builder withAutoArchive(Boolean autoArchive) { + compactionConfig.setValue(AUTO_ARCHIVE, String.valueOf(autoArchive)); + return this; + } + public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) { compactionConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode)); return this; @@ -568,6 +591,11 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } + public Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { + compactionConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); + return this; + } + public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) { compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS, String.valueOf(maxDeltaSecondsBeforeCompaction)); return this; @@ -588,6 +616,11 @@ public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPer return this; } + public Builder withLogFileSizeThresholdBasedCompaction(long logFileSizeThreshold) { + compactionConfig.setValue(COMPACTION_LOG_FILE_SIZE_THRESHOLD, String.valueOf(logFileSizeThreshold)); + return this; + } + public Builder withCommitsArchivalBatchSize(int batchSize) { compactionConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE, String.valueOf(batchSize)); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java index 561460777ff6..3d7e3a7941da 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java @@ -76,9 +76,9 @@ public class HoodieHBaseIndexConfig extends HoodieConfig { .withDocumentation("Property to set which implementation of HBase QPS resource allocator to be used, which" + "controls the batching rate dynamically."); - public static final ConfigProperty PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty + public static final ConfigProperty PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty .key("hoodie.index.hbase.put.batch.size.autocompute") - .defaultValue("false") + .defaultValue(false) .withDocumentation("Property to set to enable auto computation of put batch size"); public static final ConfigProperty QPS_FRACTION = ConfigProperty @@ -206,7 +206,7 @@ public class HoodieHBaseIndexConfig extends HoodieConfig { * @deprecated Use {@link #PUT_BATCH_SIZE_AUTO_COMPUTE} and its methods instead */ @Deprecated - public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = PUT_BATCH_SIZE_AUTO_COMPUTE.defaultValue(); + public static final Boolean DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = PUT_BATCH_SIZE_AUTO_COMPUTE.defaultValue(); /** * @deprecated Use {@link #MAX_QPS_FRACTION} and its methods instead */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 4f9b9fcd50e8..e156310c736b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -24,15 +24,20 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import javax.annotation.concurrent.Immutable; import java.io.File; import java.io.FileReader; import java.io.IOException; +import java.util.Arrays; import java.util.Properties; +import java.util.stream.Collectors; import static org.apache.hudi.config.HoodieHBaseIndexConfig.GET_BATCH_SIZE; import static org.apache.hudi.config.HoodieHBaseIndexConfig.PUT_BATCH_SIZE; @@ -54,7 +59,7 @@ public class HoodieIndexConfig extends HoodieConfig { .key("hoodie.index.type") .noDefaultValue() .withDocumentation("Type of index to use. Default is Bloom filter. " - + "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE]. " + + "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE | BUCKET]. " + "Bloom filters removes the dependency on a external system " + "and is stored in the footer of the Parquet Data Files"); @@ -200,6 +205,30 @@ public class HoodieIndexConfig extends HoodieConfig { .defaultValue("true") .withDocumentation("Similar to " + BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE + ", but for simple index."); + /** + * ***** Bucket Index Configs ***** + * Bucket Index is targeted to locate the record fast by hash in big data scenarios. + * The current implementation is a basic version, so there are some constraints: + * 1. Unsupported operation: bulk insert, cluster and so on. + * 2. Bucket num change requires rewriting the partition. + * 3. Predict the table size and future data growth well to set a reasonable bucket num. + * 4. A bucket size is recommended less than 3GB and avoid bing too small. + * more details and progress see [HUDI-3039]. + */ + // Bucket num equals file groups num in each partition. + // Bucket num can be set according to partition size and file group size. + public static final ConfigProperty BUCKET_INDEX_NUM_BUCKETS = ConfigProperty + .key("hoodie.bucket.index.num.buckets") + .defaultValue(256) + .withDocumentation("Only applies if index type is BUCKET_INDEX. Determine the number of buckets in the hudi table, " + + "and each partition is divided to N buckets."); + + public static final ConfigProperty BUCKET_INDEX_HASH_FIELD = ConfigProperty + .key("hoodie.bucket.index.hash.field") + .noDefaultValue() + .withDocumentation("Index key. It is used to index the record and find its file group. " + + "If not set, use record key field as default"); + /** * Deprecated configs. These are now part of {@link HoodieHBaseIndexConfig}. */ @@ -516,12 +545,23 @@ public Builder withEngineType(EngineType engineType) { return this; } + public Builder withBucketNum(String bucketNum) { + hoodieIndexConfig.setValue(BUCKET_INDEX_NUM_BUCKETS, bucketNum); + return this; + } + + public Builder withIndexKeyField(String keyField) { + hoodieIndexConfig.setValue(BUCKET_INDEX_HASH_FIELD, keyField); + return this; + } + public HoodieIndexConfig build() { hoodieIndexConfig.setDefaultValue(INDEX_TYPE, getDefaultIndexType(engineType)); hoodieIndexConfig.setDefaults(HoodieIndexConfig.class.getName()); // Throws IllegalArgumentException if the value set is not a known Hoodie Index Type HoodieIndex.IndexType.valueOf(hoodieIndexConfig.getString(INDEX_TYPE)); + validateBucketIndexConfig(); return hoodieIndexConfig; } @@ -540,5 +580,27 @@ private String getDefaultIndexType(EngineType engineType) { public EngineType getEngineType() { return engineType; } + + private void validateBucketIndexConfig() { + if (hoodieIndexConfig.getString(INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.toString())) { + // check the bucket index hash field + if (StringUtils.isNullOrEmpty(hoodieIndexConfig.getString(BUCKET_INDEX_HASH_FIELD))) { + hoodieIndexConfig.setValue(BUCKET_INDEX_HASH_FIELD, + hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); + } else { + boolean valid = Arrays + .stream(hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME).split(",")) + .collect(Collectors.toSet()) + .containsAll(Arrays.asList(hoodieIndexConfig.getString(BUCKET_INDEX_HASH_FIELD).split(","))); + if (!valid) { + throw new HoodieIndexException("Bucket index key (if configured) must be subset of record key."); + } + } + // check the bucket num + if (hoodieIndexConfig.getIntOrDefault(BUCKET_INDEX_NUM_BUCKETS) <= 0) { + throw new HoodieIndexException("When using bucket index, hoodie.bucket.index.num.buckets cannot be negative."); + } + } + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLayoutConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLayoutConfig.java new file mode 100644 index 000000000000..183c00c7bc63 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLayoutConfig.java @@ -0,0 +1,89 @@ +/* + * 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 org.apache.hudi.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.storage.HoodieStorageLayout; + +import javax.annotation.concurrent.Immutable; +import java.util.Properties; + +/** + * Storage layout related config. + */ +@Immutable +@ConfigClassProperty(name = "Layout Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Configurations that control storage layout and data distribution, " + + "which defines how the files are organized within a table.") +public class HoodieLayoutConfig extends HoodieConfig { + + public static final ConfigProperty LAYOUT_TYPE = ConfigProperty + .key("hoodie.storage.layout.type") + .defaultValue("DEFAULT") + .withDocumentation("Type of storage layout. Possible options are [DEFAULT | BUCKET]"); + + public static final ConfigProperty LAYOUT_PARTITIONER_CLASS_NAME = ConfigProperty + .key("hoodie.storage.layout.partitioner.class") + .noDefaultValue() + .withDocumentation("Partitioner class, it is used to distribute data in a specific way."); + + private HoodieLayoutConfig() { + super(); + } + + public static HoodieLayoutConfig.Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + public HoodieLayoutConfig layoutConfig = new HoodieLayoutConfig(); + + public Builder fromProperties(Properties props) { + this.layoutConfig.getProps().putAll(props); + return this; + } + + public Builder withLayoutType(String type) { + layoutConfig.setValue(LAYOUT_TYPE, type); + return this; + } + + public Builder withLayoutPartitioner(String partitionerClass) { + layoutConfig.setValue(LAYOUT_PARTITIONER_CLASS_NAME, partitionerClass); + return this; + } + + public HoodieLayoutConfig build() { + setDefault(); + return layoutConfig; + } + + private void setDefault() { + if (layoutConfig.getString(HoodieIndexConfig.INDEX_TYPE.key()).equals(HoodieIndex.IndexType.BUCKET.name())) { + layoutConfig.setDefaultValue(LAYOUT_TYPE, HoodieStorageLayout.LayoutType.BUCKET.name()); + } + layoutConfig.setDefaultValue(LAYOUT_TYPE, LAYOUT_TYPE.defaultValue()); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java index fd7b310a57d6..723c9f3cbff4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.lock.LockProvider; +import org.apache.hudi.common.util.Option; import java.io.File; import java.io.FileReader; @@ -53,7 +54,6 @@ import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY; - /** * Hoodie Configs for Locks. */ @@ -160,9 +160,11 @@ public class HoodieLockConfig extends HoodieConfig { public static final ConfigProperty ZK_LOCK_KEY = ConfigProperty .key(ZK_LOCK_KEY_PROP_KEY) .noDefaultValue() + .withInferFunction(p -> Option.ofNullable(p.getStringOrDefault(HoodieWriteConfig.TBL_NAME, null))) .sinceVersion("0.8.0") .withDocumentation("Key name under base_path at which to create a ZNode and acquire lock. " - + "Final path on zk will look like base_path/lock_key. We recommend setting this to the table name"); + + "Final path on zk will look like base_path/lock_key. If this parameter is not set, we would " + + "set it as the table name"); // Pluggable type of lock provider public static final ConfigProperty LOCK_PROVIDER_CLASS_NAME = ConfigProperty diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index e167b12c8204..d8e4bd3b2430 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -45,6 +45,7 @@ import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.config.metrics.HoodieMetricsPrometheusConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.SimpleAvroKeyGenerator; @@ -52,10 +53,12 @@ import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.table.RandomFileIdPrefixProvider; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.orc.CompressionKind; import org.apache.parquet.hadoop.metadata.CompressionCodecName; @@ -120,9 +123,9 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty ROLLBACK_USING_MARKERS_ENABLE = ConfigProperty .key("hoodie.rollback.using.markers") - .defaultValue("false") + .defaultValue("true") .withDocumentation("Enables a more efficient mechanism for rollbacks based on the marker files generated " - + "during the writes. Turned off by default."); + + "during the writes. Turned on by default."); public static final ConfigProperty TIMELINE_LAYOUT_VERSION_NUM = ConfigProperty .key("hoodie.timeline.layout.version") @@ -248,14 +251,17 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty MARKERS_TYPE = ConfigProperty .key("hoodie.write.markers.type") - .defaultValue(MarkerType.DIRECT.toString()) + .defaultValue(MarkerType.TIMELINE_SERVER_BASED.toString()) .sinceVersion("0.9.0") .withDocumentation("Marker type to use. Two modes are supported: " + "- DIRECT: individual marker file corresponding to each data file is directly " + "created by the writer. " + "- TIMELINE_SERVER_BASED: marker operations are all handled at the timeline service " + "which serves as a proxy. New marker entries are batch processed and stored " - + "in a limited number of underlying files for efficiency."); + + "in a limited number of underlying files for efficiency. If HDFS is used or " + + "timeline server is disabled, DIRECT markers are used as fallback even if this " + + "is configure. For Spark structured streaming, this configuration does not " + + "take effect, i.e., DIRECT markers are always used for Spark structured streaming."); public static final ConfigProperty MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS = ConfigProperty .key("hoodie.markers.timeline_server_based.batch.num_threads") @@ -354,6 +360,12 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("When enabled, we allow duplicate keys even if inserts are routed to merge with an existing file (for ensuring file sizing)." + " This is only relevant for insert operation, since upsert, delete operations will ensure unique key constraints are maintained."); + public static final ConfigProperty MERGE_SMALL_FILE_GROUP_CANDIDATES_LIMIT = ConfigProperty + .key("hoodie.merge.small.file.group.candidates.limit") + .defaultValue(1) + .withDocumentation("Limits number of file groups, whose base file satisfies small-file limit, to consider for appending records during upsert operation. " + + "Only applicable to MOR tables"); + public static final ConfigProperty CLIENT_HEARTBEAT_INTERVAL_IN_MS = ConfigProperty .key("hoodie.client.heartbeat.interval_in_ms") .defaultValue(60 * 1000) @@ -372,14 +384,6 @@ public class HoodieWriteConfig extends HoodieConfig { + "OPTIMISTIC_CONCURRENCY_CONTROL: Multiple writers can operate on the table and exactly one of them succeed " + "if a conflict (writes affect the same file group) is detected."); - public static final ConfigProperty WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE = ConfigProperty - .key("hoodie.write.concurrency.merge.deltastreamer.state") - .defaultValue(false) - .withAlternatives("hoodie.write.meta.key.prefixes") - .withDocumentation("If enabled, this writer will merge Deltastreamer state from the previous checkpoint in order to allow both realtime " - + "and batch writers to ingest into a single table. This should not be enabled on Deltastreamer writers. Enabling this config means," - + "for a spark writer, deltastreamer checkpoint will be copied over from previous commit to the current one."); - /** * Currently the use this to specify the write schema. */ @@ -1035,6 +1039,10 @@ public boolean allowDuplicateInserts() { return getBoolean(MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE); } + public int getSmallFileGroupCandidatesLimit() { + return getInt(MERGE_SMALL_FILE_GROUP_CANDIDATES_LIMIT); + } + public EngineType getEngineType() { return engineType; } @@ -1095,6 +1103,10 @@ public boolean isAutoClean() { return getBoolean(HoodieCompactionConfig.AUTO_CLEAN); } + public boolean isAutoArchive() { + return getBoolean(HoodieCompactionConfig.AUTO_ARCHIVE); + } + public boolean isAsyncClean() { return getBoolean(HoodieCompactionConfig.ASYNC_CLEAN); } @@ -1127,6 +1139,10 @@ public Long getTargetIOPerCompactionInMB() { return getLong(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB); } + public Long getCompactionLogFileSizeThreshold() { + return getLong(HoodieCompactionConfig.COMPACTION_LOG_FILE_SIZE_THRESHOLD); + } + public Boolean getCompactionLazyBlockReadEnabled() { return getBoolean(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE); } @@ -1135,6 +1151,10 @@ public Boolean getCompactionReverseLogReadEnabled() { return getBoolean(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE); } + public int getArchiveDeleteParallelism() { + return getInt(HoodieCompactionConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); + } + public boolean inlineClusteringEnabled() { return getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING); } @@ -1152,6 +1172,10 @@ public boolean isClusteringEnabled() { return inlineClusteringEnabled() || isAsyncClusteringEnabled(); } + public boolean isRollbackPendingClustering() { + return getBoolean(HoodieClusteringConfig.ROLLBACK_PENDING_CLUSTERING_ON_CONFLICT); + } + public int getInlineClusterMaxCommits() { return getInt(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS); } @@ -1192,6 +1216,19 @@ public String getClusteringPlanStrategyClass() { return getString(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME); } + public ClusteringPlanPartitionFilterMode getClusteringPlanPartitionFilterMode() { + String mode = getString(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME); + return ClusteringPlanPartitionFilterMode.valueOf(mode); + } + + public String getBeginPartitionForClustering() { + return getString(HoodieClusteringConfig.PARTITION_FILTER_BEGIN_PARTITION); + } + + public String getEndPartitionForClustering() { + return getString(HoodieClusteringConfig.PARTITION_FILTER_END_PARTITION); + } + public String getClusteringExecutionStrategyClass() { return getString(HoodieClusteringConfig.EXECUTION_STRATEGY_CLASS_NAME); } @@ -1295,8 +1332,8 @@ public int getHbaseIndexPutBatchSize() { return getInt(HoodieHBaseIndexConfig.PUT_BATCH_SIZE); } - public Boolean getHbaseIndexPutBatchSizeAutoCompute() { - return getBoolean(HoodieHBaseIndexConfig.PUT_BATCH_SIZE_AUTO_COMPUTE); + public boolean getHbaseIndexPutBatchSizeAutoCompute() { + return getBooleanOrDefault(HoodieHBaseIndexConfig.PUT_BATCH_SIZE_AUTO_COMPUTE); } public String getHBaseQPSResourceAllocatorClass() { @@ -1357,7 +1394,7 @@ public int getHbaseIndexMaxQPSPerRegionServer() { } public boolean getHbaseIndexUpdatePartitionPath() { - return getBoolean(HoodieHBaseIndexConfig.UPDATE_PARTITION_PATH_ENABLE); + return getBooleanOrDefault(HoodieHBaseIndexConfig.UPDATE_PARTITION_PATH_ENABLE); } public int getBloomIndexParallelism() { @@ -1404,6 +1441,14 @@ public boolean getGlobalSimpleIndexUpdatePartitionPath() { return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE); } + public int getBucketIndexNumBuckets() { + return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS); + } + + public String getBucketIndexHashField() { + return getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD); + } + /** * storage properties. */ @@ -1558,6 +1603,22 @@ public List getDatadogMetricTags() { HoodieMetricsDatadogConfig.METRIC_TAG_VALUES, ",").split("\\s*,\\s*")).collect(Collectors.toList()); } + public int getCloudWatchReportPeriodSeconds() { + return getInt(HoodieMetricsCloudWatchConfig.REPORT_PERIOD_SECONDS); + } + + public String getCloudWatchMetricPrefix() { + return getString(HoodieMetricsCloudWatchConfig.METRIC_PREFIX); + } + + public String getCloudWatchMetricNamespace() { + return getString(HoodieMetricsCloudWatchConfig.METRIC_NAMESPACE); + } + + public int getCloudWatchMaxDatumsPerRequest() { + return getInt(HoodieMetricsCloudWatchConfig.MAX_DATUMS_PER_REQUEST); + } + public String getMetricReporterClassName() { return getString(HoodieMetricsConfig.METRICS_REPORTER_CLASS_NAME); } @@ -1590,6 +1651,10 @@ public boolean getPushGatewayRandomJobNameSuffix() { return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX); } + public String getMetricReporterMetricsNamePrefix() { + return getStringOrDefault(HoodieMetricsConfig.METRICS_REPORTER_PREFIX); + } + /** * memory configs. */ @@ -1760,10 +1825,6 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE)); } - public Boolean mergeDeltastreamerStateFromPreviousCommit() { - return getBoolean(HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE); - } - public Boolean inlineTableServices() { return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean(); } @@ -1796,6 +1857,13 @@ public String getFileIdPrefixProviderClassName() { return getString(FILEID_PREFIX_PROVIDER_CLASS); } + /** + * Layout configs. + */ + public HoodieStorageLayout.LayoutType getLayoutType() { + return HoodieStorageLayout.LayoutType.valueOf(getString(HoodieLayoutConfig.LAYOUT_TYPE)); + } + public static class Builder { protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); @@ -1817,6 +1885,7 @@ public static class Builder { private boolean isPreCommitValidationConfigSet = false; private boolean isMetricsJmxConfigSet = false; private boolean isMetricsGraphiteConfigSet = false; + private boolean isLayoutConfigSet = false; public Builder withEngineType(EngineType engineType) { this.engineType = engineType; @@ -2047,6 +2116,12 @@ public Builder withCallbackConfig(HoodieWriteCommitCallbackConfig callbackConfig return this; } + public Builder withLayoutConfig(HoodieLayoutConfig layoutConfig) { + writeConfig.getProps().putAll(layoutConfig.getProps()); + isLayoutConfigSet = true; + return this; + } + public Builder withFinalizeWriteParallelism(int parallelism) { writeConfig.setValue(FINALIZE_WRITE_PARALLELISM_VALUE, String.valueOf(parallelism)); return this; @@ -2112,6 +2187,11 @@ public Builder withMergeAllowDuplicateOnInserts(boolean routeInsertsToNewFiles) return this; } + public Builder withMergeSmallFileGroupCandidatesLimit(int limit) { + writeConfig.setValue(MERGE_SMALL_FILE_GROUP_CANDIDATES_LIMIT, String.valueOf(limit)); + return this; + } + public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) { writeConfig.setValue(CLIENT_HEARTBEAT_INTERVAL_IN_MS, String.valueOf(heartbeatIntervalInMs)); return this; @@ -2148,6 +2228,7 @@ public Builder withProperties(Properties properties) { } protected void setDefaults() { + writeConfig.setDefaultValue(MARKERS_TYPE, getDefaultMarkersType(engineType)); // Check for mandatory properties writeConfig.setDefaults(HoodieWriteConfig.class.getName()); // Make sure the props is propagated @@ -2159,7 +2240,8 @@ protected void setDefaults() { writeConfig.setDefaultOnCondition(!isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isClusteringConfigSet, - HoodieClusteringConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + HoodieClusteringConfig.newBuilder().withEngineType(engineType) + .fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties( writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isBootstrapConfigSet, @@ -2175,11 +2257,13 @@ protected void setDefaults() { writeConfig.setDefaultOnCondition(!isPayloadConfigSet, HoodiePayloadConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isMetadataConfigSet, - HoodieMetadataConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + HoodieMetadataConfig.newBuilder().withEngineType(engineType).fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isLockConfigSet, HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isPreCommitValidationConfigSet, HoodiePreCommitValidatorConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + writeConfig.setDefaultOnCondition(!isLayoutConfigSet, + HoodieLayoutConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); } @@ -2201,5 +2285,18 @@ public HoodieWriteConfig build() { // Build WriteConfig at the end return new HoodieWriteConfig(engineType, writeConfig.getProps()); } + + private String getDefaultMarkersType(EngineType engineType) { + switch (engineType) { + case SPARK: + return MarkerType.TIMELINE_SERVER_BASED.toString(); + case FLINK: + case JAVA: + // Timeline-server-based marker is not supported for Flink and Java engines + return MarkerType.DIRECT.toString(); + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java index 60369d72fb4f..8396540394b8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsConfig.java @@ -22,6 +22,9 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieMetricsCloudWatchConfig; import org.apache.hudi.metrics.MetricsReporterType; import javax.annotation.concurrent.Immutable; @@ -62,6 +65,18 @@ public class HoodieMetricsConfig extends HoodieConfig { .sinceVersion("0.6.0") .withDocumentation(""); + public static final ConfigProperty METRICS_REPORTER_PREFIX = ConfigProperty + .key(METRIC_PREFIX + ".reporter.metricsname.prefix") + .defaultValue("") + .sinceVersion("0.11.0") + .withInferFunction(cfg -> { + if (cfg.contains(HoodieTableConfig.NAME)) { + return Option.of(cfg.getString(HoodieTableConfig.NAME)); + } + return Option.empty(); + }) + .withDocumentation("The prefix given to the metrics names."); + // Enable metrics collection from executors public static final ConfigProperty EXECUTOR_METRICS_ENABLE = ConfigProperty .key(METRIC_PREFIX + ".executor.enable") @@ -165,6 +180,8 @@ public HoodieMetricsConfig build() { HoodieMetricsJmxConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build()); hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.GRAPHITE, HoodieMetricsGraphiteConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build()); + hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.CLOUDWATCH, + HoodieMetricsCloudWatchConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build()); return hoodieMetricsConfig; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsPrometheusConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsPrometheusConfig.java index c04e8aa1e980..e27ff1bcb089 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsPrometheusConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/metrics/HoodieMetricsPrometheusConfig.java @@ -166,11 +166,6 @@ public static HoodieMetricsPrometheusConfig.Builder newBuilder() { return new HoodieMetricsPrometheusConfig.Builder(); } - @Override - public Properties getProps() { - return super.getProps(); - } - public static class Builder { private HoodieMetricsPrometheusConfig hoodieMetricsPrometheusConfig = new HoodieMetricsPrometheusConfig(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 974f4d546b10..0428d12c4030 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieNotSupportedException; @@ -122,6 +123,20 @@ public abstract HoodieData updateLocation( @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) public abstract boolean isImplicitWithStorage(); + /** + * If the `getCustomizedPartitioner` returns a partitioner, it has to be true. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public boolean requiresTagging(WriteOperationType operationType) { + switch (operationType) { + case DELETE: + case UPSERT: + return true; + default: + return false; + } + } + /** * Each index type should implement it's own logic to release any resources acquired during the process. */ @@ -129,6 +144,6 @@ public void close() { } public enum IndexType { - HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE + HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, BUCKET } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java index 50d31f9cc50d..18b094890081 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java @@ -46,8 +46,8 @@ class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter { IntervalTreeBasedGlobalIndexFileFilter(final Map> partitionToFileIndexInfo) { List allIndexFiles = new ArrayList<>(); - partitionToFileIndexInfo.forEach((parition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> { - fileIdToPartitionPathMap.put(file.getFileId(), parition); + partitionToFileIndexInfo.forEach((partition, bloomIndexFileInfoList) -> bloomIndexFileInfoList.forEach(file -> { + fileIdToPartitionPathMap.put(file.getFileId(), partition); allIndexFiles.add(file); })); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java new file mode 100644 index 000000000000..7dee9f3cdfa3 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java @@ -0,0 +1,74 @@ +/* + * 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 org.apache.hudi.index.bucket; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class BucketIdentifier { + // compatible with the spark bucket name + private static final Pattern BUCKET_NAME = Pattern.compile(".*_(\\d+)(?:\\..*)?$"); + + public static int getBucketId(HoodieRecord record, String indexKeyFields, int numBuckets) { + return getBucketId(record.getKey(), indexKeyFields, numBuckets); + } + + public static int getBucketId(HoodieKey hoodieKey, String indexKeyFields, int numBuckets) { + List hashKeyFields; + if (!hoodieKey.getRecordKey().contains(":")) { + hashKeyFields = Arrays.asList(hoodieKey.getRecordKey()); + } else { + Map recordKeyPairs = Arrays.stream(hoodieKey.getRecordKey().split(",")) + .map(p -> p.split(":")) + .collect(Collectors.toMap(p -> p[0], p -> p[1])); + hashKeyFields = Arrays.stream(indexKeyFields.split(",")) + .map(f -> recordKeyPairs.get(f)) + .collect(Collectors.toList()); + } + return (hashKeyFields.hashCode() & Integer.MAX_VALUE) % numBuckets; + } + + // only for test + public static int getBucketId(List hashKeyFields, int numBuckets) { + return hashKeyFields.hashCode() % numBuckets; + } + + public static int bucketIdFromFileId(String fileId) { + return Integer.parseInt(fileId.substring(0, 8)); + } + + public static String bucketIdStr(int n) { + return String.format("%08d", n); + } + + public static String newBucketFileIdPrefix(String bucketId) { + return FSUtils.createNewFileIdPfx().replaceFirst(".{8}", bucketId); + } + + public static boolean isBucketFileName(String name) { + return BUCKET_NAME.matcher(name).matches(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java new file mode 100644 index 000000000000..acb06ea48bed --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java @@ -0,0 +1,166 @@ +/* + * 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 org.apache.hudi.index.bucket; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.Map; + +/** + * Hash indexing mechanism. + * @param + */ +public class HoodieBucketIndex> + extends HoodieIndex { + + private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class); + + private final int numBuckets; + + public HoodieBucketIndex(HoodieWriteConfig config) { + super(config); + numBuckets = config.getBucketIndexNumBuckets(); + LOG.info("use bucket index, numBuckets=" + numBuckets); + } + + @Override + public HoodieData updateLocation(HoodieData writeStatuses, + HoodieEngineContext context, + HoodieTable hoodieTable) + throws HoodieIndexException { + return writeStatuses; + } + + @Override + public HoodieData> tagLocation(HoodieData> records, + HoodieEngineContext context, + HoodieTable hoodieTable) + throws HoodieIndexException { + HoodieData> taggedRecords = records.mapPartitions(recordIter -> { + // partitionPath -> bucketId -> fileInfo + Map>> partitionPathFileIDList = new HashMap<>(); + return new LazyIterableIterator, HoodieRecord>(recordIter) { + + @Override + protected void start() { + + } + + @Override + protected HoodieRecord computeNext() { + HoodieRecord record = recordIter.next(); + int bucketId = BucketIdentifier.getBucketId(record, config.getBucketIndexHashField(), numBuckets); + String partitionPath = record.getPartitionPath(); + if (!partitionPathFileIDList.containsKey(partitionPath)) { + partitionPathFileIDList.put(partitionPath, loadPartitionBucketIdFileIdMapping(hoodieTable, partitionPath)); + } + if (partitionPathFileIDList.get(partitionPath).containsKey(bucketId)) { + Pair fileInfo = partitionPathFileIDList.get(partitionPath).get(bucketId); + return HoodieIndexUtils.getTaggedRecord(record, Option.of( + new HoodieRecordLocation(fileInfo.getRight(), fileInfo.getLeft()) + )); + } + return record; + } + + @Override + protected void end() { + + } + }; + }, true); + return taggedRecords; + } + + private Map> loadPartitionBucketIdFileIdMapping( + HoodieTable hoodieTable, + String partition) { + // bucketId -> fileIds + Map> fileIDList = new HashMap<>(); + HoodieIndexUtils + .getLatestBaseFilesForPartition(partition, hoodieTable) + .forEach(file -> { + String fileId = file.getFileId(); + String commitTime = file.getCommitTime(); + int bucketId = BucketIdentifier.bucketIdFromFileId(fileId); + if (!fileIDList.containsKey(bucketId)) { + fileIDList.put(bucketId, Pair.of(fileId, commitTime)); + } else { + // check if bucket data is valid + throw new HoodieIOException("Find multiple files at partition path=" + + partition + " belongs to the same bucket id = " + bucketId); + } + }); + return fileIDList; + } + + @Override + public boolean rollbackCommit(String instantTime) { + return true; + } + + @Override + public boolean isGlobal() { + return false; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public boolean requiresTagging(WriteOperationType operationType) { + switch (operationType) { + case INSERT: + case INSERT_OVERWRITE: + case UPSERT: + return true; + default: + return false; + } + } + + public int getNumBuckets() { + return numBuckets; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index a33383a05c02..7cc0c5dfa465 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -29,10 +29,10 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.log.AppendResult; @@ -178,7 +178,7 @@ private void init(HoodieRecord record) { LOG.error("Error in update task at commit " + instantTime, e); writeStatus.setGlobalError(e); throw new HoodieUpsertException("Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit " - + instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + partitionPath, e); + + instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + "/" + partitionPath, e); } doInit = false; } @@ -360,7 +360,12 @@ protected void appendDataAndDeleteBlocks(Map header) header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString()); List blocks = new ArrayList<>(2); if (recordList.size() > 0) { - blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); + if (config.populateMetaFields()) { + blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); + } else { + final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header, keyField)); + } } if (keysToDelete.size() > 0) { blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 01ad45342c50..22fa31539d88 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -119,7 +119,8 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa @Override public boolean canWrite(HoodieRecord record) { - return fileWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath()); + return (fileWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath())) + || layoutControlsNumFiles(); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index b13b561f5dff..d1d67efff4b9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -250,11 +250,17 @@ protected void init(String fileId, Iterator> newRecordsItr) { + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); } - private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Option indexedRecord) { + private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option indexedRecord) { + boolean isDelete = false; if (indexedRecord.isPresent()) { updatedRecordsWritten++; + GenericRecord record = (GenericRecord) indexedRecord.get(); + if (oldRecord != record) { + // the incoming record is chosen + isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); + } } - return writeRecord(hoodieRecord, indexedRecord); + return writeRecord(hoodieRecord, indexedRecord, isDelete); } protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOException { @@ -264,12 +270,16 @@ protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOExceptio if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) { return; } - if (writeRecord(hoodieRecord, insertRecord)) { + if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) { insertRecordsWritten++; } } protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { + return writeRecord(hoodieRecord, indexedRecord, false); + } + + protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete) { Option recordMetadata = hoodieRecord.getData().getMetadata(); if (!partitionPath.equals(hoodieRecord.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " @@ -277,11 +287,8 @@ protected boolean writeRecord(HoodieRecord hoodieRecord, Option HoodieFi BloomFilter filter = createBloomFilter(config); HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), - config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter); + config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), + PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); - return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier); + return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields()); } private static HoodieFileWriter newOrcFileWriter( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java index 031f92cccdfa..7e4c519a8faf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java @@ -18,35 +18,35 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.common.bloom.BloomFilter; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hudi.common.bloom.BloomFilter; public class HoodieHFileConfig { - private Compression.Algorithm compressionAlgorithm; - private int blockSize; - private long maxFileSize; - private boolean prefetchBlocksOnOpen; - private boolean cacheDataInL1; - private boolean dropBehindCacheCompaction; - private Configuration hadoopConf; - private BloomFilter bloomFilter; - + public static final KeyValue.KVComparator HFILE_COMPARATOR = new HoodieHBaseKVComparator(); + public static final boolean PREFETCH_ON_OPEN = CacheConfig.DEFAULT_PREFETCH_ON_OPEN; + public static final boolean CACHE_DATA_IN_L1 = HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1; // This is private in CacheConfig so have been copied here. - private static boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true; - - public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize, - long maxFileSize, BloomFilter bloomFilter) { - this(hadoopConf, compressionAlgorithm, blockSize, maxFileSize, CacheConfig.DEFAULT_PREFETCH_ON_OPEN, - HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION_DEFAULT, bloomFilter); - } + public static final boolean DROP_BEHIND_CACHE_COMPACTION = true; + + private final Compression.Algorithm compressionAlgorithm; + private final int blockSize; + private final long maxFileSize; + private final boolean prefetchBlocksOnOpen; + private final boolean cacheDataInL1; + private final boolean dropBehindCacheCompaction; + private final Configuration hadoopConf; + private final BloomFilter bloomFilter; + private final KeyValue.KVComparator hfileComparator; public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize, long maxFileSize, boolean prefetchBlocksOnOpen, boolean cacheDataInL1, - boolean dropBehindCacheCompaction, BloomFilter bloomFilter) { + boolean dropBehindCacheCompaction, BloomFilter bloomFilter, KeyValue.KVComparator hfileComparator) { this.hadoopConf = hadoopConf; this.compressionAlgorithm = compressionAlgorithm; this.blockSize = blockSize; @@ -55,6 +55,7 @@ public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compres this.cacheDataInL1 = cacheDataInL1; this.dropBehindCacheCompaction = dropBehindCacheCompaction; this.bloomFilter = bloomFilter; + this.hfileComparator = hfileComparator; } public Configuration getHadoopConf() { @@ -92,4 +93,8 @@ public boolean useBloomFilter() { public BloomFilter getBloomFilter() { return bloomFilter; } + + public KeyValue.KVComparator getHfileComparator() { + return hfileComparator; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 6747c4a4a80e..a719bcb8f334 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -61,6 +62,7 @@ public class HoodieHFileWriter boolean isBootstrapNeeded(Option rollbackedTimestamps = Collections.emptyList(); - if (actionMetadata.isPresent() && actionMetadata.get() instanceof HoodieRollbackMetadata) { - isRollbackAction = true; - List rollbackedInstants = - ((HoodieRollbackMetadata) actionMetadata.get()).getInstantsRollback(); - rollbackedTimestamps = rollbackedInstants.stream().map(instant -> { - return instant.getCommitTime().toString(); - }).collect(Collectors.toList()); - } - + // Detect the commit gaps if any from the data and the metadata active timeline if (dataMetaClient.getActiveTimeline().getAllCommitsTimeline().isBeforeTimelineStarts( latestMetadataInstant.get().getTimestamp()) - && (!isRollbackAction || !rollbackedTimestamps.contains(latestMetadataInstantTimestamp))) { - LOG.warn("Metadata Table will need to be re-bootstrapped as un-synced instants have been archived." + && !isCommitRevertedByInFlightAction(actionMetadata, latestMetadataInstantTimestamp)) { + LOG.error("Metadata Table will need to be re-bootstrapped as un-synced instants have been archived." + " latestMetadataInstant=" + latestMetadataInstant.get().getTimestamp() + ", latestDataInstant=" + dataMetaClient.getActiveTimeline().firstInstant().get().getTimestamp()); return true; @@ -359,10 +370,59 @@ private boolean isBootstrapNeeded(Option - ActionMetadata type + * @return True if the latest instant action is reverted by the action + */ + private boolean isCommitRevertedByInFlightAction(Option actionMetadata, + final String latestMetadataInstantTimestamp) { + if (!actionMetadata.isPresent()) { + return false; + } + + final String INSTANT_ACTION = (actionMetadata.get() instanceof HoodieRollbackMetadata + ? HoodieTimeline.ROLLBACK_ACTION + : (actionMetadata.get() instanceof HoodieRestoreMetadata ? HoodieTimeline.RESTORE_ACTION : "")); + + List affectedInstantTimestamps; + switch (INSTANT_ACTION) { + case HoodieTimeline.ROLLBACK_ACTION: + List rollbackedInstants = + ((HoodieRollbackMetadata) actionMetadata.get()).getInstantsRollback(); + affectedInstantTimestamps = rollbackedInstants.stream().map(instant -> { + return instant.getCommitTime().toString(); + }).collect(Collectors.toList()); + + if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) { + return true; + } + break; + case HoodieTimeline.RESTORE_ACTION: + List restoredInstants = + ((HoodieRestoreMetadata) actionMetadata.get()).getRestoreInstantInfo(); + affectedInstantTimestamps = restoredInstants.stream().map(instant -> { + return instant.getCommitTime().toString(); + }).collect(Collectors.toList()); + + if (affectedInstantTimestamps.contains(latestMetadataInstantTimestamp)) { + return true; + } + break; + default: + return false; + } + + return false; + } + /** * Initialize the Metadata Table by listing files and partitions from the file system. * - * @param dataMetaClient {@code HoodieTableMetaClient} for the dataset. + * @param dataMetaClient {@code HoodieTableMetaClient} for the dataset. * @param inflightInstantTimestamp */ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient, @@ -392,15 +452,19 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(tableName) .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) - .setPayloadClassName(HoodieMetadataPayload.class.getName()) - .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) - .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); + .setPayloadClassName(HoodieMetadataPayload.class.getName()) + .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) + .setRecordKeyFields(RECORD_KEY_FIELD) + .setPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) + .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) + .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1); // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); + engineContext.setJobStatus(this.getClass().getSimpleName(), "Bootstrap: initializing metadata table by listing files and partitions"); List dirInfoList = listAllPartitions(dataMetaClient); // During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these @@ -619,7 +683,10 @@ public void close() throws Exception { * deltacommit. */ protected void compactIfNecessary(AbstractHoodieWriteClient writeClient, String instantTime) { - String latestDeltacommitTime = metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() + // finish off any pending compactions if any from previous attempt. + writeClient.runAnyPendingCompactions(); + + String latestDeltacommitTime = metadataMetaClient.reloadActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() .get().getTimestamp(); List pendingInstants = dataMetaClient.reloadActiveTimeline().filterInflightsAndRequested() .findInstantsBefore(latestDeltacommitTime).getInstants().collect(Collectors.toList()); @@ -639,7 +706,20 @@ protected void compactIfNecessary(AbstractHoodieWriteClient writeClient, String } } - protected void doClean(AbstractHoodieWriteClient writeClient, String instantTime) { + protected void cleanIfNecessary(AbstractHoodieWriteClient writeClient, String instantTime) { + Option lastCompletedCompactionInstant = metadataMetaClient.reloadActiveTimeline() + .getCommitTimeline().filterCompletedInstants().lastInstant(); + if (lastCompletedCompactionInstant.isPresent() + && metadataMetaClient.getActiveTimeline().filterCompletedInstants() + .findInstantsAfter(lastCompletedCompactionInstant.get().getTimestamp()).countInstants() < 3) { + // do not clean the log files immediately after compaction to give some buffer time for metadata table reader, + // because there is case that the reader has prepared for the log file readers already before the compaction completes + // while before/during the reading of the log files, the cleaning triggers and delete the reading files, + // then a FileNotFoundException(for LogFormatReader) or NPE(for HFileReader) would throw. + + // 3 is a value that I think is enough for metadata table reader. + return; + } // Trigger cleaning with suffixes based on the same instant time. This ensures that any future // delta commits synced over will not have an instant time lesser than the last completed instant on the // metadata table. @@ -652,7 +732,8 @@ protected void doClean(AbstractHoodieWriteClient writeClient, String instantTime * */ protected void bootstrapCommit(List partitionInfoList, String createInstantTime) { - List partitions = partitionInfoList.stream().map(p -> p.getRelativePath()).collect(Collectors.toList()); + List partitions = partitionInfoList.stream().map(p -> + p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList()); final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum(); // Record which saves the list of all partitions @@ -667,7 +748,7 @@ protected void bootstrapCommit(List partitionInfoList, String cre HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { // Record which saves files within a partition return HoodieMetadataPayload.createPartitionFilesRecord( - partitionInfo.getRelativePath(), Option.of(partitionInfo.getFileNameToSizeMap()), Option.empty()); + partitionInfo.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : partitionInfo.getRelativePath(), Option.of(partitionInfo.getFileNameToSizeMap()), Option.empty()); }); partitionRecords = partitionRecords.union(fileListRecords); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java new file mode 100644 index 000000000000..e9d7aec8c8e2 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataKeyGenerator.java @@ -0,0 +1,52 @@ +/* + * 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 org.apache.hudi.metadata; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenUtils; + +/** + * Custom key generator for the Hoodie table metadata. The metadata table record payload + * has an internal schema with a known key field HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY. + * With or without the virtual keys, getting the key from the metadata table record is always + * via the above field and there is no real need for a key generator. But, when a write + * client is instantiated for the metadata table, when virtual keys are enabled, and when + * key generator class is not configured, the default SimpleKeyGenerator will be used. + * To avoid using any other key generators for the metadata table which rely on certain + * config properties, we need this custom key generator exclusively for the metadata table. + */ +public class HoodieTableMetadataKeyGenerator extends BaseKeyGenerator { + + public HoodieTableMetadataKeyGenerator(TypedProperties config) { + super(config); + } + + @Override + public String getRecordKey(GenericRecord record) { + return KeyGenUtils.getRecordKey(record, HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY); + } + + @Override + public String getPartitionPath(GenericRecord record) { + return ""; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 67befea42456..e874047b8c64 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -215,7 +215,7 @@ public void updateIndexMetrics(final String action, final long durationInMs) { } String getMetricsName(String action, String metric) { - return config == null ? null : String.format("%s.%s.%s", tableName, action, metric); + return config == null ? null : String.format("%s.%s.%s", config.getMetricReporterMetricsNamePrefix(), action, metric); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java index 1e4d83ab57ee..b570f512f371 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java @@ -46,7 +46,7 @@ public class Metrics { private Metrics(HoodieWriteConfig metricConfig) { registry = new MetricRegistry(); - commonMetricPrefix = metricConfig.getTableName(); + commonMetricPrefix = metricConfig.getMetricReporterMetricsNamePrefix(); reporter = MetricsReporterFactory.createReporter(metricConfig, registry); if (reporter == null) { throw new RuntimeException("Cannot initialize Reporter."); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java index 820c1a3e8887..dc9e80431b8d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metrics.cloudwatch.CloudWatchMetricsReporter; import org.apache.hudi.metrics.datadog.DatadogMetricsReporter; import com.codahale.metrics.MetricRegistry; @@ -77,6 +78,9 @@ public static MetricsReporter createReporter(HoodieWriteConfig config, MetricReg case CONSOLE: reporter = new ConsoleMetricsReporter(registry); break; + case CLOUDWATCH: + reporter = new CloudWatchMetricsReporter(config, registry); + break; default: LOG.error("Reporter type[" + type + "] is not supported."); break; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java index 36b15a89ac88..3c8600159287 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java @@ -22,5 +22,5 @@ * Types of the reporter supported, hudi also supports user defined reporter. */ public enum MetricsReporterType { - GRAPHITE, INMEMORY, JMX, DATADOG, CONSOLE, PROMETHEUS_PUSHGATEWAY, PROMETHEUS + GRAPHITE, INMEMORY, JMX, DATADOG, CONSOLE, PROMETHEUS_PUSHGATEWAY, PROMETHEUS, CLOUDWATCH } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/cloudwatch/CloudWatchMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/cloudwatch/CloudWatchMetricsReporter.java new file mode 100644 index 000000000000..c7134a10360f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/cloudwatch/CloudWatchMetricsReporter.java @@ -0,0 +1,85 @@ +/* + * 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 org.apache.hudi.metrics.cloudwatch; + +import org.apache.hudi.aws.cloudwatch.CloudWatchReporter; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metrics.MetricsReporter; + +import com.codahale.metrics.MetricRegistry; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Closeable; +import java.util.concurrent.TimeUnit; + +/** + * Hudi Amazon CloudWatch metrics reporter. Responsible for reading Hoodie metrics configurations and hooking up with + * {@link org.apache.hudi.metrics.Metrics}. Internally delegates reporting tasks to {@link CloudWatchReporter}. + */ +public class CloudWatchMetricsReporter extends MetricsReporter { + + private static final Logger LOG = LogManager.getLogger(CloudWatchMetricsReporter.class); + + private final MetricRegistry registry; + private final HoodieWriteConfig config; + private final CloudWatchReporter reporter; + + public CloudWatchMetricsReporter(HoodieWriteConfig config, MetricRegistry registry) { + this.config = config; + this.registry = registry; + this.reporter = createCloudWatchReporter(); + } + + CloudWatchMetricsReporter(HoodieWriteConfig config, MetricRegistry registry, CloudWatchReporter reporter) { + this.config = config; + this.registry = registry; + this.reporter = reporter; + } + + private CloudWatchReporter createCloudWatchReporter() { + return CloudWatchReporter.forRegistry(registry) + .prefixedWith(config.getCloudWatchMetricPrefix()) + .namespace(config.getCloudWatchMetricNamespace()) + .maxDatumsPerRequest(config.getCloudWatchMaxDatumsPerRequest()) + .build(config.getProps()); + } + + @Override + public void start() { + LOG.info("Starting CloudWatch Metrics Reporter."); + reporter.start(config.getCloudWatchReportPeriodSeconds(), TimeUnit.SECONDS); + } + + @Override + public void report() { + reporter.report(); + } + + @Override + public Closeable getReporter() { + return reporter; + } + + @Override + public void stop() { + LOG.info("Stopping CloudWatch Metrics Reporter."); + reporter.stop(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/HilbertCurveUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/HilbertCurveUtils.java new file mode 100644 index 000000000000..0f216abeee74 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/HilbertCurveUtils.java @@ -0,0 +1,52 @@ +/* + * 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 org.apache.hudi.optimize; + +import org.davidmoten.hilbert.HilbertCurve; + +import java.math.BigInteger; + +/** + * Utils for Hilbert Curve. + */ +public class HilbertCurveUtils { + public static byte[] indexBytes(HilbertCurve hilbertCurve, long[] points, int paddingNum) { + BigInteger index = hilbertCurve.index(points); + return paddingToNByte(index.toByteArray(), paddingNum); + } + + public static byte[] paddingToNByte(byte[] a, int paddingNum) { + if (a.length == paddingNum) { + return a; + } + if (a.length > paddingNum) { + byte[] result = new byte[paddingNum]; + System.arraycopy(a, 0, result, 0, paddingNum); + return result; + } + int paddingSize = paddingNum - a.length; + byte[] result = new byte[paddingNum]; + for (int i = 0; i < paddingSize; i++) { + result[i] = 0; + } + System.arraycopy(a, 0, result, paddingSize, a.length); + return result; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/FileIdPrefixProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/FileIdPrefixProvider.java index d06da9b0d9b9..0fc0823184ea 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/FileIdPrefixProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/FileIdPrefixProvider.java @@ -18,17 +18,17 @@ package org.apache.hudi.table; -import java.util.Properties; +import org.apache.hudi.common.config.TypedProperties; public abstract class FileIdPrefixProvider { - private final Properties props; + private final TypedProperties props; - public FileIdPrefixProvider(Properties props) { + public FileIdPrefixProvider(TypedProperties props) { this.props = props; } - public Properties getProps() { + public TypedProperties getProps() { return props; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 6046374ba107..40e3a316db10 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -74,9 +74,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.table.storage.HoodieLayoutFactory; +import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.Nonnull; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -105,6 +108,7 @@ public abstract class HoodieTable implem private SerializableConfiguration hadoopConfiguration; protected final TaskContextSupplier taskContextSupplier; private final HoodieTableMetadata metadata; + private final HoodieStorageLayout storageLayout; private transient FileSystemViewManager viewManager; protected final transient HoodieEngineContext context; @@ -122,11 +126,16 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata); this.metaClient = metaClient; this.index = getIndex(config, context); + this.storageLayout = getStorageLayout(config); this.taskContextSupplier = context.getTaskContextSupplier(); } protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); + protected HoodieStorageLayout getStorageLayout(HoodieWriteConfig config) { + return HoodieLayoutFactory.createLayout(config); + } + private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata); @@ -245,14 +254,17 @@ public abstract HoodieWriteMetadata bulkInsertPrepped(HoodieEngineContext con public abstract HoodieWriteMetadata insertOverwriteTable(HoodieEngineContext context, String instantTime, I records); /** - * update statistics info for current table. - * to do adaptation, once RFC-27 is finished. + * Updates Metadata Indexes (like Column Stats index) + * TODO rebase onto metadata table (post RFC-27) * - * @param context HoodieEngineContext - * @param instantTime Instant time for the replace action - * @param isOptimizeOperation whether current operation is OPTIMIZE type + * @param context instance of {@link HoodieEngineContext} + * @param instantTime instant of the carried operation triggering the update */ - public abstract void updateStatistics(HoodieEngineContext context, List stats, String instantTime, Boolean isOptimizeOperation); + public abstract void updateMetadataIndexes( + @Nonnull HoodieEngineContext context, + @Nonnull List stats, + @Nonnull String instantTime + ) throws Exception; public HoodieWriteConfig getConfig() { return config; @@ -361,6 +373,10 @@ public HoodieActiveTimeline getActiveTimeline() { return index; } + public HoodieStorageLayout getStorageLayout() { + return storageLayout; + } + /** * Schedule compaction for the instant time. * @@ -442,12 +458,13 @@ public abstract Option scheduleCleaning(HoodieEngineContext c * @param context HoodieEngineContext * @param instantTime Instant Time for scheduling rollback * @param instantToRollback instant to be rolled back + * @param shouldRollbackUsingMarkers uses marker based rollback strategy when set to true. uses list based rollback when false. * @return HoodieRollbackPlan containing info on rollback. */ public abstract Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers); /** * Rollback the (inflight/committed) record changes with the given commit time. @@ -490,7 +507,7 @@ public abstract HoodieRestoreMetadata restore(HoodieEngineContext context, */ public void rollbackInflightCompaction(HoodieInstant inflightInstant) { String commitTime = HoodieActiveTimeline.createNewInstantTime(); - scheduleRollback(context, commitTime, inflightInstant, false); + scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); rollback(context, commitTime, inflightInstant, false, false); getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); } @@ -732,10 +749,11 @@ public HoodieEngineContext getContext() { /** * Get Table metadata writer. * + * @param triggeringInstantTimestamp - The instant that is triggering this metadata write * @return instance of {@link HoodieTableMetadataWriter */ - public final Option getMetadataWriter() { - return getMetadataWriter(Option.empty()); + public final Option getMetadataWriter(String triggeringInstantTimestamp) { + return getMetadataWriter(triggeringInstantTimestamp, Option.empty()); } /** @@ -747,10 +765,19 @@ public final Option getMetadataWriter() { /** * Get Table metadata writer. + *

+ * Note: + * Get the metadata writer for the conf. If the metadata table doesn't exist, + * this wil trigger the creation of the table and the initial bootstrapping. + * Since this call is under the transaction lock, other concurrent writers + * are blocked from doing the similar initial metadata table creation and + * the bootstrapping. * + * @param triggeringInstantTimestamp - The instant that is triggering this metadata write * @return instance of {@link HoodieTableMetadataWriter} */ - public Option getMetadataWriter(Option actionMetadata) { + public Option getMetadataWriter(String triggeringInstantTimestamp, + Option actionMetadata) { // Each engine is expected to override this and // provide the actual metadata writer, if enabled. return Option.empty(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index d492fb6577a9..58e03f5f39fc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -18,9 +18,11 @@ package org.apache.hudi.table; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -127,7 +129,7 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException LOG.info("Archiving instants " + instantsToArchive); archive(context, instantsToArchive); LOG.info("Deleting archived instants " + instantsToArchive); - success = deleteArchivedInstants(instantsToArchive); + success = deleteArchivedInstants(instantsToArchive, context); } else { LOG.info("No Instants to archive"); } @@ -154,10 +156,14 @@ private Stream getCleanInstantsToArchive() { private Stream getCommitInstantsToArchive() { // TODO (na) : Add a way to return actions associated with a timeline and then merge/unify - // with logic above to avoid Stream.concats + // with logic above to avoid Stream.concat HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); - Option oldestPendingCompactionInstant = - table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); + + Option oldestPendingCompactionAndReplaceInstant = table.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)) + .filter(s -> !s.isCompleted()) + .firstInstant(); + Option oldestInflightCommitInstant = table.getActiveTimeline() .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) @@ -170,11 +176,11 @@ private Stream getCommitInstantsToArchive() { // Actually do the commits Stream instantToArchiveStream = commitTimeline.getInstants() .filter(s -> { - // if no savepoint present, then dont filter + // if no savepoint present, then don't filter return !(firstSavepoint.isPresent() && HoodieTimeline.compareTimestamps(firstSavepoint.get().getTimestamp(), LESSER_THAN_OR_EQUALS, s.getTimestamp())); }).filter(s -> { // Ensure commits >= oldest pending compaction commit is retained - return oldestPendingCompactionInstant + return oldestPendingCompactionAndReplaceInstant .map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) .orElse(true); }); @@ -224,19 +230,34 @@ private Stream getInstantsToArchive() { HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream()); } - private boolean deleteArchivedInstants(List archivedInstants) throws IOException { + private boolean deleteArchivedInstants(List archivedInstants, HoodieEngineContext context) throws IOException { LOG.info("Deleting instants " + archivedInstants); boolean success = true; - for (HoodieInstant archivedInstant : archivedInstants) { - Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); - try { - if (metaClient.getFs().exists(commitFile)) { - success &= metaClient.getFs().delete(commitFile, false); - LOG.info("Archived and deleted instant file " + commitFile); - } - } catch (IOException e) { - throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); - } + List instantFiles = archivedInstants.stream().map(archivedInstant -> + new Path(metaClient.getMetaPath(), archivedInstant.getFileName()) + ).map(Path::toString).collect(Collectors.toList()); + + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants"); + Map resultDeleteInstantFiles = FSUtils.parallelizeFilesProcess(context, + metaClient.getFs(), + config.getArchiveDeleteParallelism(), + pairOfSubPathAndConf -> { + Path commitFile = new Path(pairOfSubPathAndConf.getKey()); + try { + FileSystem fs = commitFile.getFileSystem(pairOfSubPathAndConf.getValue().get()); + if (fs.exists(commitFile)) { + return fs.delete(commitFile, false); + } + return true; + } catch (IOException e) { + throw new HoodieIOException("Failed to delete archived instant " + commitFile, e); + } + }, + instantFiles); + + for (Map.Entry result : resultDeleteInstantFiles.entrySet()) { + LOG.info("Archived and deleted instant file " + result.getKey() + " : " + result.getValue()); + success &= result.getValue(); } // Remove older meta-data from auxiliary path too @@ -244,7 +265,7 @@ private boolean deleteArchivedInstants(List archivedInstants) thr || (i.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)))).max(Comparator.comparing(HoodieInstant::getTimestamp))); LOG.info("Latest Committed Instant=" + latestCommitted); if (latestCommitted.isPresent()) { - success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); + success &= deleteAllInstantsOlderOrEqualsInAuxMetaFolder(latestCommitted.get()); } return success; } @@ -256,7 +277,7 @@ private boolean deleteArchivedInstants(List archivedInstants) thr * @return success if all eligible file deleted successfully * @throws IOException in case of error */ - private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant) throws IOException { + private boolean deleteAllInstantsOlderOrEqualsInAuxMetaFolder(HoodieInstant thresholdInstant) throws IOException { List instants = null; boolean success = true; try { @@ -270,12 +291,12 @@ private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thre * On some FSs deletion of all files in the directory can auto remove the directory itself. * GCS is one example, as it doesn't have real directories and subdirectories. When client * removes all the files from a "folder" on GCS is has to create a special "/" to keep the folder - * around. If this doesn't happen (timeout, misconfigured client, ...) folder will be deleted and + * around. If this doesn't happen (timeout, mis configured client, ...) folder will be deleted and * in this case we should not break when aux folder is not found. * GCS information: (https://cloud.google.com/storage/docs/gsutil/addlhelp/HowSubdirectoriesWork) */ LOG.warn("Aux path not found. Skipping: " + metaClient.getMetaAuxiliaryPath()); - return success; + return true; } List instantsToBeDeleted = @@ -287,7 +308,7 @@ private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thre Path metaFile = new Path(metaClient.getMetaAuxiliaryPath(), deleteInstant.getFileName()); if (metaClient.getFs().exists(metaFile)) { success &= metaClient.getFs().delete(metaFile, false); - LOG.info("Deleted instant file in auxiliary metapath : " + metaFile); + LOG.info("Deleted instant file in auxiliary meta path : " + metaFile); } } return success; @@ -300,10 +321,19 @@ public void archive(HoodieEngineContext context, List instants) t List records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { try { - deleteAnyLeftOverMarkers(context, hoodieInstant); - records.add(convertToAvroRecord(hoodieInstant)); - if (records.size() >= this.config.getCommitArchivalBatchSize()) { - writeToFile(wrapperSchema, records); + if (table.getActiveTimeline().isEmpty(hoodieInstant) + && ( + hoodieInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION) + || (hoodieInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && hoodieInstant.isCompleted()) + ) + ) { + table.getActiveTimeline().deleteEmptyInstantIfExists(hoodieInstant); + } else { + deleteAnyLeftOverMarkers(context, hoodieInstant); + records.add(convertToAvroRecord(hoodieInstant)); + if (records.size() >= this.config.getCommitArchivalBatchSize()) { + writeToFile(wrapperSchema, records); + } } } catch (Exception e) { LOG.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e); @@ -329,7 +359,8 @@ private void writeToFile(Schema wrapperSchema, List records) thro if (records.size() > 0) { Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); - HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); + final String keyField = table.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header, keyField); writer.appendBlock(block); records.clear(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/RandomFileIdPrefixProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/RandomFileIdPrefixProvider.java index 89d993460e2b..5ad3eedf437c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/RandomFileIdPrefixProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/RandomFileIdPrefixProvider.java @@ -18,13 +18,12 @@ package org.apache.hudi.table; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; -import java.util.Properties; - public class RandomFileIdPrefixProvider extends FileIdPrefixProvider { - public RandomFileIdPrefixProvider(Properties props) { + public RandomFileIdPrefixProvider(TypedProperties props) { super(props); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index a22479b6bf34..221f970cb513 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -57,7 +57,8 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, * @param metadata commit metadata of interest. */ protected final void writeTableMetadata(HoodieCommitMetadata metadata, String actionType) { - table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime, table.isTableServiceAction(actionType))); + table.getMetadataWriter(instantTime).ifPresent(w -> w.update( + metadata, instantTime, table.isTableServiceAction(actionType))); } /** @@ -65,7 +66,7 @@ protected final void writeTableMetadata(HoodieCommitMetadata metadata, String ac * @param metadata clean metadata of interest. */ protected final void writeTableMetadata(HoodieCleanMetadata metadata) { - table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, instantTime)); } /** @@ -73,7 +74,7 @@ protected final void writeTableMetadata(HoodieCleanMetadata metadata) { * @param metadata rollback metadata of interest. */ protected final void writeTableMetadata(HoodieRollbackMetadata metadata) { - table.getMetadataWriter(Option.of(metadata)).ifPresent(w -> w.update(metadata, instantTime)); + table.getMetadataWriter(instantTime, Option.of(metadata)).ifPresent(w -> w.update(metadata, instantTime)); } /** @@ -81,6 +82,6 @@ protected final void writeTableMetadata(HoodieRollbackMetadata metadata) { * @param metadata restore metadata of interest. */ protected final void writeTableMetadata(HoodieRestoreMetadata metadata) { - table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + table.getMetadataWriter(instantTime, Option.of(metadata)).ifPresent(w -> w.update(metadata, instantTime)); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index a445fd3cc090..9813b2b65967 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -205,31 +205,19 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan Option.of(timer.endTimer()), cleanStats ); - writeMetadata(metadata); + if (!skipLocking) { + this.txnManager.beginTransaction(Option.empty(), Option.empty()); + } + writeTableMetadata(metadata); table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant, TimelineMetadataUtils.serializeCleanMetadata(metadata)); LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete"); return metadata; } catch (IOException e) { throw new HoodieIOException("Failed to clean up after commit", e); - } - } - - /** - * Update metadata table if available. Any update to metadata table happens within data table lock. - * @param cleanMetadata instance of {@link HoodieCleanMetadata} to be applied to metadata. - */ - private void writeMetadata(HoodieCleanMetadata cleanMetadata) { - if (config.isMetadataTableEnabled()) { - try { - if (!skipLocking) { - this.txnManager.beginTransaction(Option.empty(), Option.empty()); - } - writeTableMetadata(cleanMetadata); - } finally { - if (!skipLocking) { - this.txnManager.endTransaction(); - } + } finally { + if (!skipLocking) { + this.txnManager.endTransaction(Option.empty()); } } } @@ -242,11 +230,15 @@ public HoodieCleanMetadata execute() { .filterInflightsAndRequested().getInstants().collect(Collectors.toList()); if (pendingCleanInstants.size() > 0) { pendingCleanInstants.forEach(hoodieInstant -> { - LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant); - try { - cleanMetadataList.add(runPendingClean(table, hoodieInstant)); - } catch (Exception e) { - LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e); + if (table.getCleanTimeline().isEmpty(hoodieInstant)) { + table.getActiveTimeline().deleteEmptyInstantIfExists(hoodieInstant); + } else { + LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant); + try { + cleanMetadataList.add(runPendingClean(table, hoodieInstant)); + } catch (Exception e) { + LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e); + } } }); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java index 9b95bd718397..a64cb88454b0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java @@ -72,6 +72,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { try { CleanPlanner planner = new CleanPlanner<>(context, table, config); Option earliestInstant = planner.getEarliestCommitToRetain(); + context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned"); List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); if (partitionsToClean.isEmpty()) { @@ -82,7 +83,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); LOG.info("Using cleanerParallelism: " + cleanerParallelism); - context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned"); + context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned"); Map> cleanOps = context .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 80727ffafb31..27937af880c4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -146,11 +146,15 @@ private List getPartitionPathsForCleanByCommits(Option in if (config.incrementalCleanerModeEnabled()) { Option lastClean = hoodieTable.getCleanTimeline().filterCompletedInstants().lastInstant(); if (lastClean.isPresent()) { - HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils - .deserializeHoodieCleanMetadata(hoodieTable.getActiveTimeline().getInstantDetails(lastClean.get()).get()); - if ((cleanMetadata.getEarliestCommitToRetain() != null) - && (cleanMetadata.getEarliestCommitToRetain().length() > 0)) { - return getPartitionPathsForIncrementalCleaning(cleanMetadata, instantToRetain); + if (hoodieTable.getActiveTimeline().isEmpty(lastClean.get())) { + hoodieTable.getActiveTimeline().deleteEmptyInstantIfExists(lastClean.get()); + } else { + HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils + .deserializeHoodieCleanMetadata(hoodieTable.getActiveTimeline().getInstantDetails(lastClean.get()).get()); + if ((cleanMetadata.getEarliestCommitToRetain() != null) + && (cleanMetadata.getEarliestCommitToRetain().length() > 0)) { + return getPartitionPathsForIncrementalCleaning(cleanMetadata, instantToRetain); + } } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java index 97407e3464d7..a1820ed93b7c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java @@ -27,10 +27,15 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import java.io.IOException; import java.util.Collections; @@ -38,6 +43,8 @@ public abstract class BaseClusteringPlanActionExecutor extends BaseActionExecutor> { + private static final Logger LOG = LogManager.getLogger(BaseClusteringPlanActionExecutor.class); + private final Option> extraMetadata; public BaseClusteringPlanActionExecutor(HoodieEngineContext context, @@ -49,7 +56,33 @@ public BaseClusteringPlanActionExecutor(HoodieEngineContext context, this.extraMetadata = extraMetadata; } - protected abstract Option createClusteringPlan(); + protected Option createClusteringPlan() { + LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); + Option lastClusteringInstant = table.getActiveTimeline().getCompletedReplaceTimeline().lastInstant(); + + int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) + .countInstants(); + if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getInlineClusterMaxCommits()); + return Option.empty(); + } + + if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getAsyncClusterMaxCommits()); + return Option.empty(); + } + + LOG.info("Generating clustering plan for table " + config.getBasePath()); + ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) + ReflectionUtils.loadClass(ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config), table, context, config); + + return strategy.generateClusteringPlan(); + } @Override public Option execute() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java new file mode 100644 index 000000000000..a63eb3badbe9 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.hudi.table.action.cluster; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; + +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Partition filter utilities. Currently, we support three mode: + * NONE: skip filter + * RECENT DAYS: output recent partition given skip num and days lookback config + * SELECTED_PARTITIONS: output partition falls in the [start, end] condition + */ +public class ClusteringPlanPartitionFilter { + + public static List filter(List partitions, HoodieWriteConfig config) { + ClusteringPlanPartitionFilterMode mode = config.getClusteringPlanPartitionFilterMode(); + switch (mode) { + case NONE: + return partitions; + case RECENT_DAYS: + return recentDaysFilter(partitions, config); + case SELECTED_PARTITIONS: + return selectedPartitionsFilter(partitions, config); + default: + throw new HoodieClusteringException("Unknown partition filter, filter mode: " + mode); + } + } + + private static List recentDaysFilter(List partitions, HoodieWriteConfig config) { + int targetPartitionsForClustering = config.getTargetPartitionsForClustering(); + int skipPartitionsFromLatestForClustering = config.getSkipPartitionsFromLatestForClustering(); + return partitions.stream() + .sorted(Comparator.reverseOrder()) + .skip(Math.max(skipPartitionsFromLatestForClustering, 0)) + .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitions.size()) + .collect(Collectors.toList()); + } + + private static List selectedPartitionsFilter(List partitions, HoodieWriteConfig config) { + String beginPartition = config.getBeginPartitionForClustering(); + String endPartition = config.getEndPartitionForClustering(); + List filteredPartitions = partitions.stream() + .filter(path -> path.compareTo(beginPartition) >= 0 && path.compareTo(endPartition) <= 0) + .collect(Collectors.toList()); + return filteredPartitions; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilterMode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilterMode.java new file mode 100644 index 000000000000..fbaf79797f00 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilterMode.java @@ -0,0 +1,28 @@ +/* + * 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 org.apache.hudi.table.action.cluster; + +/** + * Clustering partition filter mode + */ +public enum ClusteringPlanPartitionFilterMode { + NONE, + RECENT_DAYS, + SELECTED_PARTITIONS +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 273ebce761e2..479f63932c5b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -30,8 +30,11 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -56,6 +59,37 @@ public abstract class ClusteringPlanStrategy buildClusteringGroupsForPartiti * Return list of partition paths to be considered for clustering. */ protected List filterPartitionPaths(List partitionPaths) { - return partitionPaths; + List filteredPartitions = ClusteringPlanPartitionFilter.filter(partitionPaths, getWriteConfig()); + LOG.debug("Filtered to the following partitions: " + filteredPartitions); + return filteredPartitions; } @Override @@ -74,13 +78,17 @@ public Option generateClusteringPlan() { return Option.empty(); } - List clusteringGroups = getEngineContext().flatMap(partitionPaths, - partitionPath -> { - List fileSlicesEligible = getFileSlicesEligibleForClustering(partitionPath).collect(Collectors.toList()); - return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible).limit(getWriteConfig().getClusteringMaxNumGroups()); - }, - partitionPaths.size()) - .stream().limit(getWriteConfig().getClusteringMaxNumGroups()).collect(Collectors.toList()); + List clusteringGroups = getEngineContext() + .flatMap( + partitionPaths, + partitionPath -> { + List fileSlicesEligible = getFileSlicesEligibleForClustering(partitionPath).collect(Collectors.toList()); + return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible).limit(getWriteConfig().getClusteringMaxNumGroups()); + }, + partitionPaths.size()) + .stream() + .limit(getWriteConfig().getClusteringMaxNumGroups()) + .collect(Collectors.toList()); if (clusteringGroups.isEmpty()) { LOG.info("No data available to cluster"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java index 3ce4f04e3d03..009790812a62 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.collection.Pair; import java.util.Set; @@ -41,8 +42,8 @@ protected UpdateStrategy(HoodieEngineContext engineContext, Set> handleUpdate(I taggedRecordsRDD); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java index 0d9cdc0aa062..3f241944c3af 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -38,7 +39,7 @@ public HoodieWriteMetadata write(String instantTime, boolean shouldCombine, int shuffleParallelism, BaseCommitActionExecutor executor, - boolean performTagging) { + WriteOperationType operationType) { try { // De-dupe/merge if needed I dedupedRecords = @@ -46,8 +47,9 @@ public HoodieWriteMetadata write(String instantTime, Instant lookupBegin = Instant.now(); I taggedRecords = dedupedRecords; - if (performTagging) { + if (table.getIndex().requiresTagging(operationType)) { // perform index loop up to get existing location of records + context.setJobStatus(this.getClass().getSimpleName(), "Tagging"); taggedRecords = tag(dedupedRecords, context, table); } Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index ce6ed5db303c..7449f3f8045a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -76,6 +76,10 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c // TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link AbstractHoodieWriteClient}. this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); + if (table.getStorageLayout().doesNotSupport(operationType)) { + throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName() + + " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName()); + } } public abstract HoodieWriteMetadata execute(I inputRecords); @@ -147,14 +151,16 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { } protected void autoCommit(Option> extraMetadata, HoodieWriteMetadata result) { - this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)), + final Option inflightInstant = Option.of(new HoodieInstant(State.INFLIGHT, + HoodieTimeline.COMMIT_ACTION, instantTime)); + this.txnManager.beginTransaction(inflightInstant, lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); try { TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner()); commit(extraMetadata, result); } finally { - this.txnManager.endTransaction(); + this.txnManager.endTransaction(inflightInstant); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index 419f88eef4fd..73e1413d9dde 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -179,6 +179,8 @@ public List compact(HoodieCompactionHandler compactionHandler, .withSpillableMapBasePath(config.getSpillableMapBasePath()) .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) + .withOperationField(config.allowOperationMetadataField()) + .withPartition(operation.getPartitionPath()) .build(); if (!scanner.iterator().hasNext()) { scanner.close(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 123f790d36df..2627454fccb6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -68,12 +69,15 @@ public ScheduleCompactionActionExecutor(HoodieEngineContext context, public Option execute() { if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() && !config.getFailedWritesCleanPolicy().isLazy()) { - // if there are inflight writes, their instantTime must not be less than that of compaction instant time - table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() - .ifPresent(earliestInflight -> ValidationUtils.checkArgument( - HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), - "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight - + ", Compaction scheduled at " + instantTime)); + // TODO(yihua): this validation is removed for Java client used by kafka-connect. Need to revisit this. + if (config.getEngineType() != EngineType.JAVA) { + // if there are inflight writes, their instantTime must not be less than that of compaction instant time + table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant() + .ifPresent(earliestInflight -> ValidationUtils.checkArgument( + HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime), + "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight + + ", Compaction scheduled at " + instantTime)); + } // Committed and pending compaction instants should have strictly lower timestamps List conflictingInstants = table.getActiveTimeline() .getWriteTimeline().filterCompletedAndCompactionInstants().getInstants() @@ -114,6 +118,7 @@ private HoodieCompactionPlan scheduleCompaction() { .collect(Collectors.toSet()); // exclude files in pending clustering from compaction. fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); @@ -184,7 +189,7 @@ private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) private Long parsedToSeconds(String time) { long timestamp; try { - timestamp = HoodieActiveTimeline.parseInstantTime(time).getTime() / 1000; + timestamp = HoodieActiveTimeline.parseDateFromInstantTime(time).getTime() / 1000; } catch (ParseException e) { throw new HoodieCompactionException(e.getMessage(), e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java index 747e0b2f3c47..09c19b1aabe8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java @@ -39,14 +39,17 @@ */ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy { - SimpleDateFormat dateFormat = new SimpleDateFormat(DayBasedCompactionStrategy.DATE_PARTITION_FORMAT); + // NOTE: {@code SimpleDataFormat} is NOT thread-safe + // TODO replace w/ DateTimeFormatter + private final ThreadLocal dateFormat = + ThreadLocal.withInitial(() -> new SimpleDateFormat(DayBasedCompactionStrategy.DATE_PARTITION_FORMAT)); @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { // The earliest partition to compact - current day minus the target partitions limit String earliestPartitionPathToCompact = - dateFormat.format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction())); + dateFormat.get().format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction())); // Filter out all partitions greater than earliestPartitionPathToCompact return operations.stream().collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet() @@ -59,7 +62,7 @@ public List orderAndFilter(HoodieWriteConfig writeCon public List filterPartitionPaths(HoodieWriteConfig writeConfig, List partitionPaths) { // The earliest partition to compact - current day minus the target partitions limit String earliestPartitionPathToCompact = - dateFormat.format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction())); + dateFormat.get().format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction())); // Get all partitions and sort them return partitionPaths.stream().map(partition -> partition.replace("/", "-")) .sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/")) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java index 4a12bb8a08b7..94d74b50dc24 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java @@ -41,12 +41,18 @@ public class DayBasedCompactionStrategy extends CompactionStrategy { // For now, use SimpleDateFormat as default partition format protected static final String DATE_PARTITION_FORMAT = "yyyy/MM/dd"; // Sorts compaction in LastInFirstCompacted order + + // NOTE: {@code SimpleDataFormat} is NOT thread-safe + // TODO replace w/ DateTimeFormatter + private static final ThreadLocal DATE_FORMAT = + ThreadLocal.withInitial(() -> new SimpleDateFormat(DATE_PARTITION_FORMAT, Locale.ENGLISH)); + protected static Comparator comparator = (String leftPartition, String rightPartition) -> { try { leftPartition = getPartitionPathWithoutPartitionKeys(leftPartition); rightPartition = getPartitionPathWithoutPartitionKeys(rightPartition); - Date left = new SimpleDateFormat(DATE_PARTITION_FORMAT, Locale.ENGLISH).parse(leftPartition); - Date right = new SimpleDateFormat(DATE_PARTITION_FORMAT, Locale.ENGLISH).parse(rightPartition); + Date left = DATE_FORMAT.get().parse(leftPartition); + Date right = DATE_FORMAT.get().parse(rightPartition); return left.after(right) ? -1 : right.after(left) ? 1 : 0; } catch (ParseException e) { throw new HoodieException("Invalid Partition Date Format", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java index fe4b47459388..c165141dfc5e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -27,7 +27,8 @@ import java.util.stream.Collectors; /** - * LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and limits the + * LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size, + * filters the file group which log files size is greater than the threshold and limits the * compactions within a configured IO bound. * * @see BoundedIOCompactionStrategy @@ -39,8 +40,12 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { + // Filter the file group which log files size is greater than the threshold in bytes. // Order the operations based on the reverse size of the logs and limit them by the IO - return super.orderAndFilter(writeConfig, operations.stream().sorted(this).collect(Collectors.toList()), + long threshold = writeConfig.getCompactionLogFileSizeThreshold(); + return super.orderAndFilter(writeConfig, operations.stream() + .filter(e -> e.getMetrics().getOrDefault(TOTAL_LOG_FILE_SIZE, 0d) >= threshold) + .sorted(this).collect(Collectors.toList()), pendingCompactionPlans); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index ac8f9940d4b3..93713401496d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -112,7 +112,7 @@ private void writeToMetadata(HoodieRestoreMetadata restoreMetadata) { this.txnManager.beginTransaction(Option.empty(), Option.empty()); writeTableMetadata(restoreMetadata); } finally { - this.txnManager.endTransaction(); + this.txnManager.endTransaction(Option.empty()); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java index 1116ef9a4dd8..facab71c6237 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java @@ -48,7 +48,7 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback } table.getMetaClient().reloadActiveTimeline(); String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, newInstantTime, instantToRollback, false); + table.scheduleRollback(context, newInstantTime, instantToRollback, false, false); table.getMetaClient().reloadActiveTimeline(); CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor( context, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java index db6fbc262015..661cee4a2e60 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java @@ -52,7 +52,7 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback } table.getMetaClient().reloadActiveTimeline(); String instantTime = HoodieActiveTimeline.createNewInstantTime(); - table.scheduleRollback(context, instantTime, instantToRollback, false); + table.scheduleRollback(context, instantTime, instantToRollback, false, false); table.getMetaClient().reloadActiveTimeline(); MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor( context, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index ff50a2961eaf..9d5895de83b1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -105,62 +105,45 @@ public BaseRollbackActionExecutor(HoodieEngineContext context, private HoodieRollbackMetadata runRollback(HoodieTable table, HoodieInstant rollbackInstant, HoodieRollbackPlan rollbackPlan) { ValidationUtils.checkArgument(rollbackInstant.getState().equals(HoodieInstant.State.REQUESTED) || rollbackInstant.getState().equals(HoodieInstant.State.INFLIGHT)); - try { - final HoodieInstant inflightInstant; - final HoodieTimer timer = new HoodieTimer(); - timer.startTimer(); - if (rollbackInstant.isRequested()) { - inflightInstant = table.getActiveTimeline().transitionRollbackRequestedToInflight(rollbackInstant, - TimelineMetadataUtils.serializeRollbackPlan(rollbackPlan)); - } else { - inflightInstant = rollbackInstant; - } + final HoodieTimer timer = new HoodieTimer(); + timer.startTimer(); + final HoodieInstant inflightInstant = rollbackInstant.isRequested() + ? table.getActiveTimeline().transitionRollbackRequestedToInflight(rollbackInstant) + : rollbackInstant; - HoodieTimer rollbackTimer = new HoodieTimer().startTimer(); - List stats = doRollbackAndGetStats(rollbackPlan); - HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.convertRollbackMetadata( - instantTime, - Option.of(rollbackTimer.endTimer()), - Collections.singletonList(instantToRollback), - stats); - if (!skipTimelinePublish) { - finishRollback(inflightInstant, rollbackMetadata); - } + HoodieTimer rollbackTimer = new HoodieTimer().startTimer(); + List stats = doRollbackAndGetStats(rollbackPlan); + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.convertRollbackMetadata( + instantTime, + Option.of(rollbackTimer.endTimer()), + Collections.singletonList(instantToRollback), + stats); + if (!skipTimelinePublish) { + finishRollback(inflightInstant, rollbackMetadata); + } - // Finally, remove the markers post rollback. - WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp()) - .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); + // Finally, remove the markers post rollback. + WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp()) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - return rollbackMetadata; - } catch (IOException e) { - throw new HoodieIOException("Failed to rollback commit ", e); - } + return rollbackMetadata; } @Override public HoodieRollbackMetadata execute() { table.getMetaClient().reloadActiveTimeline(); - List rollBackInstants = table.getRollbackTimeline() - .filterInflightsAndRequested().getInstants().collect(Collectors.toList()); - if (rollBackInstants.isEmpty()) { - throw new HoodieRollbackException("No Requested Rollback Instants found to execute rollback "); - } - HoodieInstant rollbackInstant = null; - for (HoodieInstant instant : rollBackInstants) { - if (instantTime.equals(instant.getTimestamp())) { - rollbackInstant = instant; - break; - } + Option rollbackInstant = table.getRollbackTimeline() + .filterInflightsAndRequested() + .filter(instant -> instant.getTimestamp().equals(instantTime)) + .firstInstant(); + if (!rollbackInstant.isPresent()) { + throw new HoodieRollbackException("No pending rollback instants found to execute rollback"); } - if (rollbackInstant != null) { - try { - HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(table.getMetaClient(), rollbackInstant); - return runRollback(table, rollBackInstants.get(0), rollbackPlan); - } catch (IOException e) { - throw new HoodieIOException("Failed to fetch rollback plan to rollback commit " + rollbackInstant.getTimestamp(), e); - } - } else { - throw new HoodieIOException("No inflight rollback instants found for commit time " + instantTime); + try { + HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(table.getMetaClient(), rollbackInstant.get()); + return runRollback(table, rollbackInstant.get(), rollbackPlan); + } catch (IOException e) { + throw new HoodieIOException("Failed to fetch rollback plan for commit " + instantTime, e); } } @@ -255,30 +238,18 @@ protected List executeRollback(HoodieInstant instantToRollba protected void finishRollback(HoodieInstant inflightInstant, HoodieRollbackMetadata rollbackMetadata) throws HoodieIOException { try { - writeToMetadata(rollbackMetadata); + if (!skipLocking) { + this.txnManager.beginTransaction(Option.empty(), Option.empty()); + } + writeTableMetadata(rollbackMetadata); table.getActiveTimeline().transitionRollbackInflightToComplete(inflightInstant, TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata)); LOG.info("Rollback of Commits " + rollbackMetadata.getCommitsRollback() + " is complete"); } catch (IOException e) { throw new HoodieIOException("Error executing rollback at instant " + instantTime, e); - } - } - - /** - * Update metadata table if available. Any update to metadata table happens within data table lock. - * @param rollbackMetadata instance of {@link HoodieRollbackMetadata} to be applied to metadata. - */ - private void writeToMetadata(HoodieRollbackMetadata rollbackMetadata) { - if (config.isMetadataTableEnabled()) { - try { - if (!skipLocking) { - this.txnManager.beginTransaction(Option.empty(), Option.empty()); - } - writeTableMetadata(rollbackMetadata); - } finally { - if (!skipLocking) { - this.txnManager.endTransaction(); - } + } finally { + if (!skipLocking) { + this.txnManager.endTransaction(Option.empty()); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java index 24edde27642c..f95ec5d5c9fe 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java @@ -50,6 +50,7 @@ public class BaseRollbackPlanActionExecutor table, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { + boolean skipTimelinePublish, + boolean shouldRollbackUsingMarkers) { super(context, config, table, instantTime); this.instantToRollback = instantToRollback; this.skipTimelinePublish = skipTimelinePublish; + this.shouldRollbackUsingMarkers = shouldRollbackUsingMarkers; } /** @@ -84,7 +87,7 @@ interface RollbackStrategy extends Serializable { * @return */ private BaseRollbackPlanActionExecutor.RollbackStrategy getRollbackStrategy() { - if (config.shouldRollbackUsingMarkers()) { + if (shouldRollbackUsingMarkers) { return new MarkerBasedRollbackStrategy(table, context, config, instantTime); } else { return new ListingBasedRollbackStrategy(table, context, config, instantTime); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index 266fa39cb986..e6355526e523 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -60,7 +60,7 @@ public List getRollbackRequests(HoodieInstant instantToRo List rollbackRequests = null; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, - table.getMetaClient().getBasePath(), config); + table.getMetaClient().getBasePath()); } else { rollbackRequests = RollbackUtils .generateRollbackRequestsUsingFileListingMOR(instantToRollback, table, context); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index 6ad4e1c986fb..a4b59a88b92c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -18,10 +18,6 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hadoop.fs.FileStatus; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -42,6 +38,10 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; +import org.apache.hadoop.fs.FileStatus; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -56,16 +56,18 @@ public class RollbackUtils { /** * Get Latest version of Rollback plan corresponding to a clean instant. - * @param metaClient Hoodie Table Meta Client + * + * @param metaClient Hoodie Table Meta Client * @param rollbackInstant Instant referring to rollback action * @return Rollback plan corresponding to rollback instant * @throws IOException */ - static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClient, HoodieInstant rollbackInstant) + public static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClient, HoodieInstant rollbackInstant) throws IOException { // TODO: add upgrade step if required. + final HoodieInstant requested = HoodieTimeline.getRollbackRequestedInstant(rollbackInstant); return TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getActiveTimeline().readRollbackInfoAsBytes(rollbackInstant).get(), HoodieRollbackPlan.class); + metaClient.getActiveTimeline().readRollbackInfoAsBytes(requested).get(), HoodieRollbackPlan.class); } static Map generateHeader(String instantToRollback, String rollbackInstantTime) { @@ -106,12 +108,10 @@ static HoodieRollbackStat mergeRollbackStat(HoodieRollbackStat stat1, HoodieRoll * Generate all rollback requests that needs rolling back this action without actually performing rollback for COW table type. * @param engineContext instance of {@link HoodieEngineContext} to use. * @param basePath base path of interest. - * @param config instance of {@link HoodieWriteConfig} to use. * @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected. */ - public static List generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext, - String basePath, HoodieWriteConfig config) { - return FSUtils.getAllPartitionPaths(engineContext, config.getMetadataConfig(), basePath).stream() + public static List generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext, String basePath) { + return FSUtils.getAllPartitionPaths(engineContext, basePath, false, false).stream() .map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction) .collect(Collectors.toList()); } @@ -127,7 +127,7 @@ public static List generateRollbackRequestsByListin public static List generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException { String commit = instantToRollback.getTimestamp(); HoodieWriteConfig config = table.getConfig(); - List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + List partitions = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getBasePath(), false, false); if (partitions.isEmpty()) { return new ArrayList<>(); } @@ -229,8 +229,10 @@ private static List generateAppendRollbackBlocksAct // used to write the new log files. In this case, the commit time for the log file is the compaction requested time. // But the index (global) might store the baseCommit of the base and not the requested, hence get the // baseCommit always by listing the file slice - Map fileIdToBaseCommitTimeForLogMap = table.getSliceView().getLatestFileSlices(partitionPath) - .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)); + // With multi writers, rollbacks could be lazy. and so we need to use getLatestFileSlicesBeforeOrOn() instead of getLatestFileSlices() + Map fileIdToBaseCommitTimeForLogMap = table.getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, rollbackInstant.getTimestamp(), + true).collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)); + return commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> { // Filter out stats without prevCommit since they are all inserts diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java index 7ba6cb6bef1d..4879e0bc60c9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/TimelineServerBasedWriteMarkers.java @@ -19,9 +19,11 @@ package org.apache.hudi.table.marker; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieRemoteException; import org.apache.hudi.table.HoodieTable; @@ -132,18 +134,24 @@ protected Option create(String partitionPath, String dataFileName, IOType Map paramsMap = new HashMap<>(); paramsMap.put(MARKER_DIR_PATH_PARAM, markerDirPath.toString()); - paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); + if (StringUtils.isNullOrEmpty(partitionPath)) { + paramsMap.put(MARKER_NAME_PARAM, markerFileName); + } else { + paramsMap.put(MARKER_NAME_PARAM, partitionPath + "/" + markerFileName); + } + boolean success; try { success = executeRequestToTimelineServer( - CREATE_MARKER_URL, paramsMap, new TypeReference() {}, RequestMethod.POST); + CREATE_MARKER_URL, paramsMap, new TypeReference() { + }, RequestMethod.POST); } catch (IOException e) { throw new HoodieRemoteException("Failed to create marker file " + partitionPath + "/" + markerFileName, e); } LOG.info("[timeline-server-based] Created marker file " + partitionPath + "/" + markerFileName + " in " + timer.endTimer() + " ms"); if (success) { - return Option.of(new Path(new Path(markerDirPath, partitionPath), markerFileName)); + return Option.of(new Path(FSUtils.getPartitionPath(markerDirPath, partitionPath), markerFileName)); } else { return Option.empty(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index c243b9b7ccd8..3dacf1e1302c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -85,9 +84,10 @@ public Option createIfNotExists(String partitionPath, String dataFileName, */ public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) { try { + context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory"); deleteMarkerDir(context, parallelism); - } catch (HoodieIOException ioe) { - LOG.warn("Error deleting marker directory for instant " + instantTime, ioe); + } catch (Exception e) { + LOG.warn("Error deleting marker directory for instant " + instantTime, e); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java index 044b258e8893..dfd55f295812 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java @@ -24,6 +24,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieTable; +import com.esotericsoftware.minlog.Log; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -45,11 +46,17 @@ public static WriteMarkers get(MarkerType markerType, HoodieTable table, String case DIRECT: return new DirectWriteMarkers(table, instantTime); case TIMELINE_SERVER_BASED: + if (!table.getConfig().isEmbeddedTimelineServerEnabled()) { + Log.warn("Timeline-server-based markers are configured as the marker type " + + "but embedded timeline server is not enabled. Falling back to direct markers."); + return new DirectWriteMarkers(table, instantTime); + } String basePath = table.getMetaClient().getBasePath(); if (StorageSchemes.HDFS.getScheme().equals( FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) { - throw new HoodieException("Timeline-server-based markers are not supported for HDFS: " - + "base path " + basePath); + Log.warn("Timeline-server-based markers are not supported for HDFS: " + + "base path " + basePath + ". Falling back to direct markers."); + return new DirectWriteMarkers(table, instantTime); } return new TimelineServerBasedWriteMarkers(table, instantTime); default: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java new file mode 100644 index 000000000000..156da66ff1b3 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java @@ -0,0 +1,177 @@ +/* + * 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 org.apache.hudi.table.repair; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +/** + * Utils for table repair tool. + */ +public final class RepairUtils { + /** + * Tags the instant time of each base or log file from the input file paths. + * + * @param basePath Base path of the table. + * @param baseFileExtension Base file extension, e.g., ".parquet". + * @param allPaths A {@link List} of file paths to tag. + * @return A {@link Map} of instant time in {@link String} to a {@link List} of relative file paths. + */ + public static Map> tagInstantsOfBaseAndLogFiles( + String basePath, String baseFileExtension, List allPaths) { + // Instant time -> Set of base and log file paths + Map> instantToFilesMap = new HashMap<>(); + allPaths.forEach(path -> { + String instantTime = path.toString().endsWith(baseFileExtension) + ? FSUtils.getCommitTime(path.getName()) : FSUtils.getBaseCommitTimeFromLogPath(path); + instantToFilesMap.computeIfAbsent(instantTime, k -> new ArrayList<>()); + instantToFilesMap.get(instantTime).add( + FSUtils.getRelativePartitionPath(new Path(basePath), path)); + }); + return instantToFilesMap; + } + + /** + * Gets the base and log file paths written for a given instant from the timeline. + * This reads the details of the instant metadata. + * + * @param timeline {@link HoodieTimeline} instance, can be active or archived timeline. + * @param instant Instant for lookup. + * @return A {@link Option} of {@link Set} of relative file paths to base path + * if the instant action is supported; empty {@link Option} otherwise. + * @throws IOException if reading instant details fail. + */ + public static Option> getBaseAndLogFilePathsFromTimeline( + HoodieTimeline timeline, HoodieInstant instant) throws IOException { + if (!instant.isCompleted()) { + throw new HoodieException("Cannot get base and log file paths from " + + "instant not completed: " + instant.getTimestamp()); + } + + switch (instant.getAction()) { + case COMMIT_ACTION: + case DELTA_COMMIT_ACTION: + final HoodieCommitMetadata commitMetadata = + HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + return Option.of(commitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream) + .map(HoodieWriteStat::getPath).collect(Collectors.toSet())); + case REPLACE_COMMIT_ACTION: + final HoodieReplaceCommitMetadata replaceCommitMetadata = + HoodieReplaceCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + return Option.of(replaceCommitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream) + .map(HoodieWriteStat::getPath).collect(Collectors.toSet())); + default: + return Option.empty(); + } + } + + /** + * Finds the dangling files to remove for a given instant to repair. + * + * @param instantToRepair Instant timestamp to repair. + * @param baseAndLogFilesFromFs A {@link List} of base and log files based on the file system. + * @param activeTimeline {@link HoodieActiveTimeline} instance. + * @param archivedTimeline {@link HoodieArchivedTimeline} instance. + * @return A {@link List} of relative file paths to base path for removing. + */ + public static List findInstantFilesToRemove( + String instantToRepair, List baseAndLogFilesFromFs, + HoodieActiveTimeline activeTimeline, HoodieArchivedTimeline archivedTimeline) { + // Skips the instant if it is requested or inflight in active timeline + if (activeTimeline.filter(instant -> instant.getTimestamp().equals(instantToRepair) + && !instant.isCompleted()).getInstants().findAny().isPresent()) { + return Collections.emptyList(); + } + + try { + boolean doesInstantExist = false; + Option> filesFromTimeline = Option.empty(); + Option instantOption = activeTimeline.filterCompletedInstants().filter( + instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant(); + if (instantOption.isPresent()) { + // Completed instant in active timeline + doesInstantExist = true; + filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline( + activeTimeline, instantOption.get()); + } else { + instantOption = archivedTimeline.filterCompletedInstants().filter( + instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant(); + if (instantOption.isPresent()) { + // Completed instant in archived timeline + doesInstantExist = true; + filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline( + archivedTimeline, instantOption.get()); + } + } + + if (doesInstantExist) { + if (!filesFromTimeline.isPresent() || filesFromTimeline.get().isEmpty()) { + // Skips if no instant details + return Collections.emptyList(); + } + // Excludes committed base and log files from timeline + Set filesToRemove = new HashSet<>(baseAndLogFilesFromFs); + filesToRemove.removeAll(filesFromTimeline.get()); + return new ArrayList<>(filesToRemove); + } else { + // The instant does not exist in the whole timeline (neither completed nor requested/inflight), + // this means the files from this instant are dangling, which should be removed + return baseAndLogFilesFromFs; + } + } catch (IOException e) { + // In case of failure, does not remove any files for the instant + return Collections.emptyList(); + } + } + + /** + * Serializable path filter class for Spark job. + */ + public interface SerializablePathFilter extends PathFilter, Serializable { + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java new file mode 100644 index 000000000000..6247b870fc57 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.hudi.table.storage; + +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieLayoutConfig; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.util.HashSet; +import java.util.Set; + +/** + * Storage layout when using bucket index. Data distribution and files organization are in a specific way. + */ +public class HoodieBucketLayout extends HoodieStorageLayout { + + public static final Set SUPPORTED_OPERATIONS = new HashSet() {{ + add(WriteOperationType.INSERT); + add(WriteOperationType.INSERT_PREPPED); + add(WriteOperationType.UPSERT); + add(WriteOperationType.UPSERT_PREPPED); + add(WriteOperationType.INSERT_OVERWRITE); + add(WriteOperationType.DELETE); + add(WriteOperationType.COMPACT); + add(WriteOperationType.DELETE_PARTITION); + }}; + + public HoodieBucketLayout(HoodieWriteConfig config) { + super(config); + } + + /** + * Bucketing controls the number of file groups directly. + */ + @Override + public boolean determinesNumFileGroups() { + return true; + } + + public Option layoutPartitionerClass() { + return config.contains(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME) + ? Option.of(config.getString(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME.key())) + : Option.empty(); + } + + @Override + public boolean doesNotSupport(WriteOperationType operationType) { + return !SUPPORTED_OPERATIONS.contains(operationType); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java new file mode 100644 index 000000000000..09d20707a4c8 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java @@ -0,0 +1,45 @@ +/* + * 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 org.apache.hudi.table.storage; + +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +/** + * Default storage layout with non-constraints. + */ +public class HoodieDefaultLayout extends HoodieStorageLayout { + + public HoodieDefaultLayout(HoodieWriteConfig config) { + super(config); + } + + public boolean determinesNumFileGroups() { + return false; + } + + public Option layoutPartitionerClass() { + return Option.empty(); + } + + public boolean doesNotSupport(WriteOperationType operationType) { + return false; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java new file mode 100644 index 000000000000..e86d253df4bf --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java @@ -0,0 +1,38 @@ +/* + * 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 org.apache.hudi.table.storage; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; + +/** + * A factory to generate layout. + */ +public final class HoodieLayoutFactory { + public static HoodieStorageLayout createLayout(HoodieWriteConfig config) { + switch (config.getLayoutType()) { + case DEFAULT: + return new HoodieDefaultLayout(config); + case BUCKET: + return new HoodieBucketLayout(config); + default: + throw new HoodieNotSupportedException("Unknown layout type, set " + config.getLayoutType()); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java new file mode 100644 index 000000000000..a0a4eab46304 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java @@ -0,0 +1,56 @@ +/* + * 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 org.apache.hudi.table.storage; + +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.io.Serializable; + +/** + * Storage layout defines how the files are organized within a table. + */ +public abstract class HoodieStorageLayout implements Serializable { + + protected final HoodieWriteConfig config; + + public HoodieStorageLayout(HoodieWriteConfig config) { + this.config = config; + } + + /** + * By default, layout does not directly control the total number of files. + */ + public abstract boolean determinesNumFileGroups(); + + /** + * Return the layout specific partitioner for writing data, if any. + */ + public abstract Option layoutPartitionerClass(); + + /** + * Determines if the operation is supported by the layout. + */ + public abstract boolean doesNotSupport(WriteOperationType operationType); + + public enum LayoutType { + DEFAULT, BUCKET + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java index dddd5f4ac141..efa0fe472c52 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java @@ -24,7 +24,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import java.util.HashMap; +import java.util.Hashtable; import java.util.Map; /** @@ -36,7 +36,7 @@ public class OneToTwoUpgradeHandler implements UpgradeHandler { public Map upgrade( HoodieWriteConfig config, HoodieEngineContext context, String instantTime, BaseUpgradeDowngradeHelper upgradeDowngradeHelper) { - Map tablePropsToAdd = new HashMap<>(); + Map tablePropsToAdd = new Hashtable<>(); tablePropsToAdd.put(HoodieTableConfig.PARTITION_FIELDS, upgradeDowngradeHelper.getPartitionColumns(config)); tablePropsToAdd.put(HoodieTableConfig.RECORDKEY_FIELDS, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())); tablePropsToAdd.put(HoodieTableConfig.BASE_FILE_FORMAT, config.getString(HoodieTableConfig.BASE_FILE_FORMAT)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java index e1dbfbbe2a51..bff3788d56cf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java @@ -22,10 +22,12 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieTableMetadataUtil; -import java.util.HashMap; +import java.util.Hashtable; import java.util.Map; /** @@ -40,10 +42,14 @@ public Map upgrade(HoodieWriteConfig config, HoodieEngin // table has been updated and is not backward compatible. HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); } - Map tablePropsToAdd = new HashMap<>(); + Map tablePropsToAdd = new Hashtable<>(); tablePropsToAdd.put(HoodieTableConfig.URL_ENCODE_PARTITIONING, config.getStringOrDefault(HoodieTableConfig.URL_ENCODE_PARTITIONING)); tablePropsToAdd.put(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, config.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); - tablePropsToAdd.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, config.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)); + String keyGenClassName = Option.ofNullable(config.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) + .orElse(config.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME)); + ValidationUtils.checkState(keyGenClassName != null, String.format("Missing config: %s or %s", + HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, HoodieWriteConfig.KEYGENERATOR_CLASS_NAME)); + tablePropsToAdd.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGenClassName); return tablePropsToAdd; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java index c5ae043d1781..0e8f752a8f68 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java @@ -23,21 +23,16 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpgradeDowngradeException; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; -import java.util.Date; -import java.util.HashMap; +import java.util.Hashtable; import java.util.Map; -import java.util.Properties; /** * Helper class to assist in upgrading/downgrading Hoodie when there is a version change. @@ -107,69 +102,38 @@ public boolean needsUpgradeOrDowngrade(HoodieTableVersion toVersion) { * @param instantTime current instant time that should not be touched. */ public void run(HoodieTableVersion toVersion, String instantTime) { - try { - // Fetch version from property file and current version - HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); - if (!needsUpgradeOrDowngrade(toVersion)) { - return; - } - - if (fs.exists(updatedPropsFilePath)) { - // this can be left over .updated file from a failed attempt before. Many cases exist here. - // a) We failed while writing the .updated file and it's content is partial (e.g hdfs) - // b) We failed without renaming the file to hoodie.properties. We will re-attempt everything now anyway - // c) rename() is not atomic in cloud stores. so hoodie.properties is fine, but we failed before deleting the .updated file - // All cases, it simply suffices to delete the file and proceed. - LOG.info("Deleting existing .updated file with content :" + FileIOUtils.readAsUTFString(fs.open(updatedPropsFilePath))); - fs.delete(updatedPropsFilePath, false); - } - - // Perform the actual upgrade/downgrade; this has to be idempotent, for now. - LOG.info("Attempting to move table from version " + fromVersion + " to " + toVersion); - Map tableProps = new HashMap<>(); - if (fromVersion.versionCode() < toVersion.versionCode()) { - // upgrade - while (fromVersion.versionCode() < toVersion.versionCode()) { - HoodieTableVersion nextVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() + 1); - tableProps.putAll(upgrade(fromVersion, nextVersion, instantTime)); - fromVersion = nextVersion; - } - } else { - // downgrade - while (fromVersion.versionCode() > toVersion.versionCode()) { - HoodieTableVersion prevVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() - 1); - tableProps.putAll(downgrade(fromVersion, prevVersion, instantTime)); - fromVersion = prevVersion; - } - } + // Fetch version from property file and current version + HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); + if (!needsUpgradeOrDowngrade(toVersion)) { + return; + } - // Write out the current version in hoodie.properties.updated file - for (Map.Entry entry : tableProps.entrySet()) { - metaClient.getTableConfig().setValue(entry.getKey(), entry.getValue()); + // Perform the actual upgrade/downgrade; this has to be idempotent, for now. + LOG.info("Attempting to move table from version " + fromVersion + " to " + toVersion); + Map tableProps = new Hashtable<>(); + if (fromVersion.versionCode() < toVersion.versionCode()) { + // upgrade + while (fromVersion.versionCode() < toVersion.versionCode()) { + HoodieTableVersion nextVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() + 1); + tableProps.putAll(upgrade(fromVersion, nextVersion, instantTime)); + fromVersion = nextVersion; } - metaClient.getTableConfig().setTableVersion(toVersion); - createUpdatedFile(metaClient.getTableConfig().getProps()); - - // because for different fs the fs.rename have different action,such as: - // a) for hdfs : if propsFilePath already exist,fs.rename will not replace propsFilePath, but just return false - // b) for localfs: if propsFilePath already exist,fs.rename will replace propsFilePath, and return ture - // c) for aliyun ossfs: if propsFilePath already exist,will throw FileAlreadyExistsException - // so we should delete the old propsFilePath. also upgrade and downgrade is Idempotent - if (fs.exists(propsFilePath)) { - fs.delete(propsFilePath, false); + } else { + // downgrade + while (fromVersion.versionCode() > toVersion.versionCode()) { + HoodieTableVersion prevVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() - 1); + tableProps.putAll(downgrade(fromVersion, prevVersion, instantTime)); + fromVersion = prevVersion; } - // Rename the .updated file to hoodie.properties. This is atomic in hdfs, but not in cloud stores. - // But as long as this does not leave a partial hoodie.properties file, we are okay. - fs.rename(updatedPropsFilePath, propsFilePath); - } catch (IOException e) { - throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e); } - } - private void createUpdatedFile(Properties props) throws IOException { - try (FSDataOutputStream outputStream = fs.create(updatedPropsFilePath)) { - props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); + // Write out the current version in hoodie.properties.updated file + for (Map.Entry entry : tableProps.entrySet()) { + metaClient.getTableConfig().setValue(entry.getKey(), entry.getValue()); } + metaClient.getTableConfig().setTableVersion(toVersion); + + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } protected Map upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 18815b2e132d..1aebbf6b4c42 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -102,7 +102,7 @@ protected void recreateMarkers(final String commitInstantTime, // generate rollback stats List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { - rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig()); + rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java index 702a84a97b30..97ad050e7240 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java @@ -36,38 +36,37 @@ import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY; /** - * This lock provider is used for testing purposes only. It provides a simple file system based lock using HDFS atomic - * create operation. This lock does not support cleaning/expiring the lock after a failed write hence cannot be used - * in production environments. + * This lock provider is used for testing purposes only. It provides a simple file system based lock + * using filesystem's atomic create operation. This lock does not support cleaning/expiring the lock + * after a failed write. Must not be used in production environments. */ public class FileSystemBasedLockProviderTestClass implements LockProvider, Serializable { - private static final String LOCK_NAME = "acquired"; + private static final String LOCK = "lock"; - private String lockPath; + private final int retryMaxCount; + private final int retryWaitTimeMs; private transient FileSystem fs; + private transient Path lockFile; protected LockConfiguration lockConfiguration; public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) { this.lockConfiguration = lockConfiguration; - this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY); - this.fs = FSUtils.getFs(this.lockPath, configuration); - } - - public void acquireLock() { - try { - fs.create(new Path(lockPath + "/" + LOCK_NAME), false).close(); - } catch (IOException e) { - throw new HoodieIOException("Failed to acquire lock", e); - } + final String lockDirectory = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY); + this.retryWaitTimeMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY); + this.retryMaxCount = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY); + this.lockFile = new Path(lockDirectory + "/" + LOCK); + this.fs = FSUtils.getFs(this.lockFile.toString(), configuration); } @Override public void close() { - try { - fs.delete(new Path(lockPath + "/" + LOCK_NAME), true); - } catch (IOException e) { - throw new HoodieLockException("Unable to release lock", e); + synchronized (LOCK) { + try { + fs.delete(this.lockFile, true); + } catch (IOException e) { + throw new HoodieLockException("Unable to release lock: " + getLock(), e); + } } } @@ -75,39 +74,45 @@ public void close() { public boolean tryLock(long time, TimeUnit unit) { try { int numRetries = 0; - while (fs.exists(new Path(lockPath + "/" + LOCK_NAME)) - && (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY))) { - Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)); - } - synchronized (LOCK_NAME) { - if (fs.exists(new Path(lockPath + "/" + LOCK_NAME))) { - return false; + synchronized (LOCK) { + while (fs.exists(this.lockFile)) { + LOCK.wait(retryWaitTimeMs); + numRetries++; + if (numRetries > retryMaxCount) { + return false; + } } acquireLock(); + return fs.exists(this.lockFile); } - return true; } catch (IOException | InterruptedException e) { - throw new HoodieLockException("Failed to acquire lock", e); + throw new HoodieLockException("Failed to acquire lock: " + getLock(), e); } } @Override public void unlock() { - try { - if (fs.exists(new Path(lockPath + "/" + LOCK_NAME))) { - fs.delete(new Path(lockPath + "/" + LOCK_NAME), true); + synchronized (LOCK) { + try { + if (fs.exists(this.lockFile)) { + fs.delete(this.lockFile, true); + } + } catch (IOException io) { + throw new HoodieIOException("Unable to delete lock " + getLock() + "on disk", io); } - } catch (IOException io) { - throw new HoodieIOException("Unable to delete lock on disk", io); } } @Override public String getLock() { + return this.lockFile.toString(); + } + + private void acquireLock() { try { - return fs.listStatus(new Path(lockPath))[0].getPath().toString(); - } catch (Exception e) { - throw new HoodieLockException("Failed to retrieve lock status from lock path " + lockPath); + fs.create(this.lockFile, false).close(); + } catch (IOException e) { + throw new HoodieIOException("Failed to acquire lock: " + getLock(), e); } } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java new file mode 100644 index 000000000000..9e7472c13db9 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java @@ -0,0 +1,253 @@ +/* + * 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 org.apache.hudi.client.transaction; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.LockConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class TestInProcessLockProvider { + + private static final Logger LOG = LogManager.getLogger(TestInProcessLockProvider.class); + private final Configuration hadoopConfiguration = new Configuration(); + private final LockConfiguration lockConfiguration = new LockConfiguration(new TypedProperties()); + + @Test + public void testLockAcquisition() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testLockReAcquisitionBySameThread() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + assertThrows(HoodieLockException.class, () -> { + inProcessLockProvider.lock(); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testLockReAcquisitionByDifferentThread() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + final AtomicBoolean writer2Completed = new AtomicBoolean(false); + + // Main test thread + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + + // Another writer thread in parallel, should block + // and later acquire the lock once it is released + Thread writer2 = new Thread(new Runnable() { + @Override + public void run() { + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + writer2Completed.set(true); + } + }); + writer2.start(); + + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + + try { + writer2.join(); + } catch (InterruptedException e) { + // + } + Assertions.assertTrue(writer2Completed.get()); + } + + @Test + public void testTryLockAcquisition() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + Assertions.assertTrue(inProcessLockProvider.tryLock()); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testTryLockAcquisitionWithTimeout() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + Assertions.assertTrue(inProcessLockProvider.tryLock(1, TimeUnit.MILLISECONDS)); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testTryLockReAcquisitionBySameThread() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + Assertions.assertTrue(inProcessLockProvider.tryLock()); + assertThrows(HoodieLockException.class, () -> { + inProcessLockProvider.tryLock(1, TimeUnit.MILLISECONDS); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testTryLockReAcquisitionByDifferentThread() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + final AtomicBoolean writer2Completed = new AtomicBoolean(false); + + // Main test thread + Assertions.assertTrue(inProcessLockProvider.tryLock()); + + // Another writer thread + Thread writer2 = new Thread(() -> { + Assertions.assertFalse(inProcessLockProvider.tryLock(100L, TimeUnit.MILLISECONDS)); + writer2Completed.set(true); + }); + writer2.start(); + try { + writer2.join(); + } catch (InterruptedException e) { + // + } + + Assertions.assertTrue(writer2Completed.get()); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testTryLockAcquisitionBeforeTimeOutFromTwoThreads() { + final InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + final int threadCount = 3; + final long awaitMaxTimeoutMs = 2000L; + final CountDownLatch latch = new CountDownLatch(threadCount); + final AtomicBoolean writer1Completed = new AtomicBoolean(false); + final AtomicBoolean writer2Completed = new AtomicBoolean(false); + + // Let writer1 get the lock first, then wait for others + // to join the sync up point. + Thread writer1 = new Thread(() -> { + Assertions.assertTrue(inProcessLockProvider.tryLock()); + latch.countDown(); + try { + latch.await(awaitMaxTimeoutMs, TimeUnit.MILLISECONDS); + // Following sleep is to make sure writer2 attempts + // to try lock and to get bocked on the lock which + // this thread is currently holding. + Thread.sleep(50); + } catch (InterruptedException e) { + // + } + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + writer1Completed.set(true); + }); + writer1.start(); + + // Writer2 will block on trying to acquire the lock + // and will eventually get the lock before the timeout. + Thread writer2 = new Thread(() -> { + latch.countDown(); + Assertions.assertTrue(inProcessLockProvider.tryLock(awaitMaxTimeoutMs, TimeUnit.MILLISECONDS)); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + writer2Completed.set(true); + }); + writer2.start(); + + // Let writer1 and writer2 wait at the sync up + // point to make sure they run in parallel and + // one get blocked by the other. + latch.countDown(); + try { + writer1.join(); + writer2.join(); + } catch (InterruptedException e) { + // + } + + // Make sure both writers actually completed good + Assertions.assertTrue(writer1Completed.get()); + Assertions.assertTrue(writer2Completed.get()); + } + + @Test + public void testLockReleaseByClose() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.close(); + }); + } + + @Test + public void testRedundantUnlock() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + assertDoesNotThrow(() -> { + inProcessLockProvider.lock(); + }); + assertDoesNotThrow(() -> { + inProcessLockProvider.unlock(); + }); + assertThrows(HoodieLockException.class, () -> { + inProcessLockProvider.unlock(); + }); + } + + @Test + public void testUnlockWithoutLock() { + InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration, hadoopConfiguration); + assertThrows(HoodieLockException.class, () -> { + inProcessLockProvider.unlock(); + }); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java new file mode 100644 index 000000000000..a1a7f6a3122d --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java @@ -0,0 +1,214 @@ +/* + * 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 org.apache.hudi.client.transaction; + +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieLockException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class TestTransactionManager extends HoodieCommonTestHarness { + HoodieWriteConfig writeConfig; + TransactionManager transactionManager; + + @BeforeEach + private void init() throws IOException { + initPath(); + initMetaClient(); + this.writeConfig = getWriteConfig(); + this.transactionManager = new TransactionManager(this.writeConfig, this.metaClient.getFs()); + } + + private HoodieWriteConfig getWriteConfig() { + return HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(InProcessLockProvider.class) + .build()) + .build(); + } + + @Test + public void testSingleWriterTransaction() { + transactionManager.beginTransaction(); + transactionManager.endTransaction(); + } + + @Test + public void testSingleWriterNestedTransaction() { + transactionManager.beginTransaction(); + assertThrows(HoodieLockException.class, () -> { + transactionManager.beginTransaction(); + }); + + transactionManager.endTransaction(); + assertThrows(HoodieLockException.class, () -> { + transactionManager.endTransaction(); + }); + } + + @Test + public void testMultiWriterTransactions() { + final int threadCount = 3; + final long awaitMaxTimeoutMs = 2000L; + final CountDownLatch latch = new CountDownLatch(threadCount); + final AtomicBoolean writer1Completed = new AtomicBoolean(false); + final AtomicBoolean writer2Completed = new AtomicBoolean(false); + + // Let writer1 get the lock first, then wait for others + // to join the sync up point. + Thread writer1 = new Thread(() -> { + assertDoesNotThrow(() -> { + transactionManager.beginTransaction(); + }); + latch.countDown(); + try { + latch.await(awaitMaxTimeoutMs, TimeUnit.MILLISECONDS); + // Following sleep is to make sure writer2 attempts + // to try lock and to get blocked on the lock which + // this thread is currently holding. + Thread.sleep(50); + } catch (InterruptedException e) { + // + } + assertDoesNotThrow(() -> { + transactionManager.endTransaction(); + }); + writer1Completed.set(true); + }); + writer1.start(); + + // Writer2 will block on trying to acquire the lock + // and will eventually get the lock before the timeout. + Thread writer2 = new Thread(() -> { + latch.countDown(); + try { + latch.await(awaitMaxTimeoutMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // + } + assertDoesNotThrow(() -> { + transactionManager.beginTransaction(); + }); + assertDoesNotThrow(() -> { + transactionManager.endTransaction(); + }); + writer2Completed.set(true); + }); + writer2.start(); + + // Let writer1 and writer2 wait at the sync up + // point to make sure they run in parallel and + // one get blocked by the other. + latch.countDown(); + try { + writer1.join(); + writer2.join(); + } catch (InterruptedException e) { + // + } + + // Make sure both writers actually completed good + Assertions.assertTrue(writer1Completed.get()); + Assertions.assertTrue(writer2Completed.get()); + } + + @Test + public void testTransactionsWithInstantTime() { + // 1. Begin and end by the same transaction owner + Option lastCompletedInstant = getInstant("0000001"); + Option newTxnOwnerInstant = getInstant("0000002"); + transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); + Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); + transactionManager.endTransaction(newTxnOwnerInstant); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + + // 2. Begin transaction with a new txn owner, but end transaction with no/wrong owner + lastCompletedInstant = getInstant("0000002"); + newTxnOwnerInstant = getInstant("0000003"); + transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.endTransaction(); + // Owner reset would not happen as the end txn was invoked with an incorrect current txn owner + Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); + Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); + + // 3. But, we should be able to begin a new transaction for a new owner + lastCompletedInstant = getInstant("0000003"); + newTxnOwnerInstant = getInstant("0000004"); + transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); + Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); + transactionManager.endTransaction(newTxnOwnerInstant); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + + // 4. Transactions with no owners should also go through + transactionManager.beginTransaction(); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + transactionManager.endTransaction(); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + + // 5. Transactions with new instants but with same timestamps should properly reset owners + transactionManager.beginTransaction(getInstant("0000005"), Option.empty()); + Assertions.assertTrue(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + transactionManager.endTransaction(getInstant("0000005")); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + + // 6. Transactions with no owners should also go through + transactionManager.beginTransaction(Option.empty(), Option.empty()); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + transactionManager.endTransaction(Option.empty()); + Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); + Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); + } + + private Option getInstant(String timestamp) { + return Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, timestamp)); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java index ed6b9e6cc535..59b7a9274c94 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java @@ -19,9 +19,10 @@ package org.apache.hudi.config; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.config.HoodieWriteConfig.Builder; - import org.apache.hudi.index.HoodieIndex; + import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -33,6 +34,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -68,17 +70,38 @@ public void testPropertyLoading(boolean withAlternative) throws IOException { @Test public void testDefaultIndexAccordingToEngineType() { - // default bloom - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build(); - assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType()); + testEngineSpecificConfig(HoodieWriteConfig::getIndexType, + constructConfigMap( + EngineType.SPARK, HoodieIndex.IndexType.BLOOM, + EngineType.FLINK, HoodieIndex.IndexType.INMEMORY, + EngineType.JAVA, HoodieIndex.IndexType.INMEMORY)); + } - // spark default bloom - writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.SPARK).withPath("/tmp").build(); - assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType()); + @Test + public void testDefaultClusteringPlanStrategyClassAccordingToEngineType() { + testEngineSpecificConfig(HoodieWriteConfig::getClusteringPlanStrategyClass, + constructConfigMap( + EngineType.SPARK, HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY, + EngineType.FLINK, HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY, + EngineType.JAVA, HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY)); + } - // flink default in-memory - writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath("/tmp").build(); - assertEquals(HoodieIndex.IndexType.INMEMORY, writeConfig.getIndexType()); + @Test + public void testDefaultClusteringExecutionStrategyClassAccordingToEngineType() { + testEngineSpecificConfig(HoodieWriteConfig::getClusteringExecutionStrategyClass, + constructConfigMap( + EngineType.SPARK, HoodieClusteringConfig.SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY, + EngineType.FLINK, HoodieClusteringConfig.JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY, + EngineType.JAVA, HoodieClusteringConfig.JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY)); + } + + @Test + public void testDefaultMarkersTypeAccordingToEngineType() { + testEngineSpecificConfig(HoodieWriteConfig::getMarkersType, + constructConfigMap( + EngineType.SPARK, MarkerType.TIMELINE_SERVER_BASED, + EngineType.FLINK, MarkerType.DIRECT, + EngineType.JAVA, MarkerType.DIRECT)); } private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) throws IOException { @@ -88,4 +111,44 @@ private ByteArrayOutputStream saveParamsIntoOutputStream(Map par properties.store(outStream, "Saved on " + new Date(System.currentTimeMillis())); return outStream; } + + /** + * Tests the engine-specific configuration values for one configuration key . + * + * @param getConfigFunc Function to get the config value. + * @param expectedConfigMap Expected config map, with key as the engine type + * and value as the corresponding config value for the engine. + */ + private void testEngineSpecificConfig(Function getConfigFunc, + Map expectedConfigMap) { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build(); + assertEquals(expectedConfigMap.get(EngineType.SPARK), getConfigFunc.apply(writeConfig)); + + for (EngineType engineType : expectedConfigMap.keySet()) { + writeConfig = HoodieWriteConfig.newBuilder() + .withEngineType(engineType).withPath("/tmp").build(); + assertEquals(expectedConfigMap.get(engineType), getConfigFunc.apply(writeConfig)); + } + } + + /** + * Constructs the map. + * + * @param k1 First engine type. + * @param v1 Config value for the first engine type. + * @param k2 Second engine type. + * @param v2 Config value for the second engine type. + * @param k3 Third engine type. + * @param v3 Config value for the third engine type. + * @return {@link Map} instance, with key as the engine type + * and value as the corresponding config value for the engine. + */ + private Map constructConfigMap( + EngineType k1, Object v1, EngineType k2, Object v2, EngineType k3, Object v3) { + Map mapping = new HashMap<>(); + mapping.put(k1, v1); + mapping.put(k2, v2); + mapping.put(k3, v3); + return mapping; + } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index 04920635684a..86a0886de664 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -22,6 +22,9 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -34,23 +37,35 @@ import org.apache.hadoop.hbase.util.Pair; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mockito; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Stream; import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.apache.hudi.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1; +import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION; +import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; +import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.when; public class TestHoodieHFileReaderWriter { @TempDir File tempDir; @@ -69,21 +84,34 @@ public void clearTempFile() { } } - private HoodieHFileWriter createHFileWriter(Schema avroSchema) throws Exception { + private static Stream populateMetaFieldsAndTestAvroWithMeta() { + return Arrays.stream(new Boolean[][] { + {true, true}, + {false, true}, + {true, false}, + {false, false} + }).map(Arguments::of); + } + + private HoodieHFileWriter createHFileWriter(Schema avroSchema, boolean populateMetaFields) throws Exception { BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name()); Configuration conf = new Configuration(); TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class); + Supplier partitionSupplier = Mockito.mock(Supplier.class); + when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier); + when(partitionSupplier.get()).thenReturn(10); String instantTime = "000"; HoodieHFileConfig hoodieHFileConfig = new HoodieHFileConfig(conf, Compression.Algorithm.GZ, 1024 * 1024, 120 * 1024 * 1024, - filter); - return new HoodieHFileWriter(instantTime, filePath, hoodieHFileConfig, avroSchema, mockTaskContextSupplier); + PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); + return new HoodieHFileWriter(instantTime, filePath, hoodieHFileConfig, avroSchema, mockTaskContextSupplier, populateMetaFields); } - @Test - public void testWriteReadHFile() throws Exception { - Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchema.avsc"); - HoodieHFileWriter writer = createHFileWriter(avroSchema); + @ParameterizedTest + @MethodSource("populateMetaFieldsAndTestAvroWithMeta") + public void testWriteReadHFile(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception { + Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc"); + HoodieHFileWriter writer = createHFileWriter(avroSchema, populateMetaFields); List keys = new ArrayList<>(); Map recordMap = new HashMap<>(); for (int i = 0; i < 100; i++) { @@ -93,7 +121,13 @@ public void testWriteReadHFile() throws Exception { keys.add(key); record.put("time", Integer.toString(RANDOM.nextInt())); record.put("number", i); - writer.writeAvro(key, record); + if (testAvroWithMeta) { + writer.writeAvroWithMetadata(record, new HoodieRecord(new HoodieKey((String) record.get("_row_key"), + Integer.toString((Integer) record.get("number"))), new EmptyHoodieRecordPayload())); // payload does not matter. GenericRecord passed in is what matters + // only HoodieKey will be looked up from the 2nd arg(HoodieRecord). + } else { + writer.writeAvro(key, record); + } recordMap.put(key, record); } writer.close(); @@ -105,8 +139,8 @@ public void testWriteReadHFile() throws Exception { records.forEach(entry -> assertEquals(entry.getSecond(), recordMap.get(entry.getFirst()))); hoodieHFileReader.close(); - for (int i = 0; i < 20; i++) { - int randomRowstoFetch = 5 + RANDOM.nextInt(50); + for (int i = 0; i < 2; i++) { + int randomRowstoFetch = 5 + RANDOM.nextInt(10); Set rowsToFetch = getRandomKeys(randomRowstoFetch, keys); List rowsList = new ArrayList<>(rowsToFetch); Collections.sort(rowsList); @@ -115,6 +149,11 @@ public void testWriteReadHFile() throws Exception { assertEquals(result.size(), randomRowstoFetch); result.forEach(entry -> { assertEquals(entry.getSecond(), recordMap.get(entry.getFirst())); + if (populateMetaFields && testAvroWithMeta) { + assertNotNull(entry.getSecond().get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); + } else { + assertNull(entry.getSecond().get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); + } }); hoodieHFileReader.close(); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java index 317f15230c3c..edd2302a6ecc 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java @@ -19,6 +19,7 @@ package org.apache.hudi.metrics; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.config.HoodieWriteConfig; import com.codahale.metrics.MetricRegistry; @@ -57,7 +58,7 @@ public void metricsReporterFactoryShouldReturnReporter() { public void metricsReporterFactoryShouldReturnUserDefinedReporter() { when(config.getMetricReporterClassName()).thenReturn(DummyMetricsReporter.class.getName()); - Properties props = new Properties(); + TypedProperties props = new TypedProperties(); props.setProperty("testKey", "testValue"); when(config.getProps()).thenReturn(props); @@ -70,7 +71,7 @@ public void metricsReporterFactoryShouldReturnUserDefinedReporter() { @Test public void metricsReporterFactoryShouldThrowExceptionWhenMetricsReporterClassIsIllegal() { when(config.getMetricReporterClassName()).thenReturn(IllegalTestMetricsReporter.class.getName()); - when(config.getProps()).thenReturn(new Properties()); + when(config.getProps()).thenReturn(new TypedProperties()); assertThrows(HoodieException.class, () -> MetricsReporterFactory.createReporter(config, registry)); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/cloudwatch/TestCloudWatchMetricsReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/cloudwatch/TestCloudWatchMetricsReporter.java new file mode 100644 index 000000000000..7901d8024651 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/cloudwatch/TestCloudWatchMetricsReporter.java @@ -0,0 +1,62 @@ +/* + * 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 org.apache.hudi.metrics.cloudwatch; + +import org.apache.hudi.aws.cloudwatch.CloudWatchReporter; +import org.apache.hudi.config.HoodieWriteConfig; + +import com.codahale.metrics.MetricRegistry; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.concurrent.TimeUnit; + +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class TestCloudWatchMetricsReporter { + + @Mock + private HoodieWriteConfig config; + + @Mock + private MetricRegistry registry; + + @Mock + private CloudWatchReporter reporter; + + @Test + public void testReporter() { + when(config.getCloudWatchReportPeriodSeconds()).thenReturn(30); + CloudWatchMetricsReporter metricsReporter = new CloudWatchMetricsReporter(config, registry, reporter); + + metricsReporter.start(); + verify(reporter, times(1)).start(30, TimeUnit.SECONDS); + + metricsReporter.report(); + verify(reporter, times(1)).report(); + + metricsReporter.stop(); + verify(reporter, times(1)).stop(); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestHilbertCurveUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestHilbertCurveUtils.java new file mode 100644 index 000000000000..5bb482e6d67f --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestHilbertCurveUtils.java @@ -0,0 +1,38 @@ +/* + * 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 org.apache.hudi.optimize; + +import org.davidmoten.hilbert.HilbertCurve; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestHilbertCurveUtils { + + private static final HilbertCurve INSTANCE = HilbertCurve.bits(5).dimensions(2); + + @Test + public void testIndex() { + long[] t = {1, 2}; + assertEquals(13, INSTANCE.index(t).intValue()); + long[] t1 = {0, 16}; + assertEquals(256, INSTANCE.index(t1).intValue()); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/cluster/strategy/TestClusteringPlanStrategyConfigCompatibility.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/cluster/strategy/TestClusteringPlanStrategyConfigCompatibility.java new file mode 100644 index 000000000000..34626a897dad --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/cluster/strategy/TestClusteringPlanStrategyConfigCompatibility.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.hudi.table.cluster.strategy; + +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; +import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.stream.Stream; + +public class TestClusteringPlanStrategyConfigCompatibility { + + private static Stream configParams() { + /** + * (user specified class, converted class, filter mode) + */ + Object[][] data = new Object[][] { + {"org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy", + "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy", + ClusteringPlanPartitionFilterMode.RECENT_DAYS}, + {"org.apache.hudi.client.clustering.plan.strategy.SparkSelectedPartitionsClusteringPlanStrategy", + "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy", + ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS}, + {"org.apache.hudi.client.clustering.plan.strategy.JavaRecentDaysClusteringPlanStrategy", + "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy", + ClusteringPlanPartitionFilterMode.RECENT_DAYS} + }; + return Stream.of(data).map(Arguments::of); + } + + @ParameterizedTest() + @MethodSource("configParams") + public void testCheckAndGetClusteringPlanStrategy(String oldClass, String newClass, ClusteringPlanPartitionFilterMode mode) { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withPath("") + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringPlanStrategyClass(oldClass) + .build()) + .build(); + + Assertions.assertEquals(newClass, ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config)); + Assertions.assertEquals(mode, config.getClusteringPlanPartitionFilterMode()); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java new file mode 100644 index 000000000000..21c0e8108a53 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java @@ -0,0 +1,116 @@ +/* + * 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 org.apache.hudi.table.marker; + +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; + +public class TestWriteMarkersFactory extends HoodieCommonTestHarness { + private static final String NON_HDFS_BASE_PATH = "/tmp/dir"; + private static final String HDFS_BASE_PATH = "hdfs://localhost/dir"; + private final HoodieWriteConfig writeConfig = Mockito.mock(HoodieWriteConfig.class); + private final HoodieTableMetaClient metaClient = Mockito.mock(HoodieTableMetaClient.class); + private final HoodieWrapperFileSystem fileSystem = Mockito.mock(HoodieWrapperFileSystem.class); + private final HoodieEngineContext context = Mockito.mock(HoodieEngineContext.class); + private final HoodieTable table = Mockito.mock(HoodieTable.class); + + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + public static Stream configParams() { + Object[][] data = new Object[][] { + {NON_HDFS_BASE_PATH, true}, {HDFS_BASE_PATH, false}, + {NON_HDFS_BASE_PATH, true}, {HDFS_BASE_PATH, false}, + }; + return Stream.of(data).map(Arguments::of); + } + + @ParameterizedTest + @MethodSource("configParams") + public void testDirectMarkers(String basePath, boolean isTimelineServerEnabled) { + testWriteMarkersFactory( + MarkerType.DIRECT, basePath, isTimelineServerEnabled, DirectWriteMarkers.class); + } + + @Test + public void testTimelineServerBasedMarkersWithTimelineServerEnabled() { + testWriteMarkersFactory( + MarkerType.TIMELINE_SERVER_BASED, NON_HDFS_BASE_PATH, true, + TimelineServerBasedWriteMarkers.class); + } + + @Test + public void testTimelineServerBasedMarkersWithTimelineServerDisabled() { + // Fallback to direct markers should happen + testWriteMarkersFactory( + MarkerType.TIMELINE_SERVER_BASED, NON_HDFS_BASE_PATH, false, + DirectWriteMarkers.class); + } + + @Test + public void testTimelineServerBasedMarkersWithHDFS() { + // Fallback to direct markers should happen + testWriteMarkersFactory( + MarkerType.TIMELINE_SERVER_BASED, HDFS_BASE_PATH, true, + DirectWriteMarkers.class); + } + + private void testWriteMarkersFactory( + MarkerType markerTypeConfig, String basePath, boolean isTimelineServerEnabled, + Class expectedWriteMarkersClass) { + String instantTime = "001"; + Mockito.when(table.getConfig()).thenReturn(writeConfig); + Mockito.when(writeConfig.isEmbeddedTimelineServerEnabled()) + .thenReturn(isTimelineServerEnabled); + Mockito.when(table.getMetaClient()).thenReturn(metaClient); + Mockito.when(metaClient.getFs()).thenReturn(fileSystem); + Mockito.when(metaClient.getBasePath()).thenReturn(basePath); + Mockito.when(metaClient.getMarkerFolderPath(any())).thenReturn(basePath + ".hoodie/.temp"); + Mockito.when(table.getContext()).thenReturn(context); + Mockito.when(context.getHadoopConf()).thenReturn(new SerializableConfiguration(new Configuration())); + Mockito.when(writeConfig.getViewStorageConfig()) + .thenReturn(FileSystemViewStorageConfig.newBuilder().build()); + assertEquals(expectedWriteMarkersClass, + WriteMarkersFactory.get(markerTypeConfig, table, instantTime).getClass()); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java new file mode 100644 index 000000000000..35928dc7cf31 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.hudi.table.upgrade; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.keygen.KeyGenerator; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TestTwoToThreeUpgradeHandler { + + HoodieWriteConfig config; + + @BeforeEach + void setUp() { + config = HoodieWriteConfig.newBuilder() + .forTable("foo") + .withPath("/foo") + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); + } + + @ParameterizedTest + @ValueSource(strings = {"hoodie.table.keygenerator.class", "hoodie.datasource.write.keygenerator.class"}) + void upgradeHandlerShouldRetrieveKeyGeneratorConfig(String keyGenConfigKey) { + config.setValue(keyGenConfigKey, KeyGenerator.class.getName()); + TwoToThreeUpgradeHandler handler = new TwoToThreeUpgradeHandler(); + Map kv = handler.upgrade(config, null, null, null); + assertEquals(KeyGenerator.class.getName(), kv.get(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)); + } + + @Test + void upgradeHandlerShouldThrowWhenKeyGeneratorNotSet() { + TwoToThreeUpgradeHandler handler = new TwoToThreeUpgradeHandler(); + Throwable t = assertThrows(IllegalStateException.class, () -> handler + .upgrade(config, null, null, null)); + assertTrue(t.getMessage().startsWith("Missing config:")); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java index 0c4c7712ac53..32fd200145e9 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java @@ -33,8 +33,9 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.testutils.FileCreateUtils; -import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieOrcConfig; @@ -47,6 +48,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; @@ -56,7 +58,7 @@ import java.io.IOException; import java.nio.file.Paths; -import java.util.Arrays; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -64,15 +66,21 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; -public class HoodieWriteableTestTable extends HoodieTestTable { +public class HoodieWriteableTestTable extends HoodieMetadataTestTable { private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class); protected final Schema schema; protected final BloomFilter filter; protected final boolean populateMetaFields; - protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { - super(basePath, fs, metaClient); + protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, + Schema schema, BloomFilter filter) { + this(basePath, fs, metaClient, schema, filter, null); + } + + protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, + BloomFilter filter, HoodieTableMetadataWriter metadataWriter) { + super(basePath, fs, metaClient, metadataWriter); this.schema = schema; this.filter = filter; this.populateMetaFields = metaClient.getTableConfig().populateMetaFields(); @@ -92,6 +100,12 @@ public HoodieWriteableTestTable withInserts(String partition, String fileId, Lis FileCreateUtils.createPartitionMetaFile(basePath, partition); String fileName = baseFileName(currentInstantTime, fileId); + Path baseFilePath = new Path(Paths.get(basePath, partition, fileName).toString()); + if (this.fs.exists(baseFilePath)) { + LOG.warn("Deleting the existing base file " + baseFilePath); + this.fs.delete(baseFilePath, true); + } + if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.PARQUET)) { HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( new AvroSchemaConverter().convert(schema), schema, Option.of(filter)); @@ -139,19 +153,18 @@ public HoodieWriteableTestTable withInserts(String partition, String fileId, Lis return this; } - public HoodieWriteableTestTable withLogAppends(HoodieRecord... records) throws Exception { - return withLogAppends(Arrays.asList(records)); - } - - public HoodieWriteableTestTable withLogAppends(List records) throws Exception { - for (List groupedRecords: records.stream() + public Map> withLogAppends(List records) throws Exception { + Map> partitionToLogfilesMap = new HashMap<>(); + for (List groupedRecords : records.stream() .collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) { - appendRecordsToLogFile(groupedRecords); + final Pair appendedLogFile = appendRecordsToLogFile(groupedRecords); + partitionToLogfilesMap.computeIfAbsent( + appendedLogFile.getKey(), k -> new ArrayList<>()).add(appendedLogFile.getValue()); } - return this; + return partitionToLogfilesMap; } - private void appendRecordsToLogFile(List groupedRecords) throws Exception { + private Pair appendRecordsToLogFile(List groupedRecords) throws Exception { String partitionPath = groupedRecords.get(0).getPartitionPath(); HoodieRecordLocation location = groupedRecords.get(0).getCurrentLocation(); try (HoodieLogFormat.Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) @@ -170,6 +183,7 @@ private void appendRecordsToLogFile(List groupedRecords) throws Ex return null; } }).collect(Collectors.toList()), header)); + return Pair.of(partitionPath, logWriter.getLogFile()); } } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestTransactionUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestTransactionUtils.java deleted file mode 100644 index 6a836974312f..000000000000 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestTransactionUtils.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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 org.apache.hudi.utils; - -import org.apache.hadoop.fs.Path; -import org.apache.hudi.client.utils.TransactionUtils; -import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.testutils.HoodieCommonTestHarness; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieWriteConfig; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; - -public class TestTransactionUtils extends HoodieCommonTestHarness { - - @BeforeEach - public void setUp() throws Exception { - init(); - } - - public void init() throws Exception { - initPath(); - initMetaClient(); - metaClient.getFs().mkdirs(new Path(basePath)); - } - - @Test - public void testCheckpointStateMerge() throws IOException { - HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient); - - // Create completed commit with deltastreamer checkpoint state - HoodieInstant commitInstantWithCheckpointState = new HoodieInstant( - true, - HoodieTimeline.COMMIT_ACTION, - HoodieActiveTimeline.createNewInstantTime() - ); - timeline.createNewInstant(commitInstantWithCheckpointState); - - HoodieCommitMetadata metadataWithCheckpoint = new HoodieCommitMetadata(); - String checkpointVal = "00001"; - metadataWithCheckpoint.addMetadata(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY, checkpointVal); - timeline.saveAsComplete( - commitInstantWithCheckpointState, - Option.of(metadataWithCheckpoint.toJsonString().getBytes(StandardCharsets.UTF_8)) - ); - - // Inflight commit without checkpoint metadata - HoodieInstant commitInstantWithoutCheckpointState = new HoodieInstant( - true, - HoodieTimeline.COMMIT_ACTION, - HoodieActiveTimeline.createNewInstantTime() - ); - timeline.createNewInstant(commitInstantWithoutCheckpointState); - HoodieCommitMetadata metadataWithoutCheckpoint = new HoodieCommitMetadata(); - - // Ensure that checkpoint state is merged in from previous completed commit - MockTransactionUtils.assertCheckpointStateWasMerged(metaClient, metadataWithoutCheckpoint, checkpointVal); - } - - private static class MockTransactionUtils extends TransactionUtils { - - public static void assertCheckpointStateWasMerged( - HoodieTableMetaClient metaClient, - HoodieCommitMetadata currentMetadata, - String expectedCheckpointState) { - TransactionUtils.mergeCheckpointStateFromPreviousCommit(metaClient, Option.of(currentMetadata)); - assertEquals( - expectedCheckpointState, - currentMetadata.getExtraMetadata().get(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY) - ); - } - } -} diff --git a/hudi-client/hudi-client-common/src/test/resources/exampleSchemaWithMetaFields.avsc b/hudi-client/hudi-client-common/src/test/resources/exampleSchemaWithMetaFields.avsc new file mode 100644 index 000000000000..c3fa82207a0c --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/resources/exampleSchemaWithMetaFields.avsc @@ -0,0 +1,66 @@ +/* + * 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. + */ +{ + "namespace": "example.schema", + "type": "record", + "name": "trip", + "fields": [ + { + "name": "_hoodie_commit_time", + "type": ["null","string"], + "default":null + }, + { + "name": "_hoodie_commit_seqno", + "type": ["null","string"], + "default":null + }, + { + "name": "_hoodie_record_key", + "type": ["null","string"], + "default":null + }, + { + "name": "_hoodie_partition_path", + "type": ["null","string"], + "default":null + }, + { + "name": "_hoodie_file_name", + "type": ["null","string"], + "default":null + }, + { + "name": "_hoodie_operation", + "type": ["null","string"], + "default":null + }, + { + "name": "_row_key", + "type": "string" + }, + { + "name": "time", + "type": "string" + }, + { + "name": "number", + "type": ["int", "null"] + } + ] +} diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index 8cfd5bb0c5fc..b2cc6949bb08 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-flink-client - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-flink-client jar diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 3d44a2432f36..4108ba425e8c 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -40,7 +40,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.FlinkHoodieIndexFactory; import org.apache.hudi.index.HoodieIndex; @@ -57,7 +56,6 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.HoodieTimelineArchiveLog; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -258,10 +256,10 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp } @Override - protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) { + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { this.metadataWriterOption.ifPresent(w -> { w.initTableMetadata(); // refresh the timeline - w.update(metadata, inflightInstant.getTimestamp(), getHoodieTable().isTableServiceAction(inflightInstant.getAction())); + w.update(metadata, instantTime, getHoodieTable().isTableServiceAction(actionType)); }); } @@ -283,7 +281,11 @@ public void initMetadataWriter() { * checkpoint finish. */ public void startAsyncCleaning() { - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); + if (this.asyncCleanerService == null) { + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); + } else { + this.asyncCleanerService.start(null); + } } /** @@ -332,11 +334,10 @@ protected void postCommit(HoodieTable>, List, // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), createTable(config, hadoopConf), instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - // We cannot have unbounded commit files. Archive commits if we have to archive - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); - archiveLog.archiveIfRequired(context); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); + if (config.isAutoArchive()) { + // We cannot have unbounded commit files. Archive commits if we have to archive + archive(table); + } } finally { this.heartbeatClient.stop(instantTime); } @@ -362,16 +363,24 @@ public void completeCompaction( String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()); - writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime)); - // commit to data table after committing to metadata table. - finalizeWrite(table, compactionCommitTime, writeStats); - LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); - CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); - + final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime); + try { + this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); + finalizeWrite(table, compactionCommitTime, writeStats); + // commit to data table after committing to metadata table. + // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a + // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. + table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent( + w -> w.update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); + LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); + CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); + } finally { + this.txnManager.endTransaction(Option.of(compactionInstant)); + } if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); try { - metrics.updateCommitMetrics(HoodieActiveTimeline.parseInstantTime(compactionCommitTime).getTime(), + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(compactionCommitTime).getTime(), durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); } catch (ParseException e) { throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " @@ -399,19 +408,6 @@ public HoodieWriteMetadata> cluster(final String clusteringIns throw new HoodieNotSupportedException("Clustering is not supported yet"); } - private void writeTableMetadata(HoodieTable>, List, List> table, - HoodieCommitMetadata commitMetadata, - HoodieInstant hoodieInstant) { - try { - this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty()); - // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a - // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), table.isTableServiceAction(hoodieInstant.getAction()))); - } finally { - this.txnManager.endTransaction(); - } - } - @Override protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java new file mode 100644 index 000000000000..46eff587575c --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java @@ -0,0 +1,65 @@ +/* + * 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 org.apache.hudi.execution; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.io.HoodieWriteHandle; + +import java.util.ArrayList; +import java.util.List; + +/** + * Consumes stream of hoodie records from in-memory queue and writes to one explicit create handle. + */ +public class ExplicitWriteHandler + extends BoundedInMemoryQueueConsumer, List> { + + private final List statuses = new ArrayList<>(); + + private HoodieWriteHandle handle; + + public ExplicitWriteHandler(HoodieWriteHandle handle) { + this.handle = handle; + } + + @Override + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { + final HoodieRecord insertPayload = payload.record; + handle.write(insertPayload, payload.insertValue, payload.exception); + } + + @Override + public void finish() { + closeOpenHandle(); + assert statuses.size() > 0; + } + + @Override + public List getResult() { + return statuses; + } + + private void closeOpenHandle() { + statuses.addAll(handle.close()); + } +} + diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java index b0674b2a134d..78b3cb1dc61f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -27,7 +27,8 @@ import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.io.WriteHandleFactory; +import org.apache.hudi.io.ExplicitWriteHandleFactory; +import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; @@ -36,15 +37,6 @@ import java.util.List; public class FlinkLazyInsertIterable extends HoodieLazyInsertIterable { - public FlinkLazyInsertIterable(Iterator> recordItr, - boolean areRecordsSorted, - HoodieWriteConfig config, - String instantTime, - HoodieTable hoodieTable, - String idPrefix, - TaskContextSupplier taskContextSupplier) { - super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); - } public FlinkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, @@ -53,7 +45,7 @@ public FlinkLazyInsertIterable(Iterator> recordItr, HoodieTable hoodieTable, String idPrefix, TaskContextSupplier taskContextSupplier, - WriteHandleFactory writeHandleFactory) { + ExplicitWriteHandleFactory writeHandleFactory) { super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory); } @@ -64,8 +56,8 @@ protected List computeNext() { null; try { final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); - bufferedIteratorExecutor = - new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr), Option.of(getInsertHandler()), getTransformFunction(schema, hoodieConfig)); + bufferedIteratorExecutor = new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr), + Option.of(getExplicitInsertHandler()), getTransformFunction(schema, hoodieConfig)); final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; @@ -77,4 +69,10 @@ protected List computeNext() { } } } + + @SuppressWarnings("rawtypes") + private ExplicitWriteHandler getExplicitInsertHandler() { + HoodieWriteHandle handle = ((ExplicitWriteHandleFactory) writeHandleFactory).getWriteHandle(); + return new ExplicitWriteHandler(handle); + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java index 092e945f0f9d..e598a033750d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/ExplicitWriteHandleFactory.java @@ -28,7 +28,7 @@ */ public class ExplicitWriteHandleFactory extends WriteHandleFactory { - private HoodieWriteHandle writeHandle; + private final HoodieWriteHandle writeHandle; public ExplicitWriteHandleFactory(HoodieWriteHandle writeHandle) { this.writeHandle = writeHandle; @@ -41,4 +41,8 @@ public HoodieWriteHandle create( String fileIdPrefix, TaskContextSupplier taskContextSupplier) { return writeHandle; } + + public HoodieWriteHandle getWriteHandle() { + return writeHandle; + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java index 1872637aeefd..b514896aa1e3 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkAppendHandle.java @@ -74,6 +74,11 @@ protected void createMarkerFile(String partitionPath, String dataFileName) { writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType()); } + @Override + public boolean canWrite(HoodieRecord record) { + return true; + } + @Override protected boolean needsUpdateLocation() { return false; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java index 1c8b988b076b..3d9524eaa30e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java @@ -18,19 +18,22 @@ package org.apache.hudi.io.storage.row.parquet; +import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalDataUtils; +import org.apache.flink.table.data.MapData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.util.Preconditions; import org.apache.parquet.io.api.Binary; import org.apache.parquet.io.api.RecordConsumer; import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.Type; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -46,7 +49,8 @@ /** * Writes a record to the Parquet API with the expected schema in order to be written to a file. * - *

Reference org.apache.flink.formats.parquet.row.ParquetRowDataWriter to support timestamp of INT64 8 bytes. + *

Reference {@code org.apache.flink.formats.parquet.row.ParquetRowDataWriter} + * to support timestamp of INT64 8 bytes and complex data types. */ public class ParquetRowDataWriter { @@ -67,7 +71,7 @@ public ParquetRowDataWriter( this.filedWriters = new FieldWriter[rowType.getFieldCount()]; this.fieldNames = rowType.getFieldNames().toArray(new String[0]); for (int i = 0; i < rowType.getFieldCount(); i++) { - this.filedWriters[i] = createWriter(rowType.getTypeAt(i), schema.getType(i)); + this.filedWriters[i] = createWriter(rowType.getTypeAt(i)); } } @@ -91,59 +95,75 @@ public void write(final RowData record) { recordConsumer.endMessage(); } - private FieldWriter createWriter(LogicalType t, Type type) { - if (type.isPrimitive()) { - switch (t.getTypeRoot()) { - case CHAR: - case VARCHAR: - return new StringWriter(); - case BOOLEAN: - return new BooleanWriter(); - case BINARY: - case VARBINARY: - return new BinaryWriter(); - case DECIMAL: - DecimalType decimalType = (DecimalType) t; - return createDecimalWriter(decimalType.getPrecision(), decimalType.getScale()); - case TINYINT: - return new ByteWriter(); - case SMALLINT: - return new ShortWriter(); - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case INTEGER: - return new IntWriter(); - case BIGINT: - return new LongWriter(); - case FLOAT: - return new FloatWriter(); - case DOUBLE: - return new DoubleWriter(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - TimestampType timestampType = (TimestampType) t; - if (timestampType.getPrecision() == 3) { - return new Timestamp64Writer(); - } else { - return new Timestamp96Writer(timestampType.getPrecision()); - } - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t; - if (localZonedTimestampType.getPrecision() == 3) { - return new Timestamp64Writer(); - } else { - return new Timestamp96Writer(localZonedTimestampType.getPrecision()); - } - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } else { - throw new IllegalArgumentException("Unsupported data type: " + t); + private FieldWriter createWriter(LogicalType t) { + switch (t.getTypeRoot()) { + case CHAR: + case VARCHAR: + return new StringWriter(); + case BOOLEAN: + return new BooleanWriter(); + case BINARY: + case VARBINARY: + return new BinaryWriter(); + case DECIMAL: + DecimalType decimalType = (DecimalType) t; + return createDecimalWriter(decimalType.getPrecision(), decimalType.getScale()); + case TINYINT: + return new ByteWriter(); + case SMALLINT: + return new ShortWriter(); + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTEGER: + return new IntWriter(); + case BIGINT: + return new LongWriter(); + case FLOAT: + return new FloatWriter(); + case DOUBLE: + return new DoubleWriter(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) t; + if (timestampType.getPrecision() == 3) { + return new Timestamp64Writer(); + } else { + return new Timestamp96Writer(timestampType.getPrecision()); + } + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t; + if (localZonedTimestampType.getPrecision() == 3) { + return new Timestamp64Writer(); + } else { + return new Timestamp96Writer(localZonedTimestampType.getPrecision()); + } + case ARRAY: + ArrayType arrayType = (ArrayType) t; + LogicalType elementType = arrayType.getElementType(); + FieldWriter elementWriter = createWriter(elementType); + return new ArrayWriter(elementWriter); + case MAP: + MapType mapType = (MapType) t; + LogicalType keyType = mapType.getKeyType(); + LogicalType valueType = mapType.getValueType(); + FieldWriter keyWriter = createWriter(keyType); + FieldWriter valueWriter = createWriter(valueType); + return new MapWriter(keyWriter, valueWriter); + case ROW: + RowType rowType = (RowType) t; + FieldWriter[] fieldWriters = rowType.getFields().stream() + .map(RowType.RowField::getType).map(this::createWriter).toArray(FieldWriter[]::new); + String[] fieldNames = rowType.getFields().stream() + .map(RowType.RowField::getName).toArray(String[]::new); + return new RowWriter(fieldNames, fieldWriters); + default: + throw new UnsupportedOperationException("Unsupported type: " + t); } } private interface FieldWriter { - void write(RowData row, int ordinal); + + void write(ArrayData array, int ordinal); } private class BooleanWriter implements FieldWriter { @@ -152,6 +172,11 @@ private class BooleanWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addBoolean(row.getBoolean(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addBoolean(array.getBoolean(ordinal)); + } } private class ByteWriter implements FieldWriter { @@ -160,6 +185,11 @@ private class ByteWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addInteger(row.getByte(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addInteger(array.getByte(ordinal)); + } } private class ShortWriter implements FieldWriter { @@ -168,6 +198,11 @@ private class ShortWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addInteger(row.getShort(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addInteger(array.getShort(ordinal)); + } } private class LongWriter implements FieldWriter { @@ -176,6 +211,11 @@ private class LongWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addLong(row.getLong(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addLong(array.getLong(ordinal)); + } } private class FloatWriter implements FieldWriter { @@ -184,6 +224,11 @@ private class FloatWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addFloat(row.getFloat(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addFloat(array.getFloat(ordinal)); + } } private class DoubleWriter implements FieldWriter { @@ -192,6 +237,11 @@ private class DoubleWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addDouble(row.getDouble(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addDouble(array.getDouble(ordinal)); + } } private class StringWriter implements FieldWriter { @@ -200,6 +250,11 @@ private class StringWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addBinary(Binary.fromReusedByteArray(row.getString(ordinal).toBytes())); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addBinary(Binary.fromReusedByteArray(array.getString(ordinal).toBytes())); + } } private class BinaryWriter implements FieldWriter { @@ -208,6 +263,11 @@ private class BinaryWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addBinary(Binary.fromReusedByteArray(row.getBinary(ordinal))); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addBinary(Binary.fromReusedByteArray(array.getBinary(ordinal))); + } } private class IntWriter implements FieldWriter { @@ -216,6 +276,11 @@ private class IntWriter implements FieldWriter { public void write(RowData row, int ordinal) { recordConsumer.addInteger(row.getInt(ordinal)); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addInteger(array.getInt(ordinal)); + } } /** @@ -231,6 +296,11 @@ private Timestamp64Writer() { public void write(RowData row, int ordinal) { recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3))); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3))); + } } private long timestampToInt64(TimestampData timestampData) { @@ -254,6 +324,11 @@ private Timestamp96Writer(int precision) { public void write(RowData row, int ordinal) { recordConsumer.addBinary(timestampToInt96(row.getTimestamp(ordinal, precision))); } + + @Override + public void write(ArrayData array, int ordinal) { + recordConsumer.addBinary(timestampToInt96(array.getTimestamp(ordinal, precision))); + } } private Binary timestampToInt96(TimestampData timestampData) { @@ -304,10 +379,20 @@ private LongUnscaledBytesWriter() { @Override public void write(RowData row, int ordinal) { long unscaledLong = row.getDecimal(ordinal, precision, scale).toUnscaledLong(); + doWrite(unscaledLong); + } + + @Override + public void write(ArrayData array, int ordinal) { + long unscaledLong = array.getDecimal(ordinal, precision, scale).toUnscaledLong(); + doWrite(unscaledLong); + } + + private void doWrite(long unscaled) { int i = 0; int shift = initShift; while (i < numBytes) { - decimalBuffer[i] = (byte) (unscaledLong >> shift); + decimalBuffer[i] = (byte) (unscaled >> shift); i += 1; shift -= 8; } @@ -328,6 +413,16 @@ private UnscaledBytesWriter() { @Override public void write(RowData row, int ordinal) { byte[] bytes = row.getDecimal(ordinal, precision, scale).toUnscaledBytes(); + doWrite(bytes); + } + + @Override + public void write(ArrayData array, int ordinal) { + byte[] bytes = array.getDecimal(ordinal, precision, scale).toUnscaledBytes(); + doWrite(bytes); + } + + private void doWrite(byte[] bytes) { byte[] writtenBytes; if (bytes.length == numBytes) { // Avoid copy. @@ -353,5 +448,132 @@ public void write(RowData row, int ordinal) { // 19 <= precision <= 38, writes as FIXED_LEN_BYTE_ARRAY return new UnscaledBytesWriter(); } + + private class ArrayWriter implements FieldWriter { + private final FieldWriter elementWriter; + + private ArrayWriter(FieldWriter elementWriter) { + this.elementWriter = elementWriter; + } + + @Override + public void write(RowData row, int ordinal) { + ArrayData arrayData = row.getArray(ordinal); + doWrite(arrayData); + } + + @Override + public void write(ArrayData array, int ordinal) { + ArrayData arrayData = array.getArray(ordinal); + doWrite(arrayData); + } + + private void doWrite(ArrayData arrayData) { + recordConsumer.startGroup(); + if (arrayData.size() > 0) { + final String repeatedGroup = "list"; + final String elementField = "element"; + recordConsumer.startField(repeatedGroup, 0); + for (int i = 0; i < arrayData.size(); i++) { + recordConsumer.startGroup(); + if (!arrayData.isNullAt(i)) { + // Only creates the element field if the current array element is not null. + recordConsumer.startField(elementField, 0); + elementWriter.write(arrayData, i); + recordConsumer.endField(elementField, 0); + } + recordConsumer.endGroup(); + } + recordConsumer.endField(repeatedGroup, 0); + } + recordConsumer.endGroup(); + } + } + + private class MapWriter implements FieldWriter { + private final FieldWriter keyWriter; + private final FieldWriter valueWriter; + + private MapWriter(FieldWriter keyWriter, FieldWriter valueWriter) { + this.keyWriter = keyWriter; + this.valueWriter = valueWriter; + } + + @Override + public void write(RowData row, int ordinal) { + MapData map = row.getMap(ordinal); + doWrite(map); + } + + @Override + public void write(ArrayData array, int ordinal) { + MapData map = array.getMap(ordinal); + doWrite(map); + } + + private void doWrite(MapData mapData) { + ArrayData keyArray = mapData.keyArray(); + ArrayData valueArray = mapData.valueArray(); + recordConsumer.startGroup(); + if (mapData.size() > 0) { + final String repeatedGroup = "key_value"; + final String kField = "key"; + final String vField = "value"; + recordConsumer.startField(repeatedGroup, 0); + for (int i = 0; i < mapData.size(); i++) { + recordConsumer.startGroup(); + // key + recordConsumer.startField(kField, 0); + this.keyWriter.write(keyArray, i); + recordConsumer.endField(kField, 0); + // value + if (!valueArray.isNullAt(i)) { + // Only creates the "value" field if the value if non-empty + recordConsumer.startField(vField, 1); + this.valueWriter.write(valueArray, i); + recordConsumer.endField(vField, 1); + } + recordConsumer.endGroup(); + } + recordConsumer.endField(repeatedGroup, 0); + } + recordConsumer.endGroup(); + } + } + + private class RowWriter implements FieldWriter { + private final String[] fieldNames; + private final FieldWriter[] fieldWriters; + + private RowWriter(String[] fieldNames, FieldWriter[] fieldWriters) { + this.fieldNames = fieldNames; + this.fieldWriters = fieldWriters; + } + + @Override + public void write(RowData row, int ordinal) { + RowData nested = row.getRow(ordinal, fieldWriters.length); + doWrite(nested); + } + + @Override + public void write(ArrayData array, int ordinal) { + RowData nested = array.getRow(ordinal, fieldWriters.length); + doWrite(nested); + } + + private void doWrite(RowData row) { + recordConsumer.startGroup(); + for (int i = 0; i < row.getArity(); i++) { + if (!row.isNullAt(i)) { + String fieldName = fieldNames[i]; + recordConsumer.startField(fieldName, i); + fieldWriters[i].write(row, i); + recordConsumer.endField(fieldName, i); + } + } + recordConsumer.endGroup(); + } + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java index 5187660c8cae..5da45bf25d3f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java @@ -25,9 +25,11 @@ import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; @@ -564,7 +566,7 @@ private static Type convertToParquetType( int scale = ((DecimalType) type).getScale(); int numBytes = computeMinBytesForDecimalPrecision(precision); return Types.primitive( - PrimitiveType.PrimitiveTypeName.BINARY, repetition) + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) .precision(precision) .scale(scale) .length(numBytes) @@ -616,6 +618,45 @@ private static Type convertToParquetType( return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition) .named(name); } + case ARRAY: + // group (LIST) { + // repeated group list { + // element; + // } + // } + ArrayType arrayType = (ArrayType) type; + LogicalType elementType = arrayType.getElementType(); + return Types + .buildGroup(repetition).as(OriginalType.LIST) + .addField( + Types.repeatedGroup() + .addField(convertToParquetType("element", elementType, repetition)) + .named("list")) + .named(name); + case MAP: + // group (MAP) { + // repeated group key_value { + // required key; + // value; + // } + // } + MapType mapType = (MapType) type; + LogicalType keyType = mapType.getKeyType(); + LogicalType valueType = mapType.getValueType(); + return Types + .buildGroup(repetition).as(OriginalType.MAP) + .addField( + Types + .repeatedGroup() + .addField(convertToParquetType("key", keyType, repetition)) + .addField(convertToParquetType("value", valueType, repetition)) + .named("key_value")) + .named(name); + case ROW: + RowType rowType = (RowType) type; + Types.GroupBuilder builder = Types.buildGroup(repetition); + rowType.getFields().forEach(field -> builder.addField(convertToParquetType(field.getName(), field.getType(), repetition))); + return builder.named(name); default: throw new UnsupportedOperationException("Unsupported type: " + type); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 174fc1eb906c..af9fee068804 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -56,14 +56,23 @@ public static HoodieTableMetadataWriter create(Co HoodieWriteConfig writeConfig, HoodieEngineContext context, Option actionMetadata) { - return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata); + return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, Option.empty()); + } + + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieEngineContext context, + Option actionMetadata, + Option inFlightInstantTimestamp) { + return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, inFlightInstantTimestamp); } FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext, - Option actionMetadata) { - super(hadoopConf, writeConfig, engineContext, actionMetadata, Option.empty()); + Option actionMetadata, + Option inFlightInstantTimestamp) { + super(hadoopConf, writeConfig, engineContext, actionMetadata, inFlightInstantTimestamp); } @Override @@ -130,7 +139,8 @@ protected void commit(HoodieData hoodieDataRecords, String partiti metadataMetaClient.reloadActiveTimeline(); if (canTriggerTableService) { compactIfNecessary(writeClient, instantTime); - doClean(writeClient, instantTime); + cleanIfNecessary(writeClient, instantTime); + writeClient.archive(); } } @@ -144,7 +154,7 @@ protected void commit(HoodieData hoodieDataRecords, String partiti * The record is tagged with respective file slice's location based on its record key. */ private List prepRecords(List records, String partitionName, int numFileGroups) { - List fileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName); + List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName); ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups)); return records.stream().map(r -> { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 85ad1364de9c..9aceffe44f86 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -27,6 +27,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -38,12 +39,15 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; import org.apache.hudi.io.HoodieWriteHandle; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; @@ -62,6 +66,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import java.io.IOException; import java.util.Collections; import java.util.Iterator; @@ -240,7 +245,7 @@ public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String } @Override - public void updateStatistics(HoodieEngineContext context, List stats, String instantTime, Boolean isOptimizeOperation) { + public void updateMetadataIndexes(@Nonnull HoodieEngineContext context, @Nonnull List stats, @Nonnull String instantTime) { throw new HoodieNotSupportedException("update statistics is not supported yet"); } @@ -316,8 +321,9 @@ public Option scheduleCleaning(HoodieEngineContext context, S @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override @@ -373,12 +379,21 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { + Option keyGeneratorOpt = Option.empty(); + if (!config.populateMetaFields()) { + try { + keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieAvroKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()))); + } catch (IOException e) { + throw new HoodieIOException("Only BaseKeyGenerator (or any key generator that extends from BaseKeyGenerator) are supported when meta " + + "columns are disabled. Please choose the right key generator if you wish to disable meta fields.", e); + } + } if (requireSortedRecords()) { return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, Option.empty()); + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } else { return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, Option.empty()); + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index 5ad87e0831e9..a65e03da761d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -120,8 +120,9 @@ context, config, this, compactionInstantTime, new HoodieFlinkMergeOnReadTableCom @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index f00781f8fa69..164b00e2d6ce 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -31,17 +31,12 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.FlinkHoodieIndexFactory; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; -import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hadoop.fs.Path; - -import java.io.IOException; import java.util.List; import static org.apache.hudi.common.data.HoodieList.getList; @@ -50,9 +45,6 @@ public abstract class HoodieFlinkTable extends HoodieTable>, List, List> implements ExplicitWriteHandleTable { - private boolean isMetadataAvailabilityUpdated = false; - private boolean isMetadataTableAvailable; - protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); } @@ -108,22 +100,11 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - public Option getMetadataWriter(Option actionMetadata) { - synchronized (this) { - if (!isMetadataAvailabilityUpdated) { - // This code assumes that if metadata availability is updated once it will not change. - // Please revisit this logic if that's not the case. This is done to avoid repeated calls to fs.exists(). - try { - isMetadataTableAvailable = config.isMetadataTableEnabled() - && metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()))); - } catch (IOException e) { - throw new HoodieMetadataException("Checking existence of metadata table failed", e); - } - isMetadataAvailabilityUpdated = true; - } - } - if (isMetadataTableAvailable) { - return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context)); + public Option getMetadataWriter(String triggeringInstantTimestamp, + Option actionMetadata) { + if (config.isMetadataTableEnabled()) { + return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, + context, actionMetadata, Option.of(triggeringInstantTimestamp))); } else { return Option.empty(); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java index 041598314f34..363aa28675a4 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java @@ -47,6 +47,6 @@ public FlinkInsertCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java index 583e0b6a94b9..ca1ae6760b79 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteCommitActionExecutor.java @@ -64,6 +64,6 @@ protected String getCommitActionType() { @Override public HoodieWriteMetadata> execute() { return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java index a31679b63f70..f64d8d20d0b2 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertOverwriteTableCommitActionExecutor.java @@ -45,6 +45,6 @@ public FlinkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context @Override public HoodieWriteMetadata> execute() { return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java index 5859bb585fcd..c76733db1de3 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java @@ -47,6 +47,6 @@ public FlinkUpsertCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java index 0863ad8e48e0..3914e486f8a8 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; @@ -64,7 +65,7 @@ public static FlinkWriteHelper newInstance() { @Override public HoodieWriteMetadata> write(String instantTime, List> inputRecords, HoodieEngineContext context, HoodieTable>, List, List> table, boolean shouldCombine, int shuffleParallelism, - BaseCommitActionExecutor>, List, List, R> executor, boolean performTagging) { + BaseCommitActionExecutor>, List, List, R> executor, WriteOperationType operationType) { try { Instant lookupBegin = Instant.now(); Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java index 5fdf46f6ebcb..7053f7a16203 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkUpsertDeltaCommitActionExecutor.java @@ -47,6 +47,6 @@ public FlinkUpsertDeltaCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata execute() { return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java new file mode 100644 index 000000000000..5305bcc8aba7 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java @@ -0,0 +1,74 @@ +/* + * 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 org.apache.hudi.io.storage.row.parquet; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Test cases for {@link ParquetSchemaConverter}. + */ +public class TestParquetSchemaConverter { + @Test + void testConvertComplexTypes() { + DataType dataType = DataTypes.ROW( + DataTypes.FIELD("f_array", + DataTypes.ARRAY(DataTypes.CHAR(10))), + DataTypes.FIELD("f_map", + DataTypes.MAP(DataTypes.INT(), DataTypes.VARCHAR(20))), + DataTypes.FIELD("f_row", + DataTypes.ROW( + DataTypes.FIELD("f_row_f0", DataTypes.INT()), + DataTypes.FIELD("f_row_f1", DataTypes.VARCHAR(10)), + DataTypes.FIELD("f_row_f2", + DataTypes.ROW( + DataTypes.FIELD("f_row_f2_f0", DataTypes.INT()), + DataTypes.FIELD("f_row_f2_f1", DataTypes.VARCHAR(10))))))); + org.apache.parquet.schema.MessageType messageType = + ParquetSchemaConverter.convertToParquetMessageType("converted", (RowType) dataType.getLogicalType()); + assertThat(messageType.getColumns().size(), is(7)); + final String expected = "message converted {\n" + + " optional group f_array (LIST) {\n" + + " repeated group list {\n" + + " optional binary element (UTF8);\n" + + " }\n" + + " }\n" + + " optional group f_map (MAP) {\n" + + " repeated group key_value {\n" + + " optional int32 key;\n" + + " optional binary value (UTF8);\n" + + " }\n" + + " }\n" + + " optional group f_row {\n" + + " optional int32 f_row_f0;\n" + + " optional binary f_row_f1 (UTF8);\n" + + " optional group f_row_f2 {\n" + + " optional int32 f_row_f2_f0;\n" + + " optional binary f_row_f2_f1 (UTF8);\n" + + " }\n" + + " }\n" + + "}\n"; + assertThat(messageType.toString(), is(expected)); + } +} diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java index 60ae294e6a92..50e8f776ac63 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; @@ -38,7 +39,9 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -105,14 +108,18 @@ public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId return (HoodieFlinkWriteableTestTable) withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null)); } - public HoodieFlinkWriteableTestTable withLogAppends(List records) throws Exception { - for (List groupedRecords: records.stream().collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) { - appendRecordsToLogFile(groupedRecords); + public Map> withLogAppends(List records) throws Exception { + Map> partitionToLogfilesMap = new HashMap<>(); + for (List groupedRecords : records.stream().collect( + Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) { + final Pair appendedLogFile = appendRecordsToLogFile(groupedRecords); + partitionToLogfilesMap.computeIfAbsent( + appendedLogFile.getKey(), k -> new ArrayList<>()).add(appendedLogFile.getValue()); } - return this; + return partitionToLogfilesMap; } - private void appendRecordsToLogFile(List groupedRecords) throws Exception { + private Pair appendRecordsToLogFile(List groupedRecords) throws Exception { String partitionPath = groupedRecords.get(0).getPartitionPath(); HoodieRecordLocation location = groupedRecords.get(0).getCurrentLocation(); try (HoodieLogFormat.Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) @@ -131,6 +138,7 @@ private void appendRecordsToLogFile(List groupedRecords) throws Ex return null; } }).collect(Collectors.toList()), header)); + return Pair.of(partitionPath, logWriter.getLogFile()); } } } diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index af97f68f341e..3471bfb8ba36 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-java-client - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-java-client jar diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java new file mode 100644 index 000000000000..ec7202f4d862 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java @@ -0,0 +1,126 @@ +/* + * 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 org.apache.hudi.client.clustering.plan.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieJavaCopyOnWriteTable; +import org.apache.hudi.table.HoodieJavaMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; + +/** + * Clustering Strategy for Java engine based on following. + * 1) Creates clustering groups based on max size allowed per group. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class JavaSizeBasedClusteringPlanStrategy> + extends PartitionAwareClusteringPlanStrategy>, List, List> { + private static final Logger LOG = LogManager.getLogger(JavaSizeBasedClusteringPlanStrategy.class); + + public JavaSizeBasedClusteringPlanStrategy(HoodieJavaCopyOnWriteTable table, + HoodieJavaEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public JavaSizeBasedClusteringPlanStrategy(HoodieJavaMergeOnReadTable table, + HoodieJavaEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + List, Integer>> fileSliceGroups = new ArrayList<>(); + List currentGroup = new ArrayList<>(); + long totalSizeSoFar = 0; + HoodieWriteConfig writeConfig = getWriteConfig(); + for (FileSlice currentSlice : fileSlices) { + // assume each filegroup size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); + // check if max size is reached and create new group, if needed. + if (totalSizeSoFar >= writeConfig.getClusteringMaxBytesInGroup() && !currentGroup.isEmpty()) { + int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes()); + LOG.info("Adding one clustering group " + totalSizeSoFar + " max bytes: " + + writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size() + " output groups: " + numOutputGroups); + fileSliceGroups.add(Pair.of(currentGroup, numOutputGroups)); + currentGroup = new ArrayList<>(); + totalSizeSoFar = 0; + } + currentGroup.add(currentSlice); + // totalSizeSoFar could be 0 when new group was created in the previous conditional block. + // reset to the size of current slice, otherwise the number of output file group will become 0 even though current slice is present. + if (totalSizeSoFar == 0) { + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); + } + } + if (!currentGroup.isEmpty()) { + int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes()); + LOG.info("Adding final clustering group " + totalSizeSoFar + " max bytes: " + + writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size() + " output groups: " + numOutputGroups); + fileSliceGroups.add(Pair.of(currentGroup, numOutputGroups)); + } + + return fileSliceGroups.stream().map(fileSliceGroup -> HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + @Override + protected Stream getFileSlicesEligibleForClustering(final String partition) { + return super.getFileSlicesEligibleForClustering(partition) + // Only files that have basefile size smaller than small file size are eligible. + .filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit()); + } + + private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) { + return (int) Math.ceil(groupSize / (double) targetFileSize); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java new file mode 100644 index 000000000000..c830925419b5 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java @@ -0,0 +1,242 @@ +/* + * 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 org.apache.hudi.client.clustering.run.strategy; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.JavaTaskContextSupplier; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.RewriteAvroPayload; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.execution.bulkinsert.JavaCustomColumnsSortPartitioner; +import org.apache.hudi.io.IOUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.keygen.KeyGenUtils; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader; +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; + +/** + * Clustering strategy for Java engine. + */ +public abstract class JavaExecutionStrategy> + extends ClusteringExecutionStrategy>, List, List> { + + private static final Logger LOG = LogManager.getLogger(JavaExecutionStrategy.class); + + public JavaExecutionStrategy( + HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public HoodieWriteMetadata> performClustering( + HoodieClusteringPlan clusteringPlan, Schema schema, String instantTime) { + // execute clustering for each group and collect WriteStatus + List writeStatusList = new ArrayList<>(); + clusteringPlan.getInputGroups().forEach( + inputGroup -> writeStatusList.addAll(runClusteringForGroup( + inputGroup, clusteringPlan.getStrategy().getStrategyParams(), + Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), + instantTime))); + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); + writeMetadata.setWriteStatuses(writeStatusList); + return writeMetadata; + } + + /** + * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. + * The number of new file groups created is bounded by numOutputGroups. + * Note that commit is not done as part of strategy. commit is callers responsibility. + * + * @param inputRecords List of {@link HoodieRecord}. + * @param numOutputGroups Number of output file groups. + * @param instantTime Clustering (replace commit) instant time. + * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. + * @param schema Schema of the data including metadata fields. + * @param fileGroupIdList File group id corresponding to each out group. + * @param preserveHoodieMetadata Whether to preserve commit metadata while clustering. + * @return List of {@link WriteStatus}. + */ + public abstract List performClusteringWithRecordList( + final List> inputRecords, final int numOutputGroups, final String instantTime, + final Map strategyParams, final Schema schema, + final List fileGroupIdList, final boolean preserveHoodieMetadata); + + /** + * Create {@link BulkInsertPartitioner} based on strategy params. + * + * @param strategyParams Strategy parameters containing columns to sort the data by when clustering. + * @param schema Schema of the data including metadata fields. + * @return empty for now. + */ + protected Option> getPartitioner(Map strategyParams, Schema schema) { + if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) { + return Option.of(new JavaCustomColumnsSortPartitioner( + strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","), + HoodieAvroUtils.addMetadataFields(schema))); + } else { + return Option.empty(); + } + } + + /** + * Executes clustering for the group. + */ + private List runClusteringForGroup( + HoodieClusteringGroup clusteringGroup, Map strategyParams, + boolean preserveHoodieMetadata, String instantTime) { + List> inputRecords = readRecordsForGroup(clusteringGroup, instantTime); + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); + List inputFileIds = clusteringGroup.getSlices().stream() + .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) + .collect(Collectors.toList()); + return performClusteringWithRecordList(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata); + } + + /** + * Get a list of all records for the group. This includes all records from file slice + * (Apply updates from log files, if any). + */ + private List> readRecordsForGroup(HoodieClusteringGroup clusteringGroup, String instantTime) { + List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); + boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); + if (hasLogFiles) { + // if there are log files, we read all records into memory for a file group and apply updates. + return readRecordsForGroupWithLogs(clusteringOps, instantTime); + } else { + // We want to optimize reading records for case there are no log files. + return readRecordsForGroupBaseFiles(clusteringOps); + } + } + + /** + * Read records from baseFiles and apply updates. + */ + private List> readRecordsForGroupWithLogs(List clusteringOps, + String instantTime) { + HoodieWriteConfig config = getWriteConfig(); + HoodieTable table = getHoodieTable(); + List> records = new ArrayList<>(); + + clusteringOps.forEach(clusteringOp -> { + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new JavaTaskContextSupplier(), config); + LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(table.getMetaClient().getFs()) + .withBasePath(table.getMetaClient().getBasePath()) + .withLogFilePaths(clusteringOp.getDeltaFilePaths()) + .withReaderSchema(readerSchema) + .withLatestInstantTime(instantTime) + .withMaxMemorySizeInBytes(maxMemoryPerCompaction) + .withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled()) + .withReverseReader(config.getCompactionReverseLogReadEnabled()) + .withBufferSize(config.getMaxDFSStreamBufferSize()) + .withSpillableMapBasePath(config.getSpillableMapBasePath()) + .withPartition(clusteringOp.getPartitionPath()) + .build(); + + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + ? Option.empty() + : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + Iterator> fileSliceReader = getFileSliceReader(baseFileReader, scanner, readerSchema, + tableConfig.getPayloadClass(), + tableConfig.getPreCombineField(), + tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp()))); + fileSliceReader.forEachRemaining(records::add); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + return records; + } + + /** + * Read records from baseFiles. + */ + private List> readRecordsForGroupBaseFiles(List clusteringOps) { + List> records = new ArrayList<>(); + clusteringOps.forEach(clusteringOp -> { + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + Iterator recordIterator = baseFileReader.getRecordIterator(readerSchema); + recordIterator.forEachRemaining(record -> records.add(transform(record))); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + return records; + } + + /** + * Transform IndexedRecord into HoodieRecord. + */ + private HoodieRecord transform(IndexedRecord indexedRecord) { + GenericRecord record = (GenericRecord) indexedRecord; + Option keyGeneratorOpt = Option.empty(); + String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt); + String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt); + HoodieKey hoodieKey = new HoodieKey(key, partition); + + HoodieRecordPayload avroPayload = new RewriteAvroPayload(record); + HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload); + return hoodieRecord; + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java new file mode 100644 index 000000000000..a33af7ccd021 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java @@ -0,0 +1,70 @@ +/* + * 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 org.apache.hudi.client.clustering.run.strategy; + +import org.apache.avro.Schema; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.CreateHandleFactory; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.JavaBulkInsertHelper; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Clustering Strategy based on following. + * 1) Java execution engine. + * 2) Uses bulk_insert to write data into new files. + */ +public class JavaSortAndSizeExecutionStrategy> + extends JavaExecutionStrategy { + private static final Logger LOG = LogManager.getLogger(JavaSortAndSizeExecutionStrategy.class); + + public JavaSortAndSizeExecutionStrategy(HoodieTable table, + HoodieEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public List performClusteringWithRecordList( + final List> inputRecords, final int numOutputGroups, + final String instantTime, final Map strategyParams, final Schema schema, + final List fileGroupIdList, final boolean preserveHoodieMetadata) { + LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); + Properties props = getWriteConfig().getProps(); + props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups)); + // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files. + props.put(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key(), Boolean.FALSE.toString()); + props.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build(); + return (List) JavaBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java new file mode 100644 index 000000000000..bb7cd5e23a5b --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java @@ -0,0 +1,62 @@ +/* + * 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 org.apache.hudi.execution.bulkinsert; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.table.BulkInsertPartitioner; + +import org.apache.avro.Schema; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * A partitioner that does sorting based on specified column values for Java client. + * + * @param HoodieRecordPayload type + */ +public class JavaCustomColumnsSortPartitioner + implements BulkInsertPartitioner>> { + + private final String[] sortColumnNames; + private final Schema schema; + + public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema) { + this.sortColumnNames = columnNames; + this.schema = schema; + } + + @Override + public List> repartitionRecords( + List> records, int outputSparkPartitions) { + return records.stream().sorted((o1, o2) -> { + Object values1 = HoodieAvroUtils.getRecordColumnValues(o1, sortColumnNames, schema); + Object values2 = HoodieAvroUtils.getRecordColumnValues(o2, sortColumnNames, schema); + return values1.toString().compareTo(values2.toString()); + }).collect(Collectors.toList()); + } + + @Override + public boolean arePartitionRecordsSorted() { + return true; + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index a9e582110c20..62a6980d509a 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -29,6 +29,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -39,17 +40,24 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.io.HoodieCreateHandle; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.HoodieSortedMergeHandle; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; -import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor; +import org.apache.hudi.table.action.cluster.JavaClusteringPlanActionExecutor; +import org.apache.hudi.table.action.cluster.JavaExecuteClusteringCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaInsertOverwriteCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaInsertOverwriteTableCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaMergeHelper; import org.apache.hudi.table.action.commit.JavaUpsertCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaUpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; @@ -57,10 +65,21 @@ import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import javax.annotation.Nonnull; import java.util.List; import java.util.Map; -public class HoodieJavaCopyOnWriteTable extends HoodieJavaTable { +public class HoodieJavaCopyOnWriteTable + extends HoodieJavaTable implements HoodieCompactionHandler { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieJavaCopyOnWriteTable.class); + protected HoodieJavaCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { @@ -152,7 +171,7 @@ public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineC } @Override - public void updateStatistics(HoodieEngineContext context, List stats, String instantTime, Boolean isOptimizeOperation) { + public void updateMetadataIndexes(@Nonnull HoodieEngineContext context, @Nonnull List stats, @Nonnull String instantTime) { throw new HoodieNotSupportedException("update statistics is not supported yet"); } @@ -160,23 +179,23 @@ public void updateStatistics(HoodieEngineContext context, List public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { - throw new HoodieNotSupportedException("ScheduleCompaction is not supported yet"); + throw new HoodieNotSupportedException("ScheduleCompaction is not supported on a CopyOnWrite table"); } @Override public HoodieWriteMetadata> compact(HoodieEngineContext context, String compactionInstantTime) { - throw new HoodieNotSupportedException("Compact is not supported yet"); + throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } @Override public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { - throw new HoodieNotSupportedException("Clustering is not supported yet"); + return new JavaClusteringPlanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute(); } @Override public HoodieWriteMetadata> cluster(final HoodieEngineContext context, final String clusteringInstantTime) { - throw new HoodieNotSupportedException("Clustering is not supported yet"); + return new JavaExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); } @Override @@ -193,8 +212,9 @@ public void rollbackBootstrap(HoodieEngineContext context, @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, HoodieInstant instantToRollback, - boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override @@ -234,4 +254,53 @@ public HoodieRestoreMetadata restore(HoodieEngineContext context, return new CopyOnWriteRestoreActionExecutor( context, config, this, restoreInstantTime, instantToRestore).execute(); } + + @Override + public Iterator> handleUpdate( + String instantTime, String partitionPath, String fileId, + Map> keyToNewRecords, HoodieBaseFile oldDataFile) + throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile); + return handleUpdateInternal(upsertHandle, instantTime, fileId); + } + + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime, + String fileId) throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); + } else { + JavaMergeHelper.newInstance().runMerge(this, upsertHandle); + } + + // TODO(yihua): This needs to be revisited + if (upsertHandle.getPartitionPath() == null) { + LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.writeStatuses()); + } + + return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); + } + + protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, + Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { + if (requireSortedRecords()) { + return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier, Option.empty()); + } else { + return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier, Option.empty()); + } + } + + @Override + public Iterator> handleInsert( + String instantTime, String partitionPath, String fileId, + Map> recordMap) { + HoodieCreateHandle createHandle = + new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); + createHandle.write(); + return Collections.singletonList(createHandle.close()).iterator(); + } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java index b219ba1a9901..136c25b8cdb2 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -29,9 +30,13 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.compact.HoodieJavaMergeOnReadTableCompactor; +import org.apache.hudi.table.action.compact.RunCompactionActionExecutor; +import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor; import org.apache.hudi.table.action.deltacommit.JavaUpsertPreppedDeltaCommitActionExecutor; import java.util.List; +import java.util.Map; public class HoodieJavaMergeOnReadTable extends HoodieJavaCopyOnWriteTable { protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { @@ -60,4 +65,21 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineCont return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); } + + @Override + public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( + context, config, this, instantTime, extraMetadata, + new HoodieJavaMergeOnReadTableCompactor()); + return scheduleCompactionExecutor.execute(); + } + + @Override + public HoodieWriteMetadata> compact( + HoodieEngineContext context, String compactionInstantTime) { + RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor( + context, config, this, compactionInstantTime, new HoodieJavaMergeOnReadTableCompactor(), + new HoodieJavaCopyOnWriteTable(config, context, getMetaClient())); + return convertMetadata(compactionExecutor.execute()); + } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 8b0a7a95ef87..f9c7caff6ebb 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -31,9 +32,12 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.JavaHoodieIndexFactory; +import org.apache.hudi.table.action.HoodieWriteMetadata; import java.util.List; +import static org.apache.hudi.common.data.HoodieList.getList; + public abstract class HoodieJavaTable extends HoodieTable>, List, List> { protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { @@ -61,6 +65,11 @@ public static HoodieJavaTable create(HoodieWr } } + public static HoodieWriteMetadata> convertMetadata( + HoodieWriteMetadata> metadata) { + return metadata.clone(getList(metadata.getWriteStatuses())); + } + @Override protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return JavaHoodieIndexFactory.createIndex(config); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaClusteringPlanActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaClusteringPlanActionExecutor.java new file mode 100644 index 000000000000..1d78ecc2bf41 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaClusteringPlanActionExecutor.java @@ -0,0 +1,43 @@ +/* + * 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 org.apache.hudi.table.action.cluster; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import java.util.List; +import java.util.Map; + +public class JavaClusteringPlanActionExecutor extends + BaseClusteringPlanActionExecutor>, List, List> { + + public JavaClusteringPlanActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime, Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java new file mode 100644 index 000000000000..83364bdc3ad3 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java @@ -0,0 +1,123 @@ +/* + * 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 org.apache.hudi.table.action.cluster; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.hudi.table.action.commit.BaseJavaCommitActionExecutor; + +import org.apache.avro.Schema; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class JavaExecuteClusteringCommitActionExecutor> + extends BaseJavaCommitActionExecutor { + + private final HoodieClusteringPlan clusteringPlan; + + public JavaExecuteClusteringCommitActionExecutor( + HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime) { + super(context, config, table, instantTime, WriteOperationType.CLUSTER); + this.clusteringPlan = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)) + .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException( + "Unable to read clustering plan for instant: " + instantTime)); + } + + @Override + public HoodieWriteMetadata> execute() { + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime); + // Mark instant as clustering inflight + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + table.getMetaClient().reloadActiveTimeline(); + + final Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieWriteMetadata> writeMetadata = ( + (ClusteringExecutionStrategy>, List, List>) + ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(), + new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config)) + .performClustering(clusteringPlan, schema, instantTime); + List writeStatusList = writeMetadata.getWriteStatuses(); + List statuses = updateIndex(writeStatusList, writeMetadata); + writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList())); + writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeMetadata)); + validateWriteResult(writeMetadata); + commitOnAutoCommit(writeMetadata); + if (!writeMetadata.getCommitMetadata().isPresent()) { + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeMetadata.getWriteStats().get(), writeMetadata.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + writeMetadata.setCommitMetadata(Option.of(commitMetadata)); + } + return writeMetadata; + } + + /** + * Validate actions taken by clustering. In the first implementation, we validate at least one new file is written. + * But we can extend this to add more validation. E.g. number of records read = number of records written etc. + * We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions. + */ + private void validateWriteResult(HoodieWriteMetadata> writeMetadata) { + if (writeMetadata.getWriteStatuses().isEmpty()) { + throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + + " #groups: " + clusteringPlan.getInputGroups().size() + " expected at least " + + clusteringPlan.getInputGroups().stream().mapToInt(HoodieClusteringGroup::getNumOutputFileGroups).sum() + + " write statuses"); + } + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + Set newFilesWritten = writeMetadata.getWriteStats().get().stream() + .map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).collect(Collectors.toSet()); + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan) + .filter(fg -> !newFilesWritten.contains(fg)) + .collect(Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList()))); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index 66cb40758bdc..2a93c5012ce1 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -126,13 +126,14 @@ public HoodieWriteMetadata> execute(List> inpu return result; } - protected void updateIndex(List writeStatuses, HoodieWriteMetadata> result) { + protected List updateIndex(List writeStatuses, HoodieWriteMetadata> result) { Instant indexStartTime = Instant.now(); // Update the index back List statuses = HoodieList.getList( table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table)); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); + return statuses; } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java index 096aafcffc09..c1fae07a9d6c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java @@ -45,6 +45,6 @@ public JavaInsertCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java index b80191909351..a99485e67bb8 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteCommitActionExecutor.java @@ -55,7 +55,7 @@ public JavaInsertOverwriteCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java index 06fce78a36d3..ed0af4402869 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java @@ -45,6 +45,6 @@ public JavaUpsertCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java new file mode 100644 index 000000000000..30bdcda759ce --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java @@ -0,0 +1,56 @@ +/* + * 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 org.apache.hudi.table.action.compact; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import java.util.List; + +/** + * Compacts a hoodie table with merge on read storage in Java engine. Computes all possible + * compactions, passes it through a CompactionFilter and executes all the compactions and + * writes a new version of base files and make a normal commit. + */ +public class HoodieJavaMergeOnReadTableCompactor + extends HoodieCompactor>, List, List> { + + @Override + public void preCompact( + HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) { + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { + table.rollbackInflightCompaction(inflightInstant); + table.getMetaClient().reloadActiveTimeline(); + } + } + + @Override + public void maybePersist(HoodieData writeStatus, HoodieWriteConfig config) { + // No OP + } +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java new file mode 100644 index 000000000000..5d6f21164589 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java @@ -0,0 +1,98 @@ +/* + * 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 org.apache.hudi.execution.bulkinsert; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.testutils.HoodieJavaClientTestBase; + +import org.apache.avro.Schema; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestJavaBulkInsertInternalPartitioner extends HoodieJavaClientTestBase { + private static final Comparator KEY_COMPARATOR = + Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())); + + public static List generateTestRecordsForBulkInsert(int numRecords) { + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + List records = dataGenerator.generateInserts("0", numRecords); + return records; + } + + public static Map generatePartitionNumRecords(List records) { + return records.stream().map(record -> record.getPartitionPath()) + .collect(Collectors.groupingBy(Function.identity(), Collectors.counting())); + } + + @ParameterizedTest + @ValueSource(strings = {"rider", "rider,driver"}) + public void testCustomColumnSortPartitioner(String sortColumnString) throws Exception { + String[] sortColumns = sortColumnString.split(","); + Comparator columnComparator = + getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns); + + List records = generateTestRecordsForBulkInsert(1000); + testBulkInsertInternalPartitioner( + new JavaCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA), + records, true, generatePartitionNumRecords(records), Option.of(columnComparator)); + } + + private Comparator getCustomColumnComparator(Schema schema, String[] sortColumns) { + return Comparator.comparing( + record -> HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema).toString()); + } + + private void verifyRecordAscendingOrder(List records, + Option> comparator) { + List expectedRecords = new ArrayList<>(records); + Collections.sort(expectedRecords, comparator.orElse(KEY_COMPARATOR)); + assertEquals(expectedRecords, records); + } + + private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner, + List records, + boolean isSorted, + Map expectedPartitionNumRecords, + Option> comparator) { + List actualRecords = + (List) partitioner.repartitionRecords(records, 1); + if (isSorted) { + // Verify global order + verifyRecordAscendingOrder(actualRecords, comparator); + } + + // Verify number of records per partition path + assertEquals(expectedPartitionNumRecords, generatePartitionNumRecords(actualRecords)); + } +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 4a3f3d5bcef8..796d7b74a83c 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; @@ -115,8 +114,7 @@ private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() { return HoodieWriteConfig.newBuilder() .withEngineType(EngineType.JAVA) .withPath(basePath) - .withSchema(SCHEMA.toString()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); + .withSchema(SCHEMA.toString()); } @Test diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 21cfe6bff24f..e4a8fd56b6a6 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-spark-client - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-spark-client jar diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 891800bb992d..9b2aad3ebafa 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.metrics.Registry; @@ -45,7 +46,6 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndexFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.table.BulkInsertPartitioner; @@ -96,21 +96,6 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, Option timelineService) { super(context, writeConfig, timelineService); - initializeMetadataTable(Option.empty()); - } - - private void initializeMetadataTable(Option inflightInstantTimestamp) { - if (config.isMetadataTableEnabled()) { - // Defer bootstrap if upgrade / downgrade is pending - HoodieTableMetaClient metaClient = createMetaClient(true); - UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade( - metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance()); - if (!upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) { - // TODO: Check if we can remove this requirement - auto bootstrap on commit - SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context, Option.empty(), - inflightInstantTimestamp); - } - } } /** @@ -135,6 +120,7 @@ protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { @Override public boolean commit(String instantTime, JavaRDD writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) { + context.setJobStatus(this.getClass().getSimpleName(), "Committing stats"); List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); } @@ -314,17 +300,23 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStats = writeStatuses.map(WriteStatus::getStat).collect(); - writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime)); - // commit to data table after committing to metadata table. - finalizeWrite(table, compactionCommitTime, writeStats); - LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); - CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); + final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime); + try { + this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); + finalizeWrite(table, compactionCommitTime, writeStats); + // commit to data table after committing to metadata table. + writeTableMetadataForTableServices(table, metadata, compactionInstant); + LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); + CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); + } finally { + this.txnManager.endTransaction(Option.of(compactionInstant)); + } WriteMarkersFactory.get(config.getMarkersType(), table, compactionCommitTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); try { - metrics.updateCommitMetrics(HoodieActiveTimeline.parseInstantTime(compactionCommitTime).getTime(), + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(compactionCommitTime).getTime(), durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); } catch (ParseException e) { throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " @@ -386,26 +378,31 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD s.getTotalWriteErrors() > 0L).map(s -> s.getFileId()).collect(Collectors.joining(","))); } - writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime)); - finalizeWrite(table, clusteringCommitTime, writeStats); + final HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime); try { - // try to save statistics info to hudi - if (config.isDataSkippingEnabled() && config.isLayoutOptimizationEnabled() && !config.getClusteringSortColumns().isEmpty()) { - table.updateStatistics(context, writeStats, clusteringCommitTime, true); + this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); + finalizeWrite(table, clusteringCommitTime, writeStats); + writeTableMetadataForTableServices(table, metadata,clusteringInstant); + // Update outstanding metadata indexes + if (config.isLayoutOptimizationEnabled() + && !config.getClusteringSortColumns().isEmpty()) { + table.updateMetadataIndexes(context, writeStats, clusteringCommitTime); } LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata); table.getActiveTimeline().transitionReplaceInflightToComplete( HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException e) { + } catch (Exception e) { throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); + } finally { + this.txnManager.endTransaction(Option.of(clusteringInstant)); } WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (clusteringTimer != null) { long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); try { - metrics.updateCommitMetrics(HoodieActiveTimeline.parseInstantTime(clusteringCommitTime).getTime(), + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(clusteringCommitTime).getTime(), durationInMs, metadata, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); } catch (ParseException e) { throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " @@ -415,17 +412,13 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata, + private void writeTableMetadataForTableServices(HoodieTable>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata, HoodieInstant hoodieInstant) { - try { - this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty()); - boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction()); - // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a - // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); - } finally { - this.txnManager.endTransaction(); - } + boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction()); + // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a + // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. + table.getMetadataWriter(hoodieInstant.getTimestamp()).ifPresent( + w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); } @Override @@ -433,34 +426,45 @@ protected HoodieTable>, JavaRDD, JavaRDD instantsToRollback = getInstantsToRollback( + metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime)); + Map> pendingRollbacks = getPendingRollbackInfos(metaClient); + instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); + this.rollbackFailedWrites(pendingRollbacks, true); + new UpgradeDowngrade( + metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance()) + .run(HoodieTableVersion.current(), instantTime); + metaClient.reloadActiveTimeline(); + initializeMetadataTable(Option.of(instantTime)); } - metaClient.reloadActiveTimeline(); - - // re-bootstrap metadata table if required - initializeMetadataTable(Option.of(instantTime)); + } finally { + this.txnManager.endTransaction(); } metaClient.validateTableProperties(config.getProps(), operationType); return getTableAndInitCtx(metaClient, operationType, instantTime); } + /** + * Initialize the metadata table if needed. Creating the metadata table writer + * will trigger the initial bootstrapping from the data table. + * + * @param inFlightInstantTimestamp - The in-flight action responsible for the metadata table initialization + */ + private void initializeMetadataTable(Option inFlightInstantTimestamp) { + if (config.isMetadataTableEnabled()) { + SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, + context, Option.empty(), inFlightInstantTimestamp); + } + } + // TODO : To enforce priority between table service and ingestion writer, use transactions here and invoke strategy private void completeTableService(TableServiceType tableServiceType, HoodieCommitMetadata metadata, JavaRDD writeStatuses, - HoodieTable>, JavaRDD, JavaRDD> table, - String commitInstant) { + HoodieTable>, JavaRDD, JavaRDD> table, + String commitInstant) { switch (tableServiceType) { case CLUSTER: @@ -497,8 +501,6 @@ protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata met HoodieTable table = createTable(config, hadoopConf); TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner()); - table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, inflightInstant.getTimestamp(), - table.isTableServiceAction(inflightInstant.getAction()))); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java deleted file mode 100644 index 5c132773c9ce..000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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 org.apache.hudi.client.clustering.plan.strategy; - -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; -import org.apache.hudi.table.HoodieSparkMergeOnReadTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; - -/** - * Clustering Strategy that only looks at latest 'daybased.lookback.partitions' partitions. - */ -public class SparkRecentDaysClusteringPlanStrategy> - extends SparkSizeBasedClusteringPlanStrategy { - private static final Logger LOG = LogManager.getLogger(SparkRecentDaysClusteringPlanStrategy.class); - - public SparkRecentDaysClusteringPlanStrategy(HoodieSparkCopyOnWriteTable table, - HoodieSparkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public SparkRecentDaysClusteringPlanStrategy(HoodieSparkMergeOnReadTable table, - HoodieSparkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - protected List filterPartitionPaths(List partitionPaths) { - int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering(); - int skipPartitionsFromLatestForClustering = getWriteConfig().getSkipPartitionsFromLatestForClustering(); - return partitionPaths.stream() - .sorted(Comparator.reverseOrder()) - .skip(Math.max(skipPartitionsFromLatestForClustering, 0)) - .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size()) - .collect(Collectors.toList()); - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSelectedPartitionsClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSelectedPartitionsClusteringPlanStrategy.java deleted file mode 100644 index 549935d2fdfc..000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSelectedPartitionsClusteringPlanStrategy.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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 org.apache.hudi.client.clustering.plan.strategy; - -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; -import org.apache.hudi.table.HoodieSparkMergeOnReadTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX; - -/** - * Clustering Strategy to filter just specified partitions from [begin, end]. Note both begin and end are inclusive. - */ -public class SparkSelectedPartitionsClusteringPlanStrategy> - extends SparkSizeBasedClusteringPlanStrategy { - private static final Logger LOG = LogManager.getLogger(SparkSelectedPartitionsClusteringPlanStrategy.class); - - public static final String CONF_BEGIN_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.begin.partition"; - public static final String CONF_END_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.end.partition"; - - public SparkSelectedPartitionsClusteringPlanStrategy(HoodieSparkCopyOnWriteTable table, - HoodieSparkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public SparkSelectedPartitionsClusteringPlanStrategy(HoodieSparkMergeOnReadTable table, - HoodieSparkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - @Override - protected List filterPartitionPaths(List partitionPaths) { - String beginPartition = getWriteConfig().getProps().getProperty(CONF_BEGIN_PARTITION); - String endPartition = getWriteConfig().getProps().getProperty(CONF_END_PARTITION); - List filteredPartitions = partitionPaths.stream() - .filter(path -> path.compareTo(beginPartition) >= 0 && path.compareTo(endPartition) <= 0) - .collect(Collectors.toList()); - LOG.info("Filtered to the following partitions: " + filteredPartitions); - return filteredPartitions; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java index b98dbac396a3..88c3057f2f0a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java @@ -48,6 +48,7 @@ public SparkSingleFileSortPlanStrategy(HoodieSparkMergeOnReadTable table, Hoo super(table, engineContext, writeConfig); } + @Override protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { List, Integer>> fileSliceGroups = fileSlices.stream() .map(fileSlice -> Pair.of(Collections.singletonList(fileSlice), 1)).collect(Collectors.toList()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java index c89ff2f2bfc6..b38931c2d93d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java @@ -68,13 +68,13 @@ public SparkSizeBasedClusteringPlanStrategy(HoodieSparkMergeOnReadTable table @Override protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + HoodieWriteConfig writeConfig = getWriteConfig(); + List, Integer>> fileSliceGroups = new ArrayList<>(); List currentGroup = new ArrayList<>(); long totalSizeSoFar = 0; - HoodieWriteConfig writeConfig = getWriteConfig(); + for (FileSlice currentSlice : fileSlices) { - // assume each filegroup size is ~= parquet.max.file.size - totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); // check if max size is reached and create new group, if needed. if (totalSizeSoFar >= writeConfig.getClusteringMaxBytesInGroup() && !currentGroup.isEmpty()) { int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes()); @@ -84,13 +84,13 @@ protected Stream buildClusteringGroupsForPartition(String currentGroup = new ArrayList<>(); totalSizeSoFar = 0; } + + // Add to the current file-group currentGroup.add(currentSlice); - // totalSizeSoFar could be 0 when new group was created in the previous conditional block. - // reset to the size of current slice, otherwise the number of output file group will become 0 even though current slice is present. - if (totalSizeSoFar == 0) { - totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); - } + // assume each filegroup size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); } + if (!currentGroup.isEmpty()) { int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes()); LOG.info("Adding final clustering group " + totalSizeSoFar + " max bytes: " @@ -98,11 +98,12 @@ protected Stream buildClusteringGroupsForPartition(String fileSliceGroups.add(Pair.of(currentGroup, numOutputGroups)); } - return fileSliceGroups.stream().map(fileSliceGroup -> HoodieClusteringGroup.newBuilder() - .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) - .setNumOutputFileGroups(fileSliceGroup.getRight()) - .setMetrics(buildMetrics(fileSliceGroup.getLeft())) - .build()); + return fileSliceGroups.stream().map(fileSliceGroup -> + HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); } @Override @@ -114,11 +115,6 @@ protected Map getStrategyParams() { return params; } - @Override - protected List filterPartitionPaths(List partitionPaths) { - return partitionPaths; - } - @Override protected Stream getFileSlicesEligibleForClustering(final String partition) { return super.getFileSlicesEligibleForClustering(partition) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 044b77362010..4824c757cd9d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -35,7 +35,9 @@ import org.apache.hudi.common.model.RewriteAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.FutureUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; @@ -87,16 +89,17 @@ public MultipleSparkJobExecutionStrategy(HoodieTable table, HoodieEngineContext @Override public HoodieWriteMetadata> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) { - // execute clustering for each group async and collect WriteStatus JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext()); // execute clustering for each group async and collect WriteStatus - Stream> writeStatusRDDStream = clusteringPlan.getInputGroups().stream() + Stream> writeStatusRDDStream = FutureUtils.allOf( + clusteringPlan.getInputGroups().stream() .map(inputGroup -> runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false), instantTime)) - .map(CompletableFuture::join); - + .collect(Collectors.toList())) + .join() + .stream(); JavaRDD[] writeStatuses = convertStreamToArray(writeStatusRDDStream); JavaRDD writeStatusRDD = engineContext.union(writeStatuses); @@ -144,7 +147,6 @@ protected Option> getPartitioner(Map st } } - /** * Submit job to execute clustering for the group. */ @@ -191,7 +193,6 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(table.getMetaClient().getFs()) .withBasePath(table.getMetaClient().getBasePath()) @@ -203,8 +204,12 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js .withReverseReader(config.getCompactionReverseLogReadEnabled()) .withBufferSize(config.getMaxDFSStreamBufferSize()) .withSpillableMapBasePath(config.getSpillableMapBasePath()) + .withPartition(clusteringOp.getPartitionPath()) .build(); + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + ? Option.empty() + : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); recordIterators.add(getFileSliceReader(baseFileReader, scanner, readerSchema, tableConfig.getPayloadClass(), diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 3f89745abcc0..22d5300215d0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -64,7 +64,7 @@ public JavaRDD performClusteringWithRecordsRDD(final JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, - false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); + return (JavaRDD) SparkBulkInsertHelper.newInstance() + .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java index 1547e8c24466..403a0c2e1ca8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java @@ -22,10 +22,15 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; + import org.apache.spark.api.java.JavaRDD; import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; /** * Allow ingestion commits during clustering job. @@ -37,8 +42,19 @@ public SparkAllowUpdateStrategy( super(engineContext, fileGroupsInPendingClustering); } + private List getGroupIdsWithUpdate(JavaRDD> inputRecords) { + List fileGroupIdsWithUpdates = inputRecords + .filter(record -> record.getCurrentLocation() != null) + .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collect(); + return fileGroupIdsWithUpdates; + } + @Override - public JavaRDD> handleUpdate(JavaRDD> taggedRecordsRDD) { - return taggedRecordsRDD; + public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { + List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); + Set fileGroupIdsWithUpdatesAndPendingClustering = fileGroupIdsWithRecordUpdate.stream() + .filter(f -> fileGroupsInPendingClustering.contains(f)) + .collect(Collectors.toSet()); + return Pair.of(taggedRecordsRDD, fileGroupIdsWithUpdatesAndPendingClustering); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java index 134e49024668..b12d9ad43571 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -22,14 +22,18 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieClusteringUpdateException; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; +import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Set; /** * Update strategy based on following. @@ -50,7 +54,7 @@ private List getGroupIdsWithUpdate(JavaRDD> i } @Override - public JavaRDD> handleUpdate(JavaRDD> taggedRecordsRDD) { + public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); fileGroupIdsWithRecordUpdate.forEach(fileGroupIdWithRecordUpdate -> { if (fileGroupsInPendingClustering.contains(fileGroupIdWithRecordUpdate)) { @@ -61,7 +65,7 @@ public JavaRDD> handleUpdate(JavaRDD> taggedReco throw new HoodieClusteringUpdateException(msg); } }); - return taggedRecordsRDD; + return Pair.of(taggedRecordsRDD, Collections.emptySet()); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java index fb3c5ec0dc3c..fb243f5696ef 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -18,18 +18,16 @@ package org.apache.hudi.execution.bulkinsert; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.BulkInsertPartitioner; -import org.apache.spark.api.java.JavaRDD; -import java.io.IOException; +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaRDD; /** * A partitioner that does sorting based on specified column values for each RDD partition. @@ -57,8 +55,18 @@ public JavaRDD> repartitionRecords(JavaRDD> reco int outputSparkPartitions) { final String[] sortColumns = this.sortColumnNames; final SerializableSchema schema = this.serializableSchema; - return records.sortBy(record -> getRecordSortColumnValues(record, sortColumns, schema), + return records.sortBy( + record -> { + Object recordValue = HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema); + // null values are replaced with empty string for null_first order + if (recordValue == null) { + return StringUtils.EMPTY_STRING; + } else { + return StringUtils.objToString(record); + } + }, true, outputSparkPartitions); + } @Override @@ -66,26 +74,6 @@ public boolean arePartitionRecordsSorted() { return true; } - private static Object getRecordSortColumnValues(HoodieRecord record, - String[] sortColumns, - SerializableSchema schema) { - try { - GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema.get()).get(); - if (sortColumns.length == 1) { - return HoodieAvroUtils.getNestedFieldVal(genericRecord, sortColumns[0], true); - } else { - StringBuilder sb = new StringBuilder(); - for (String col : sortColumns) { - sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true)); - } - - return sb.toString(); - } - } catch (IOException e) { - throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e); - } - } - private String[] getSortColumnName(HoodieWriteConfig config) { return config.getUserDefinedBulkInsertPartitionerSortColumns().split(","); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java index fa12159eeac6..896a2aaabdfd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java @@ -18,6 +18,8 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -27,19 +29,22 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.sort.SpaceCurveSortingHelper; import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.spark.ZCurveOptimizeHelper; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; + /** - * A partitioner that does spartial curve optimization sorting based on specified column values for each RDD partition. + * A partitioner that does spatial curve optimization sorting based on specified column values for each RDD partition. * support z-curve optimization, hilbert will come soon. * @param HoodieRecordPayload type */ @@ -74,21 +79,45 @@ public JavaRDD> repartitionRecords(JavaRDD> reco private JavaRDD prepareGenericRecord(JavaRDD> inputRecords, final int numOutputGroups, final Schema schema) { SerializableSchema serializableSchema = new SerializableSchema(schema); JavaRDD genericRecordJavaRDD = inputRecords.map(f -> (GenericRecord) f.getData().getInsertValue(serializableSchema.get()).get()); - Dataset originDF = AvroConversionUtils.createDataFrame(genericRecordJavaRDD.rdd(), schema.toString(), sparkEngineContext.getSqlContext().sparkSession()); - Dataset zDataFrame; + Dataset originDF = + AvroConversionUtils.createDataFrame( + genericRecordJavaRDD.rdd(), + schema.toString(), + sparkEngineContext.getSqlContext().sparkSession() + ); + + Dataset sortedDF = reorder(originDF, numOutputGroups); + + return HoodieSparkUtils.createRdd(sortedDF, schema.getName(), + schema.getNamespace(), false, org.apache.hudi.common.util.Option.empty()).toJavaRDD(); + } + + private Dataset reorder(Dataset originDF, int numOutputGroups) { + String orderedColumnsListConfig = config.getClusteringSortColumns(); + + if (isNullOrEmpty(orderedColumnsListConfig) || numOutputGroups <= 0) { + // No-op + return originDF; + } - switch (config.getLayoutOptimizationCurveBuildMethod()) { + List orderedCols = + Arrays.stream(orderedColumnsListConfig.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy = + HoodieClusteringConfig.LayoutOptimizationStrategy.fromValue(config.getLayoutOptimizationStrategy()); + + HoodieClusteringConfig.BuildCurveStrategyType curveBuildStrategyType = config.getLayoutOptimizationCurveBuildMethod(); + + switch (curveBuildStrategyType) { case DIRECT: - zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameByMapValue(originDF, config.getClusteringSortColumns(), numOutputGroups); - break; + return SpaceCurveSortingHelper.orderDataFrameByMappingValues(originDF, layoutOptStrategy, orderedCols, numOutputGroups); case SAMPLE: - zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameBySample(originDF, config.getClusteringSortColumns(), numOutputGroups); - break; + return SpaceCurveSortingHelper.orderDataFrameBySamplingValues(originDF, layoutOptStrategy, orderedCols, numOutputGroups); default: - throw new HoodieException("Not a valid build curve method for doWriteOperation: "); + throw new UnsupportedOperationException(String.format("Unsupported space-curve curve building strategy (%s)", curveBuildStrategyType)); } - return HoodieSparkUtils.createRdd(zDataFrame, schema.getName(), - schema.getNamespace(), false, org.apache.hudi.common.util.Option.empty()).toJavaRDD(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java index 5e686463bc98..69e18714c49c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java @@ -28,6 +28,7 @@ import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper; +import org.apache.hudi.index.bucket.HoodieBucketIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex; @@ -55,6 +56,8 @@ public static HoodieIndex createIndex(HoodieWriteConfig config) { return new SparkHoodieHBaseIndex<>(config); case INMEMORY: return new HoodieInMemoryHashIndex<>(config); + case BUCKET: + return new HoodieBucketIndex(config); case BLOOM: return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); case GLOBAL_BLOOM: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java new file mode 100644 index 000000000000..d92bac4d8471 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java @@ -0,0 +1,498 @@ +/* + * 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 org.apache.hudi.index.columnstats; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.parquet.io.api.Binary; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Row$; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.util.SerializableConfiguration; +import scala.collection.JavaConversions; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.hudi.util.DataTypeUtils.areCompatible; + +public class ColumnStatsIndexHelper { + + private static final Logger LOG = LogManager.getLogger(ColumnStatsIndexHelper.class); + + private static final String SPARK_JOB_DESCRIPTION = "spark.job.description"; + + private static final String Z_INDEX_FILE_COLUMN_NAME = "file"; + + private static final String Z_INDEX_MIN_VALUE_STAT_NAME = "minValue"; + private static final String Z_INDEX_MAX_VALUE_STAT_NAME = "maxValue"; + private static final String Z_INDEX_NUM_NULLS_STAT_NAME = "num_nulls"; + + public static String getMinColumnNameFor(String colName) { + return composeZIndexColName(colName, Z_INDEX_MIN_VALUE_STAT_NAME); + } + + public static String getMaxColumnNameFor(String colName) { + return composeZIndexColName(colName, Z_INDEX_MAX_VALUE_STAT_NAME); + } + + public static String getNumNullsColumnNameFor(String colName) { + return composeZIndexColName(colName, Z_INDEX_NUM_NULLS_STAT_NAME); + } + + /** + * Parse min/max statistics from Parquet footers for provided columns and composes column-stats + * index table in the following format with 3 statistics denominated for each + * linear/Z-curve/Hilbert-curve-ordered column. For ex, if original table contained + * column {@code A}: + * + *

+   * +---------------------------+------------+------------+-------------+
+   * |          file             | A_minValue | A_maxValue | A_num_nulls |
+   * +---------------------------+------------+------------+-------------+
+   * | one_base_file.parquet     |          1 |         10 |           0 |
+   * | another_base_file.parquet |        -10 |          0 |           5 |
+   * +---------------------------+------------+------------+-------------+
+   * 
+ * + * NOTE: Currently {@link TimestampType} is not supported, since Parquet writer + * does not support statistics for it. + * + * TODO leverage metadata table after RFC-27 lands + * @VisibleForTesting + * + * @param sparkSession encompassing Spark session + * @param baseFilesPaths list of base-files paths to be sourced for column-stats index + * @param orderedColumnSchemas target ordered columns + * @return Spark's {@link Dataset} holding an index table + */ + @Nonnull + public static Dataset buildColumnStatsTableFor( + @Nonnull SparkSession sparkSession, + @Nonnull List baseFilesPaths, + @Nonnull List orderedColumnSchemas + ) { + SparkContext sc = sparkSession.sparkContext(); + JavaSparkContext jsc = new JavaSparkContext(sc); + + SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration()); + int numParallelism = (baseFilesPaths.size() / 3 + 1); + List> colMinMaxInfos; + String previousJobDescription = sc.getLocalProperty(SPARK_JOB_DESCRIPTION); + try { + jsc.setJobDescription("Listing parquet column statistics"); + colMinMaxInfos = + jsc.parallelize(baseFilesPaths, numParallelism) + .mapPartitions(paths -> { + ParquetUtils utils = (ParquetUtils) BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); + Iterable iterable = () -> paths; + return StreamSupport.stream(iterable.spliterator(), false) + .flatMap(path -> + utils.readRangeFromParquetMetadata( + serializableConfiguration.value(), + new Path(path), + orderedColumnSchemas.stream() + .map(StructField::name) + .collect(Collectors.toList()) + ) + .stream() + ) + .iterator(); + }) + .collect(); + } finally { + jsc.setJobDescription(previousJobDescription); + } + + // Group column's metadata by file-paths of the files it belongs to + Map>> filePathToColumnMetadataMap = + colMinMaxInfos.stream() + .collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getFilePath)); + + JavaRDD allMetaDataRDD = + jsc.parallelize(new ArrayList<>(filePathToColumnMetadataMap.values()), 1) + .map(fileColumnsMetadata -> { + int colSize = fileColumnsMetadata.size(); + if (colSize == 0) { + return null; + } + + String filePath = fileColumnsMetadata.get(0).getFilePath(); + + List indexRow = new ArrayList<>(); + + // First columns of the Z-index's row is target file-path + indexRow.add(filePath); + + // For each column + orderedColumnSchemas.forEach(colSchema -> { + String colName = colSchema.name(); + + HoodieColumnRangeMetadata colMetadata = + fileColumnsMetadata.stream() + .filter(s -> s.getColumnName().trim().equalsIgnoreCase(colName)) + .findFirst() + .orElse(null); + + DataType colType = colSchema.dataType(); + if (colMetadata == null || colType == null) { + throw new HoodieException(String.format("Cannot collect min/max statistics for column (%s)", colSchema)); + } + + Pair minMaxValue = fetchMinMaxValues(colType, colMetadata); + + indexRow.add(minMaxValue.getLeft()); // min + indexRow.add(minMaxValue.getRight()); // max + indexRow.add(colMetadata.getNumNulls()); + }); + + return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow)); + }) + .filter(Objects::nonNull); + + StructType indexSchema = composeIndexSchema(orderedColumnSchemas); + + return sparkSession.createDataFrame(allMetaDataRDD, indexSchema); + } + + /** + *

+ * Updates state of the column-stats index by: + *

    + *
  1. Updating column-stats index with statistics for {@code sourceBaseFiles}, + * collecting corresponding column statistics from Parquet footers
  2. + *
  3. Merging newly built column-stats index table with the most recent one (if present + * and not preempted)
  4. + *
  5. Cleans up any residual index tables, that weren't cleaned up before
  6. + *
+ * + * @param sparkSession encompassing Spark session + * @param sourceTableSchema instance of {@link StructType} bearing source table's writer's schema + * @param sourceBaseFiles list of base-files to be indexed + * @param orderedCols target ordered columns + * @param indexFolderPath col-stats index folder path + * @param commitTime current operation commit instant + * @param completedCommits all previously completed commit instants + */ + public static void updateColumnStatsIndexFor( + @Nonnull SparkSession sparkSession, + @Nonnull StructType sourceTableSchema, + @Nonnull List sourceBaseFiles, + @Nonnull List orderedCols, + @Nonnull String indexFolderPath, + @Nonnull String commitTime, + @Nonnull List completedCommits + ) { + FileSystem fs = FSUtils.getFs(indexFolderPath, sparkSession.sparkContext().hadoopConfiguration()); + + // Compose new col-stats index table for the given source base files + Dataset newColStatsIndexDf = + buildColumnStatsTableFor( + sparkSession, + sourceBaseFiles, + orderedCols.stream() + .map(col -> sourceTableSchema.fields()[sourceTableSchema.fieldIndex(col)]) + .collect(Collectors.toList()) + ); + + try { + // + // Column Stats Index has the following folder structure: + // + // .hoodie/ + // ├── .colstatsindex/ + // │ ├── / + // │ │ ├── .parquet + // │ │ └── ... + // + // If index is currently empty (no persisted tables), we simply create one + // using clustering operation's commit instance as it's name + Path newIndexTablePath = new Path(indexFolderPath, commitTime); + + if (!fs.exists(new Path(indexFolderPath))) { + newColStatsIndexDf.repartition(1) + .write() + .format("parquet") + .mode("overwrite") + .save(newIndexTablePath.toString()); + return; + } + + // Filter in all index tables (w/in {@code .zindex} folder) + List allIndexTables = + Arrays.stream( + fs.listStatus(new Path(indexFolderPath)) + ) + .filter(FileStatus::isDirectory) + .map(f -> f.getPath().getName()) + .collect(Collectors.toList()); + + // Compile list of valid index tables that were produced as part + // of previously successfully committed iterations + List validIndexTables = + allIndexTables.stream() + .filter(completedCommits::contains) + .sorted() + .collect(Collectors.toList()); + + List tablesToCleanup = + allIndexTables.stream() + .filter(f -> !completedCommits.contains(f)) + .collect(Collectors.toList()); + + Dataset finalColStatsIndexDf; + + // Before writing out new version of the col-stats-index table we need to merge it + // with the most recent one that were successfully persisted previously + if (validIndexTables.isEmpty()) { + finalColStatsIndexDf = newColStatsIndexDf; + } else { + // NOTE: That Parquet schema might deviate from the original table schema (for ex, + // by upcasting "short" to "integer" types, etc), and hence we need to re-adjust it + // prior to merging, since merging might fail otherwise due to schemas incompatibility + finalColStatsIndexDf = + tryMergeMostRecentIndexTableInto( + sparkSession, + newColStatsIndexDf, + // Load current most recent col-stats-index table + sparkSession.read().load( + new Path(indexFolderPath, validIndexTables.get(validIndexTables.size() - 1)).toString() + ) + ); + + // Clean up all index tables (after creation of the new index) + tablesToCleanup.addAll(validIndexTables); + } + + // Persist new col-stats-index table + finalColStatsIndexDf + .repartition(1) + .write() + .format("parquet") + .save(newIndexTablePath.toString()); + + // Clean up residual col-stats-index tables that have might have been dangling since + // previous iterations (due to intermittent failures during previous clean up) + tablesToCleanup.forEach(f -> { + try { + fs.delete(new Path(indexFolderPath, f), true); + } catch (IOException ie) { + // NOTE: Exception is deliberately swallowed to not affect overall clustering operation, + // since failing col-stats-index table will be attempted to be cleaned up upon subsequent + // clustering iteration + LOG.warn(String.format("Failed to cleanup residual col-stats-index table: %s", f), ie); + } + }); + } catch (IOException e) { + LOG.error("Failed to build new col-stats-index table", e); + throw new HoodieException("Failed to build new col-stats-index table", e); + } + } + + @Nonnull + private static Dataset tryMergeMostRecentIndexTableInto( + @Nonnull SparkSession sparkSession, + @Nonnull Dataset newIndexTableDf, + @Nonnull Dataset existingIndexTableDf + ) { + // NOTE: If new col-stats index table schema is incompatible with that one of existing table + // that is most likely due to changing settings of list of Z-ordered columns, that + // occurred since last index table have been persisted. + // + // In that case, we simply drop existing index table and just persist the new one; + // + // Also note that we're checking compatibility of _old_ index-table with new one and that + // COMPATIBILITY OPERATION DOES NOT COMMUTE (ie if A is compatible w/ B, + // B might not necessarily be compatible w/ A) + if (!areCompatible(existingIndexTableDf.schema(), newIndexTableDf.schema())) { + return newIndexTableDf; + } + + String randomSuffix = UUID.randomUUID().toString().replace("-", ""); + + String existingIndexTempTableName = "existingIndexTable_" + randomSuffix; + String newIndexTempTableName = "newIndexTable_" + randomSuffix; + + existingIndexTableDf.registerTempTable(existingIndexTempTableName); + newIndexTableDf.registerTempTable(newIndexTempTableName); + + List newTableColumns = Arrays.asList(newIndexTableDf.schema().fieldNames()); + + // Create merged table by doing full-out join + return sparkSession.sql(createIndexMergeSql(existingIndexTempTableName, newIndexTempTableName, newTableColumns)); + } + + /** + * @VisibleForTesting + */ + @Nonnull + public static StructType composeIndexSchema(@Nonnull List zorderedColumnsSchemas) { + List schema = new ArrayList<>(); + schema.add(new StructField(Z_INDEX_FILE_COLUMN_NAME, StringType$.MODULE$, true, Metadata.empty())); + zorderedColumnsSchemas.forEach(colSchema -> { + schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MIN_VALUE_STAT_NAME, colSchema.dataType())); + schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MAX_VALUE_STAT_NAME, colSchema.dataType())); + schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_NUM_NULLS_STAT_NAME, LongType$.MODULE$)); + }); + return StructType$.MODULE$.apply(schema); + } + + private static StructField composeColumnStatStructType(String col, String statName, DataType dataType) { + return new StructField(composeZIndexColName(col, statName), dataType, true, Metadata.empty()); + } + + private static String composeZIndexColName(String col, String statName) { + // TODO add escaping for + return String.format("%s_%s", col, statName); + } + + private static Pair + fetchMinMaxValues( + @Nonnull DataType colType, + @Nonnull HoodieColumnRangeMetadata colMetadata) { + if (colType instanceof IntegerType) { + return Pair.of( + new Integer(colMetadata.getMinValue().toString()), + new Integer(colMetadata.getMaxValue().toString()) + ); + } else if (colType instanceof DoubleType) { + return Pair.of( + new Double(colMetadata.getMinValue().toString()), + new Double(colMetadata.getMaxValue().toString()) + ); + } else if (colType instanceof StringType) { + return Pair.of( + new String(((Binary) colMetadata.getMinValue()).getBytes()), + new String(((Binary) colMetadata.getMaxValue()).getBytes()) + ); + } else if (colType instanceof DecimalType) { + return Pair.of( + new BigDecimal(colMetadata.getMinValue().toString()), + new BigDecimal(colMetadata.getMaxValue().toString())); + } else if (colType instanceof DateType) { + return Pair.of( + java.sql.Date.valueOf(colMetadata.getMinValue().toString()), + java.sql.Date.valueOf(colMetadata.getMaxValue().toString())); + } else if (colType instanceof LongType) { + return Pair.of( + new Long(colMetadata.getMinValue().toString()), + new Long(colMetadata.getMaxValue().toString())); + } else if (colType instanceof ShortType) { + return Pair.of( + new Short(colMetadata.getMinValue().toString()), + new Short(colMetadata.getMaxValue().toString())); + } else if (colType instanceof FloatType) { + return Pair.of( + new Float(colMetadata.getMinValue().toString()), + new Float(colMetadata.getMaxValue().toString())); + } else if (colType instanceof BinaryType) { + return Pair.of( + ((Binary) colMetadata.getMinValue()).getBytes(), + ((Binary) colMetadata.getMaxValue()).getBytes()); + } else if (colType instanceof BooleanType) { + return Pair.of( + Boolean.valueOf(colMetadata.getMinValue().toString()), + Boolean.valueOf(colMetadata.getMaxValue().toString())); + } else if (colType instanceof ByteType) { + return Pair.of( + Byte.valueOf(colMetadata.getMinValue().toString()), + Byte.valueOf(colMetadata.getMaxValue().toString())); + } else { + throw new HoodieException(String.format("Not support type: %s", colType)); + } + } + + /** + * @VisibleForTesting + */ + @Nonnull + static String createIndexMergeSql( + @Nonnull String originalIndexTable, + @Nonnull String newIndexTable, + @Nonnull List columns + ) { + StringBuilder selectBody = new StringBuilder(); + + for (int i = 0; i < columns.size(); ++i) { + String col = columns.get(i); + String originalTableColumn = String.format("%s.%s", originalIndexTable, col); + String newTableColumn = String.format("%s.%s", newIndexTable, col); + + selectBody.append( + // NOTE: We prefer values from the new index table, and fallback to the original one only + // in case it does not contain statistics for the given file path + String.format("if (%s is null, %s, %s) AS %s", newTableColumn, originalTableColumn, newTableColumn, col) + ); + + if (i < columns.size() - 1) { + selectBody.append(", "); + } + } + + return String.format( + "SELECT %s FROM %s FULL JOIN %s ON %s = %s", + selectBody, + originalIndexTable, + newIndexTable, + String.format("%s.%s", originalIndexTable, columns.get(0)), + String.format("%s.%s", newIndexTable, columns.get(0)) + ); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index 0317b961f3d2..e940c0b8211c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -567,7 +567,7 @@ public boolean rollbackCommit(String instantTime) { BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) { final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS); - Long rollbackTime = HoodieActiveTimeline.parseInstantTime(instantTime).getTime(); + Long rollbackTime = HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(); Long currentTime = new Date().getTime(); Scan scan = new Scan(); scan.addFamily(SYSTEM_COLUMN_FAMILY); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index 60765b05da74..b84a8abdcc79 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -47,9 +47,9 @@ public SimpleKeyGenerator(TypedProperties props) { SimpleKeyGenerator(TypedProperties props, String recordKeyField, String partitionPathField) { super(props); this.recordKeyFields = recordKeyField == null - ? Collections.emptyList() - : Collections.singletonList(recordKeyField); - this.partitionPathFields = Collections.singletonList(partitionPathField); + ? Collections.emptyList() : Collections.singletonList(recordKeyField); + this.partitionPathFields = partitionPathField == null + ? Collections.emptyList() : Collections.singletonList(partitionPathField); simpleAvroKeyGenerator = new SimpleAvroKeyGenerator(props, recordKeyField, partitionPathField); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 0673ca976998..5329751b2018 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -122,6 +122,7 @@ protected void initialize(HoodieEngineContext eng } protected void commit(HoodieData hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) { + ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); JavaRDD records = (JavaRDD) hoodieDataRecords.get(); JavaRDD recordRDD = prepRecords(records, partitionName, 1); @@ -153,7 +154,8 @@ protected void commit(HoodieData hoodieDataRecords, String partiti metadataMetaClient.reloadActiveTimeline(); if (canTriggerTableService) { compactIfNecessary(writeClient, instantTime); - doClean(writeClient, instantTime); + cleanIfNecessary(writeClient, instantTime); + writeClient.archive(); } } @@ -167,7 +169,7 @@ protected void commit(HoodieData hoodieDataRecords, String partiti * The record is tagged with respective file slice's location based on its record key. */ private JavaRDD prepRecords(JavaRDD recordsRDD, String partitionName, int numFileGroups) { - List fileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName); + List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, partitionName); ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups)); return recordsRDD.map(r -> { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/sort/SpaceCurveSortingHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/sort/SpaceCurveSortingHelper.java new file mode 100644 index 000000000000..8ebc032a18a0 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/sort/SpaceCurveSortingHelper.java @@ -0,0 +1,260 @@ +/* + * 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 org.apache.hudi.sort; + +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.optimize.HilbertCurveUtils; +import org.apache.hudi.common.util.BinaryUtil; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Row$; +import org.apache.spark.sql.hudi.execution.RangeSampleSort$; +import org.apache.spark.sql.hudi.execution.ByteArraySorting; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; +import org.apache.spark.sql.types.TimestampType; +import org.davidmoten.hilbert.HilbertCurve; +import scala.collection.JavaConversions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class SpaceCurveSortingHelper { + + private static final Logger LOG = LogManager.getLogger(SpaceCurveSortingHelper.class); + + /** + * Orders provided {@link Dataset} by mapping values of the provided list of columns + * {@code orderByCols} onto a specified space curve (Z-curve, Hilbert, etc) + * + *

+ * NOTE: Only support base data-types: long,int,short,double,float,string,timestamp,decimal,date,byte. + * This method is more effective than {@link #orderDataFrameBySamplingValues} leveraging + * data sampling instead of direct mapping + * + * @param df Spark {@link Dataset} holding data to be ordered + * @param orderByCols list of columns to be ordered by + * @param targetPartitionCount target number of output partitions + * @param layoutOptStrategy target layout optimization strategy + * @return a {@link Dataset} holding data ordered by mapping tuple of values from provided columns + * onto a specified space-curve + */ + public static Dataset orderDataFrameByMappingValues( + Dataset df, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + List orderByCols, + int targetPartitionCount + ) { + Map columnsMap = + Arrays.stream(df.schema().fields()) + .collect(Collectors.toMap(StructField::name, Function.identity())); + + List checkCols = + orderByCols.stream() + .filter(columnsMap::containsKey) + .collect(Collectors.toList()); + + if (orderByCols.size() != checkCols.size()) { + LOG.error(String.format("Trying to ordering over a column(s) not present in the schema (%s); skipping", CollectionUtils.diff(orderByCols, checkCols))); + return df; + } + + // In case when there's just one column to be ordered by, we can skip space-curve + // ordering altogether (since it will match linear ordering anyway) + if (orderByCols.size() == 1) { + String orderByColName = orderByCols.get(0); + LOG.debug(String.format("Single column to order by (%s), skipping space-curve ordering", orderByColName)); + + // TODO validate if we need Spark to re-partition + return df.repartitionByRange(targetPartitionCount, new Column(orderByColName)); + } + + int fieldNum = df.schema().fields().length; + + Map fieldMap = + orderByCols.stream() + .collect( + Collectors.toMap(e -> Arrays.asList(df.schema().fields()).indexOf(columnsMap.get(e)), columnsMap::get)); + + JavaRDD sortedRDD; + switch (layoutOptStrategy) { + case ZORDER: + sortedRDD = createZCurveSortedRDD(df.toJavaRDD(), fieldMap, fieldNum, targetPartitionCount); + break; + case HILBERT: + sortedRDD = createHilbertSortedRDD(df.toJavaRDD(), fieldMap, fieldNum, targetPartitionCount); + break; + default: + throw new IllegalArgumentException(String.format("new only support z-order/hilbert optimize but find: %s", layoutOptStrategy)); + } + + // Compose new {@code StructType} for ordered RDDs + StructType newStructType = composeOrderedRDDStructType(df.schema()); + + return df.sparkSession() + .createDataFrame(sortedRDD, newStructType) + .drop("Index"); + } + + private static StructType composeOrderedRDDStructType(StructType schema) { + return StructType$.MODULE$.apply( + CollectionUtils.combine( + Arrays.asList(schema.fields()), + Arrays.asList(new StructField("Index", BinaryType$.MODULE$, true, Metadata.empty())) + ) + ); + } + + private static JavaRDD createZCurveSortedRDD(JavaRDD originRDD, Map fieldMap, int fieldNum, int fileNum) { + return originRDD.map(row -> { + List zBytesList = fieldMap.entrySet().stream().map(entry -> { + int index = entry.getKey(); + StructField field = entry.getValue(); + DataType dataType = field.dataType(); + if (dataType instanceof LongType) { + return BinaryUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getLong(index)); + } else if (dataType instanceof DoubleType) { + return BinaryUtil.doubleTo8Byte(row.isNullAt(index) ? Double.MAX_VALUE : row.getDouble(index)); + } else if (dataType instanceof IntegerType) { + return BinaryUtil.intTo8Byte(row.isNullAt(index) ? Integer.MAX_VALUE : row.getInt(index)); + } else if (dataType instanceof FloatType) { + return BinaryUtil.doubleTo8Byte(row.isNullAt(index) ? Float.MAX_VALUE : row.getFloat(index)); + } else if (dataType instanceof StringType) { + return BinaryUtil.utf8To8Byte(row.isNullAt(index) ? "" : row.getString(index)); + } else if (dataType instanceof DateType) { + return BinaryUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDate(index).getTime()); + } else if (dataType instanceof TimestampType) { + return BinaryUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getTimestamp(index).getTime()); + } else if (dataType instanceof ByteType) { + return BinaryUtil.byteTo8Byte(row.isNullAt(index) ? Byte.MAX_VALUE : row.getByte(index)); + } else if (dataType instanceof ShortType) { + return BinaryUtil.intTo8Byte(row.isNullAt(index) ? Short.MAX_VALUE : row.getShort(index)); + } else if (dataType instanceof DecimalType) { + return BinaryUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDecimal(index).longValue()); + } else if (dataType instanceof BooleanType) { + boolean value = row.isNullAt(index) ? false : row.getBoolean(index); + return BinaryUtil.intTo8Byte(value ? 1 : 0); + } else if (dataType instanceof BinaryType) { + return BinaryUtil.paddingTo8Byte(row.isNullAt(index) ? new byte[] {0} : (byte[]) row.get(index)); + } + return null; + }).filter(f -> f != null).collect(Collectors.toList()); + byte[][] zBytes = new byte[zBytesList.size()][]; + for (int i = 0; i < zBytesList.size(); i++) { + zBytes[i] = zBytesList.get(i); + } + List zVaules = new ArrayList<>(); + zVaules.addAll(scala.collection.JavaConverters.bufferAsJavaListConverter(row.toSeq().toBuffer()).asJava()); + zVaules.add(BinaryUtil.interleaving(zBytes, 8)); + return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(zVaules)); + }) + .sortBy(f -> new ByteArraySorting((byte[]) f.get(fieldNum)), true, fileNum); + } + + private static JavaRDD createHilbertSortedRDD(JavaRDD originRDD, Map fieldMap, int fieldNum, int fileNum) { + return originRDD.mapPartitions(rows -> { + HilbertCurve hilbertCurve = HilbertCurve.bits(63).dimensions(fieldMap.size()); + return new Iterator() { + + @Override + public boolean hasNext() { + return rows.hasNext(); + } + + @Override + public Row next() { + Row row = rows.next(); + List longList = fieldMap.entrySet().stream().map(entry -> { + int index = entry.getKey(); + StructField field = entry.getValue(); + DataType dataType = field.dataType(); + if (dataType instanceof LongType) { + return row.isNullAt(index) ? Long.MAX_VALUE : row.getLong(index); + } else if (dataType instanceof DoubleType) { + return row.isNullAt(index) ? Long.MAX_VALUE : Double.doubleToLongBits(row.getDouble(index)); + } else if (dataType instanceof IntegerType) { + return row.isNullAt(index) ? Long.MAX_VALUE : (long)row.getInt(index); + } else if (dataType instanceof FloatType) { + return row.isNullAt(index) ? Long.MAX_VALUE : Double.doubleToLongBits((double) row.getFloat(index)); + } else if (dataType instanceof StringType) { + return row.isNullAt(index) ? Long.MAX_VALUE : BinaryUtil.convertStringToLong(row.getString(index)); + } else if (dataType instanceof DateType) { + return row.isNullAt(index) ? Long.MAX_VALUE : row.getDate(index).getTime(); + } else if (dataType instanceof TimestampType) { + return row.isNullAt(index) ? Long.MAX_VALUE : row.getTimestamp(index).getTime(); + } else if (dataType instanceof ByteType) { + return row.isNullAt(index) ? Long.MAX_VALUE : BinaryUtil.convertBytesToLong(new byte[] {row.getByte(index)}); + } else if (dataType instanceof ShortType) { + return row.isNullAt(index) ? Long.MAX_VALUE : (long)row.getShort(index); + } else if (dataType instanceof DecimalType) { + return row.isNullAt(index) ? Long.MAX_VALUE : row.getDecimal(index).longValue(); + } else if (dataType instanceof BooleanType) { + boolean value = row.isNullAt(index) ? false : row.getBoolean(index); + return value ? Long.MAX_VALUE : 0; + } else if (dataType instanceof BinaryType) { + return row.isNullAt(index) ? Long.MAX_VALUE : BinaryUtil.convertBytesToLong((byte[]) row.get(index)); + } + return null; + }).filter(f -> f != null).collect(Collectors.toList()); + + byte[] hilbertValue = HilbertCurveUtils.indexBytes( + hilbertCurve, longList.stream().mapToLong(l -> l).toArray(), 63); + List values = new ArrayList<>(); + values.addAll(scala.collection.JavaConverters.bufferAsJavaListConverter(row.toSeq().toBuffer()).asJava()); + values.add(hilbertValue); + return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(values)); + } + }; + }).sortBy(f -> new ByteArraySorting((byte[]) f.get(fieldNum)), true, fileNum); + } + + public static Dataset orderDataFrameBySamplingValues( + Dataset df, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + List orderByCols, + int targetPartitionCount + ) { + return RangeSampleSort$.MODULE$.sortDataFrameBySample(df, layoutOptStrategy, JavaConversions.asScalaBuffer(orderByCols), targetPartitionCount); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 0971b87c4467..aa9a924ed692 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -18,7 +18,8 @@ package org.apache.hudi.table; -import org.apache.hadoop.fs.Path; +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -37,13 +38,16 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; @@ -71,13 +75,17 @@ import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.ZCurveOptimizeHelper; import org.apache.spark.api.java.JavaRDD; -import scala.collection.JavaConversions; + +import javax.annotation.Nonnull; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -163,29 +171,60 @@ public HoodieWriteMetadata> insertOverwriteTable(HoodieEngi } @Override - public void updateStatistics(HoodieEngineContext context, List stats, String instantTime, Boolean isOptimizeOperation) { - // deal with z-order/hilbert statistic info - if (isOptimizeOperation) { - updateOptimizeOperationStatistics(context, stats, instantTime); - } + public void updateMetadataIndexes(@Nonnull HoodieEngineContext context, @Nonnull List stats, @Nonnull String instantTime) throws Exception { + updateColumnsStatsIndex(context, stats, instantTime); } - private void updateOptimizeOperationStatistics(HoodieEngineContext context, List stats, String instantTime) { - String cols = config.getClusteringSortColumns(); + private void updateColumnsStatsIndex( + @Nonnull HoodieEngineContext context, + @Nonnull List updatedFilesStats, + @Nonnull String instantTime + ) throws Exception { + String sortColsList = config.getClusteringSortColumns(); String basePath = metaClient.getBasePath(); - String indexPath = metaClient.getZindexPath(); - List validateCommits = metaClient.getCommitsTimeline() - .filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList()); - List touchFiles = stats.stream().map(s -> new Path(basePath, s.getPath()).toString()).collect(Collectors.toList()); - if (touchFiles.isEmpty() || cols.isEmpty() || indexPath.isEmpty()) { - LOG.warn("save nothing to index table"); + String indexPath = metaClient.getColumnStatsIndexPath(); + + List completedCommits = + metaClient.getCommitsTimeline() + .filterCompletedInstants() + .getInstants() + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + + List touchedFiles = + updatedFilesStats.stream() + .map(s -> new Path(basePath, s.getPath()).toString()) + .collect(Collectors.toList()); + + if (touchedFiles.isEmpty() || StringUtils.isNullOrEmpty(sortColsList) || StringUtils.isNullOrEmpty(indexPath)) { return; } + + LOG.info(String.format("Updating column-statistics index table (%s)", indexPath)); + + List sortCols = Arrays.stream(sortColsList.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + HoodieSparkEngineContext sparkEngineContext = (HoodieSparkEngineContext)context; - ZCurveOptimizeHelper.saveStatisticsInfo(sparkEngineContext - .getSqlContext().sparkSession().read().load(JavaConversions.asScalaBuffer(touchFiles)), - cols, indexPath, instantTime, validateCommits); - LOG.info(String.format("save statistic info sucessfully at commitTime: %s", instantTime)); + + // Fetch table schema to appropriately construct col-stats index schema + Schema tableWriteSchema = + HoodieAvroUtils.createHoodieWriteSchema( + new TableSchemaResolver(metaClient).getTableAvroSchemaWithoutMetadataFields() + ); + + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + sparkEngineContext.getSqlContext().sparkSession(), + AvroConversionUtils.convertAvroSchemaToStructType(tableWriteSchema), + touchedFiles, + sortCols, + indexPath, + instantTime, + completedCommits + ); + + LOG.info(String.format("Successfully updated column-statistics index at instant (%s)", instantTime)); } @Override @@ -230,8 +269,9 @@ public Option scheduleCleaning(HoodieEngineContext context, S @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, - HoodieInstant instantToRollback, boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index 9e053aaa0da4..75af5d0f685f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -156,8 +156,9 @@ public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { @Override public Option scheduleRollback(HoodieEngineContext context, String instantTime, - HoodieInstant instantToRollback, boolean skipTimelinePublish) { - return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute(); + HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers) { + return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish, + shouldRollbackUsingMarkers).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index f14d39c70064..35c9ab3a0fe9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.avro.specific.SpecificRecordBase; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; @@ -38,8 +39,6 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.action.HoodieWriteMetadata; - -import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; @@ -49,8 +48,7 @@ public abstract class HoodieSparkTable extends HoodieTable>, JavaRDD, JavaRDD> { - private boolean isMetadataAvailabilityUpdated = false; - private boolean isMetadataTableAvailable; + private volatile boolean isMetadataTableExists = false; protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); @@ -112,25 +110,25 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - public Option getMetadataWriter(Option actionMetadata) { - synchronized (this) { - if (!isMetadataAvailabilityUpdated) { - // This code assumes that if metadata availability is updated once it will not change. - // Please revisit this logic if that's not the case. This is done to avoid repeated calls to fs.exists(). - try { - isMetadataTableAvailable = config.isMetadataTableEnabled() - && metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()))); - } catch (IOException e) { - throw new HoodieMetadataException("Checking existence of metadata table failed", e); + public Option getMetadataWriter(String triggeringInstantTimestamp, + Option actionMetadata) { + if (config.isMetadataTableEnabled()) { + // Create the metadata table writer. First time after the upgrade this creation might trigger + // metadata table bootstrapping. Bootstrapping process could fail and checking the table + // existence after the creation is needed. + final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( + context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp)); + try { + if (isMetadataTableExists || metaClient.getFs().exists(new Path( + HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())))) { + isMetadataTableExists = true; + return Option.of(metadataWriter); } - isMetadataAvailabilityUpdated = true; + } catch (IOException e) { + throw new HoodieMetadataException("Checking existence of metadata table failed", e); } } - if (isMetadataTableAvailable) { - return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context, - actionMetadata, Option.empty())); - } else { - return Option.empty(); - } + + return Option.empty(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 8b353d64c4f5..3c876e75c28f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -77,6 +77,7 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; @@ -144,6 +145,9 @@ public HoodieBootstrapWriteMetadata execute() { Option metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); // if there are full bootstrap to be performed, perform that too Option fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); + // Delete the marker directory for the instant + WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); return new HoodieBootstrapWriteMetadata(metadataResult, fullBootstrapResult); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java index 683d852131f6..81a0a74aee1d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java @@ -18,20 +18,14 @@ package org.apache.hudi.table.action.cluster; -import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.util.Map; @@ -40,8 +34,6 @@ public class SparkClusteringPlanActionExecutor extends BaseClusteringPlanActionExecutor>, JavaRDD, JavaRDD> { - private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanActionExecutor.class); - public SparkClusteringPlanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable>, JavaRDD, JavaRDD> table, @@ -49,33 +41,4 @@ public SparkClusteringPlanActionExecutor(HoodieEngineContext context, Option> extraMetadata) { super(context, config, table, instantTime, extraMetadata); } - - @Override - protected Option createClusteringPlan() { - LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); - Option lastClusteringInstant = table.getActiveTimeline().getCompletedReplaceTimeline().lastInstant(); - - int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) - .countInstants(); - if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { - LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering - + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " - + config.getInlineClusterMaxCommits()); - return Option.empty(); - } - - if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) { - LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering - + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " - + config.getAsyncClusterMaxCommits()); - return Option.empty(); - } - - LOG.info("Generating clustering plan for table " + config.getBasePath()); - ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) - ReflectionUtils.loadClass(config.getClusteringPlanStrategyClass(), table, context, config); - return strategy.generateClusteringPlan(); - } - } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java index 1e8b7bead48f..5b0224b0f005 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -84,7 +84,6 @@ public HoodieWriteMetadata> execute() { JavaRDD statuses = updateIndex(writeStatusRDD, writeMetadata); writeMetadata.setWriteStats(statuses.map(WriteStatus::getStat).collect()); writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeMetadata)); - validateWriteResult(writeMetadata); commitOnAutoCommit(writeMetadata); if (!writeMetadata.getCommitMetadata().isPresent()) { HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeMetadata.getWriteStats().get(), writeMetadata.getPartitionToReplaceFileIds(), diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 2bcd6d787a26..c551310bafdd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -77,6 +77,8 @@ import java.util.Set; import java.util.Map; +import static org.apache.hudi.common.util.ClusteringUtils.getAllFileGroupsInPendingClusteringPlans; + public abstract class BaseSparkCommitActionExecutor extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { @@ -113,15 +115,32 @@ private void initKeyGenIfNeeded(boolean populateMetaFields) { } private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { - if (config.isClusteringEnabled()) { - Set fileGroupsInPendingClustering = - table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); - UpdateStrategy updateStrategy = (UpdateStrategy)ReflectionUtils - .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); - return (JavaRDD>)updateStrategy.handleUpdate(inputRecordsRDD); - } else { - return inputRecordsRDD; + context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering"); + Set fileGroupsInPendingClustering = + table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); + UpdateStrategy updateStrategy = (UpdateStrategy) ReflectionUtils + .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); + Pair>, Set> recordsAndPendingClusteringFileGroups = + (Pair>, Set>) updateStrategy.handleUpdate(inputRecordsRDD); + Set fileGroupsWithUpdatesAndPendingClustering = recordsAndPendingClusteringFileGroups.getRight(); + if (fileGroupsWithUpdatesAndPendingClustering.isEmpty()) { + return recordsAndPendingClusteringFileGroups.getLeft(); + } + // there are filegroups pending clustering and receiving updates, so rollback the pending clustering instants + // there could be race condition, for example, if the clustering completes after instants are fetched but before rollback completed + if (config.isRollbackPendingClustering()) { + Set pendingClusteringInstantsToRollback = getAllFileGroupsInPendingClusteringPlans(table.getMetaClient()).entrySet().stream() + .filter(e -> fileGroupsWithUpdatesAndPendingClustering.contains(e.getKey())) + .map(Map.Entry::getValue) + .collect(Collectors.toSet()); + pendingClusteringInstantsToRollback.forEach(instant -> { + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + table.scheduleRollback(context, commitTime, instant, false, config.shouldRollbackUsingMarkers()); + table.rollback(context, commitTime, instant, true, true); + }); + table.getMetaClient().reloadActiveTimeline(); } + return recordsAndPendingClusteringFileGroups.getLeft(); } @Override @@ -148,6 +167,7 @@ public HoodieWriteMetadata> execute(JavaRDD // partition using the insert partitioner final Partitioner partitioner = getPartitioner(profile); + context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data"); JavaRDD> partitionedRecords = partition(inputRecordsRDDWithClusteringUpdate, partitioner); JavaRDD writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> { if (WriteOperationType.isChangingRecords(operationType)) { @@ -196,7 +216,10 @@ private Pair, WorkloadStat> buildProfile(JavaRDD layoutPartitionerClass = table.getStorageLayout().layoutPartitionerClass(); + if (layoutPartitionerClass.isPresent()) { + return getLayoutPartitioner(profile, layoutPartitionerClass.get()); + } else if (WriteOperationType.isChangingRecords(operationType)) { return getUpsertPartitioner(profile); } else { return getInsertPartitioner(profile); @@ -258,7 +281,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { String actionType = getCommitActionType(); - LOG.info("Committing " + instantTime + ", action Type " + actionType); + LOG.info("Committing " + instantTime + ", action Type " + actionType + ", operation Type " + operationType); result.setCommitted(true); result.setWriteStats(writeStats); // Finalize write @@ -285,7 +308,7 @@ protected Map> getPartitionToReplacedFileIds(HoodieWriteMet @SuppressWarnings("unchecked") protected Iterator> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { - UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; + SparkHoodiePartitioner upsertPartitioner = (SparkHoodiePartitioner) partitioner; BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); BucketType btype = binfo.bucketType; try { @@ -374,6 +397,12 @@ public Partitioner getInsertPartitioner(WorkloadProfile profile) { return getUpsertPartitioner(profile); } + public Partitioner getLayoutPartitioner(WorkloadProfile profile, String layoutPartitionerClass) { + return (Partitioner) ReflectionUtils.loadClass(layoutPartitionerClass, + new Class[] { WorkloadProfile.class, HoodieEngineContext.class, HoodieTable.class, HoodieWriteConfig.class }, + profile, context, table, config); + } + @Override protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java new file mode 100644 index 000000000000..71da2244db56 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java @@ -0,0 +1,136 @@ +/* + * 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 org.apache.hudi.table.action.commit; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.hudi.index.bucket.BucketIdentifier; +import scala.Tuple2; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.index.bucket.HoodieBucketIndex; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; + +/** + * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). + */ +public class SparkBucketIndexPartitioner> extends + SparkHoodiePartitioner { + + private final int numBuckets; + private final String indexKeyField; + private final int totalPartitionPaths; + private final List partitionPaths; + /** + * Helps get the RDD partition id, partition id is partition offset + bucket id. + * The partition offset is a multiple of the bucket num. + */ + private final Map partitionPathOffset; + + /** + * Partition path and file groups in it pair. Decide the file group an incoming update should go to. + */ + private Map> updatePartitionPathFileIds; + + public SparkBucketIndexPartitioner(WorkloadProfile profile, + HoodieEngineContext context, + HoodieTable table, + HoodieWriteConfig config) { + super(profile, table); + if (!(table.getIndex() instanceof HoodieBucketIndex)) { + throw new HoodieException( + " Bucket index partitioner should only be used by BucketIndex other than " + + table.getIndex().getClass().getSimpleName()); + } + this.numBuckets = ((HoodieBucketIndex) table.getIndex()).getNumBuckets(); + this.indexKeyField = config.getBucketIndexHashField(); + this.totalPartitionPaths = profile.getPartitionPaths().size(); + partitionPaths = new ArrayList<>(profile.getPartitionPaths()); + partitionPathOffset = new HashMap<>(); + int i = 0; + for (Object partitionPath : profile.getPartitionPaths()) { + partitionPathOffset.put(partitionPath.toString(), i); + i += numBuckets; + } + assignUpdates(profile); + } + + private void assignUpdates(WorkloadProfile profile) { + updatePartitionPathFileIds = new HashMap<>(); + // each update location gets a partition + Set> partitionStatEntries = profile.getPartitionPathStatMap() + .entrySet(); + for (Entry partitionStat : partitionStatEntries) { + if (!updatePartitionPathFileIds.containsKey(partitionStat.getKey())) { + updatePartitionPathFileIds.put(partitionStat.getKey(), new HashSet<>()); + } + for (Entry> updateLocEntry : + partitionStat.getValue().getUpdateLocationToCount().entrySet()) { + updatePartitionPathFileIds.get(partitionStat.getKey()).add(updateLocEntry.getKey()); + } + } + } + + @Override + public BucketInfo getBucketInfo(int bucketNumber) { + String partitionPath = partitionPaths.get(bucketNumber / numBuckets); + String bucketId = BucketIdentifier.bucketIdStr(bucketNumber % numBuckets); + Option fileIdOption = Option.fromJavaOptional(updatePartitionPathFileIds + .getOrDefault(partitionPath, Collections.emptySet()).stream() + .filter(e -> e.startsWith(bucketId)) + .findFirst()); + if (fileIdOption.isPresent()) { + return new BucketInfo(BucketType.UPDATE, fileIdOption.get(), partitionPath); + } else { + return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath); + } + } + + @Override + public int numPartitions() { + return totalPartitionPaths * numBuckets; + } + + @Override + public int getPartition(Object key) { + Tuple2> keyLocation = (Tuple2>) key; + String partitionPath = keyLocation._1.getPartitionPath(); + Option location = keyLocation._2; + int bucketId = location.isPresent() + ? BucketIdentifier.bucketIdFromFileId(location.get().getFileId()) + : BucketIdentifier.getBucketId(keyLocation._1, indexKeyField, numBuckets); + return partitionPathOffset.get(partitionPath) + bucketId; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java new file mode 100644 index 000000000000..4a5bff42153f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkHoodiePartitioner.java @@ -0,0 +1,50 @@ +/* + * 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 org.apache.hudi.table.action.commit; + +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.spark.Partitioner; + +/** + * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). + */ +public abstract class SparkHoodiePartitioner> extends Partitioner + implements org.apache.hudi.table.action.commit.Partitioner { + + /** + * Stat for the current workload. Helps in determining inserts, upserts etc. + */ + protected WorkloadProfile profile; + + protected final HoodieTable table; + + public SparkHoodiePartitioner(WorkloadProfile profile, HoodieTable table) { + this.profile = profile; + this.table = table; + } + + @Override + public int getNumPartitions() { + return numPartitions(); + } + + public abstract BucketInfo getBucketInfo(int bucketNumber); +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java index 25891e05a4dd..ba91fe160791 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java @@ -44,6 +44,6 @@ public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context, @Override public HoodieWriteMetadata> execute() { return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index bff85e7fe1c9..7a3549c9e9c4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -58,12 +58,14 @@ public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } @Override protected Partitioner getPartitioner(WorkloadProfile profile) { - return new SparkInsertOverwritePartitioner(profile, context, table, config); + return table.getStorageLayout().layoutPartitionerClass() + .map(c -> getLayoutPartitioner(profile, c)) + .orElse(new SparkInsertOverwritePartitioner(profile, context, table, config)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java index fe90212b0be1..c914384cb21e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java @@ -44,6 +44,6 @@ public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context, @Override public HoodieWriteMetadata> execute() { return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 35a8bddf94fb..6729da72d65e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -37,7 +37,6 @@ import org.apache.hudi.table.WorkloadStat; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; @@ -57,7 +56,7 @@ /** * Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition). */ -public class UpsertPartitioner> extends Partitioner { +public class UpsertPartitioner> extends SparkHoodiePartitioner { private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class); @@ -69,10 +68,6 @@ public class UpsertPartitioner> extends Partiti * Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into. */ private int totalBuckets = 0; - /** - * Stat for the current workload. Helps in determining inserts, upserts etc. - */ - private WorkloadProfile profile; /** * Helps decide which bucket an incoming update should go to. */ @@ -86,17 +81,14 @@ public class UpsertPartitioner> extends Partiti */ private HashMap bucketInfoMap; - protected final HoodieTable table; - protected final HoodieWriteConfig config; public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table, HoodieWriteConfig config) { + super(profile, table); updateLocationToBucket = new HashMap<>(); partitionPathToInsertBucketInfos = new HashMap<>(); bucketInfoMap = new HashMap<>(); - this.profile = profile; - this.table = table; this.config = config; assignUpdates(profile); assignInserts(profile, context); @@ -288,6 +280,10 @@ protected List getSmallFiles(String partitionPath) { return smallFileLocations; } + public List getBucketInfos() { + return Collections.unmodifiableList(new ArrayList<>(bucketInfoMap.values())); + } + public BucketInfo getBucketInfo(int bucketNumber) { return bucketInfoMap.get(bucketNumber); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java index c92cd928474c..3b3edd308457 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java @@ -74,8 +74,8 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { public Iterator> handleUpdate(String partitionPath, String fileId, Iterator> recordItr) throws IOException { LOG.info("Merging updates for commit " + instantTime + " for file " + fileId); - - if (!table.getIndex().canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + if (!table.getIndex().canIndexLogFiles() && mergeOnReadUpsertPartitioner != null + && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId); return super.handleUpdate(partitionPath, fileId, recordItr); } else { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java index fcaedee1127d..7dd91710d66e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java @@ -45,6 +45,6 @@ public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, @Override public HoodieWriteMetadata> execute() { return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false); + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java index 82aa08152405..c6f3901a352b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java @@ -44,6 +44,6 @@ public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, @Override public HoodieWriteMetadata execute() { return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, - config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true); + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java index a2bcbf5d2cc4..8dd3146f5161 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java @@ -26,15 +26,16 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; - import org.apache.hudi.table.action.commit.SmallFile; import org.apache.hudi.table.action.commit.UpsertPartitioner; +import javax.annotation.Nonnull; import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; @@ -51,68 +52,68 @@ public SparkUpsertDeltaCommitPartitioner(WorkloadProfile profile, HoodieSparkEng @Override protected List getSmallFiles(String partitionPath) { - - // smallFiles only for partitionPath - List smallFileLocations = new ArrayList<>(); - // Init here since this class (and member variables) might not have been initialized HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); - // Find out all eligible small file slices - if (!commitTimeline.empty()) { - HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - // find smallest file in partition and append to it - List allSmallFileSlices = new ArrayList<>(); - // If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to - // it. Doing this overtime for a partition, we ensure that we handle small file issues - if (!table.getIndex().canIndexLogFiles()) { - // TODO : choose last N small files since there can be multiple small files written to a single partition - // by different spark partitions in a single batch - Option smallFileSlice = Option.fromJavaOptional(table.getSliceView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false) - .filter( - fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getBaseFile().get().getFileSize() < config - .getParquetSmallFileLimit()) - .min((FileSlice left, FileSlice right) -> - left.getBaseFile().get().getFileSize() < right.getBaseFile().get().getFileSize() ? -1 : 1)); - if (smallFileSlice.isPresent()) { - allSmallFileSlices.add(smallFileSlice.get()); - } + if (commitTimeline.empty()) { + return Collections.emptyList(); + } + + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + + // Find out all eligible small file slices, looking for + // smallest file in the partition to append to + List smallFileSlicesCandidates = getSmallFileCandidates(partitionPath, latestCommitTime); + List smallFileLocations = new ArrayList<>(); + + // Create SmallFiles from the eligible file slices + for (FileSlice smallFileSlice : smallFileSlicesCandidates) { + SmallFile sf = new SmallFile(); + if (smallFileSlice.getBaseFile().isPresent()) { + // TODO : Move logic of file name, file id, base commit time handling inside file slice + String filename = smallFileSlice.getBaseFile().get().getFileName(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); + sf.sizeBytes = getTotalFileSize(smallFileSlice); + smallFileLocations.add(sf); } else { - // If we can index log files, we can add more inserts to log files for fileIds NOT including those under - // pending compaction - List allFileSlices = - table.getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false) - .collect(Collectors.toList()); - for (FileSlice fileSlice : allFileSlices) { - if (isSmallFile(fileSlice)) { - allSmallFileSlices.add(fileSlice); - } - } - } - // Create SmallFiles from the eligible file slices - for (FileSlice smallFileSlice : allSmallFileSlices) { - SmallFile sf = new SmallFile(); - if (smallFileSlice.getBaseFile().isPresent()) { - // TODO : Move logic of file name, file id, base commit time handling inside file slice - String filename = smallFileSlice.getBaseFile().get().getFileName(); - sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); - sf.sizeBytes = getTotalFileSize(smallFileSlice); - smallFileLocations.add(sf); - } else { - HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get(); - sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()), - FSUtils.getFileIdFromLogPath(logFile.getPath())); - sf.sizeBytes = getTotalFileSize(smallFileSlice); - smallFileLocations.add(sf); - } + HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get(); + sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()), + FSUtils.getFileIdFromLogPath(logFile.getPath())); + sf.sizeBytes = getTotalFileSize(smallFileSlice); + smallFileLocations.add(sf); } } return smallFileLocations; } + @Nonnull + private List getSmallFileCandidates(String partitionPath, HoodieInstant latestCommitInstant) { + // If we can index log files, we can add more inserts to log files for fileIds NOT including those under + // pending compaction + if (table.getIndex().canIndexLogFiles()) { + return table.getSliceView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false) + .filter(this::isSmallFile) + .collect(Collectors.toList()); + } + + // If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to + // it. Doing this overtime for a partition, we ensure that we handle small file issues + return table.getSliceView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false) + .filter( + fileSlice -> + // NOTE: We can not pad slices with existing log-files w/o compacting these, + // hence skipping + fileSlice.getLogFiles().count() < 1 + && fileSlice.getBaseFile().get().getFileSize() < config.getParquetSmallFileLimit()) + .sorted(Comparator.comparing(fileSlice -> fileSlice.getBaseFile().get().getFileSize())) + .limit(config.getSmallFileGroupCandidatesLimit()) + .collect(Collectors.toList()); + } + public List getSmallFileIds() { - return (List) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId()) + return smallFiles.stream().map(smallFile -> smallFile.location.getFileId()) .collect(Collectors.toList()); } @@ -132,8 +133,12 @@ private boolean isSmallFile(FileSlice fileSlice) { // TODO (NA) : Make this static part of utility public long convertLogFilesSizeToExpectedParquetSize(List hoodieLogFiles) { - long totalSizeOfLogFiles = hoodieLogFiles.stream().map(HoodieLogFile::getFileSize) - .filter(size -> size > 0).reduce(Long::sum).orElse(0L); + long totalSizeOfLogFiles = + hoodieLogFiles.stream() + .map(HoodieLogFile::getFileSize) + .filter(size -> size > 0) + .reduce(Long::sum) + .orElse(0L); // Here we assume that if there is no base parquet file, all log files contain only inserts. // We can then just get the parquet equivalent size of these log files, compare that with // {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/DataTypeUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/DataTypeUtils.java new file mode 100644 index 000000000000..bf800536e98e --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/DataTypeUtils.java @@ -0,0 +1,148 @@ +/* + * 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 org.apache.hudi.util; + +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.ByteType$; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType$; +import org.apache.spark.sql.types.FloatType$; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType$; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.VarcharType$; + +import javax.annotation.Nonnull; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +public class DataTypeUtils { + + private static Map, Set>> sparkPrimitiveTypesCompatibilityMap = + new HashMap, Set>>() {{ + + // Integral types + put(ShortType$.class, + newHashSet(ByteType$.class, ShortType$.class)); + put(IntegerType$.class, + newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class)); + put(LongType$.class, + newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class, LongType$.class)); + + // Float types + put(DoubleType$.class, + newHashSet(FloatType$.class, DoubleType$.class)); + + // String types + put(StringType$.class, + newHashSet(VarcharType$.class, StringType$.class)); + }}; + + /** + * Validates whether one {@link StructType} is compatible w/ the other one. + * Compatibility rules are defined like following: types A and B are considered + * compatible iff + * + *
    + *
  1. A and B are identical
  2. + *
  3. All values comprising A domain are contained w/in B domain (for ex, {@code ShortType} + * in this sense is compatible w/ {@code IntegerType})
  4. + *
+ * + * @param left operand + * @param right operand + * @return true if {@code left} instance of {@link StructType} is compatible w/ the {@code right} + */ + public static boolean areCompatible(@Nonnull DataType left, @Nonnull DataType right) { + // First, check if types are equal + if (Objects.equals(left, right)) { + return true; + } + + // If not, check whether both are instances of {@code StructType} that + // should be matched structurally + if (left instanceof StructType && right instanceof StructType) { + return areCompatible((StructType) left, (StructType) right); + } + + // If not, simply check if those data-types constitute compatibility + // relationship outlined above; otherwise return false + return sparkPrimitiveTypesCompatibilityMap.getOrDefault(left.getClass(), Collections.emptySet()) + .contains(right.getClass()); + } + + private static boolean areCompatible(@Nonnull StructType left, @Nonnull StructType right) { + StructField[] oneSchemaFields = left.fields(); + StructField[] anotherSchemaFields = right.fields(); + + if (oneSchemaFields.length != anotherSchemaFields.length) { + return false; + } + + for (int i = 0; i < oneSchemaFields.length; ++i) { + StructField oneField = oneSchemaFields[i]; + StructField anotherField = anotherSchemaFields[i]; + // NOTE: Metadata is deliberately omitted from comparison + if (!Objects.equals(oneField.name(), anotherField.name()) + || !areCompatible(oneField.dataType(), anotherField.dataType()) + || oneField.nullable() != anotherField.nullable()) { + return false; + } + } + + return true; + } + + private static HashSet newHashSet(T... ts) { + return new HashSet<>(Arrays.asList(ts)); + } + + /** + * Try to find current sparktype whether contains that DecimalType which's scale < Decimal.MAX_LONG_DIGITS(). + * + * @param sparkType spark schema. + * @return found result. + */ + public static boolean foundSmallPrecisionDecimalType(DataType sparkType) { + if (sparkType instanceof StructType) { + StructField[] fields = ((StructType) sparkType).fields(); + return Arrays.stream(fields).anyMatch(f -> foundSmallPrecisionDecimalType(f.dataType())); + } else if (sparkType instanceof MapType) { + MapType map = (MapType) sparkType; + return foundSmallPrecisionDecimalType(map.keyType()) || foundSmallPrecisionDecimalType(map.valueType()); + } else if (sparkType instanceof ArrayType) { + return foundSmallPrecisionDecimalType(((ArrayType) sparkType).elementType()); + } else if (sparkType instanceof DecimalType) { + DecimalType decimalType = (DecimalType) sparkType; + return decimalType.precision() < Decimal.MAX_LONG_DIGITS(); + } + return false; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/spark/ZCurveOptimizeHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/spark/ZCurveOptimizeHelper.java deleted file mode 100644 index 7ba1c9465bfd..000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/spark/ZCurveOptimizeHelper.java +++ /dev/null @@ -1,355 +0,0 @@ -/* - * 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 org.apache.spark; - -import scala.collection.JavaConversions; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.HoodieSparkUtils$; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; -import org.apache.hudi.common.model.HoodieFileFormat; -import org.apache.hudi.common.util.BaseFileUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ParquetUtils; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.optimize.ZOrderingUtil; -import org.apache.parquet.io.api.Binary; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.Row$; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.hudi.execution.RangeSampleSort$; -import org.apache.spark.sql.hudi.execution.ZorderingBinarySort; -import org.apache.spark.sql.types.BinaryType; -import org.apache.spark.sql.types.BinaryType$; -import org.apache.spark.sql.types.BooleanType; -import org.apache.spark.sql.types.ByteType; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DateType; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.sql.types.DoubleType; -import org.apache.spark.sql.types.FloatType; -import org.apache.spark.sql.types.IntegerType; -import org.apache.spark.sql.types.LongType; -import org.apache.spark.sql.types.LongType$; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.ShortType; -import org.apache.spark.sql.types.StringType; -import org.apache.spark.sql.types.StringType$; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType$; -import org.apache.spark.sql.types.TimestampType; -import org.apache.spark.util.SerializableConfiguration; - -import java.io.IOException; -import java.math.BigDecimal; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class ZCurveOptimizeHelper { - - private static final String SPARK_JOB_DESCRIPTION = "spark.job.description"; - - /** - * Create z-order DataFrame directly - * first, map all base type data to byte[8], then create z-order DataFrame - * only support base type data. long,int,short,double,float,string,timestamp,decimal,date,byte - * this method is more effective than createZIndexDataFrameBySample - * - * @param df a spark DataFrame holds parquet files to be read. - * @param zCols z-sort cols - * @param fileNum spark partition num - * @return a dataFrame sorted by z-order. - */ - public static Dataset createZIndexedDataFrameByMapValue(Dataset df, List zCols, int fileNum) { - Map columnsMap = Arrays.stream(df.schema().fields()).collect(Collectors.toMap(e -> e.name(), e -> e)); - int fieldNum = df.schema().fields().length; - List checkCols = zCols.stream().filter(f -> columnsMap.containsKey(f)).collect(Collectors.toList()); - if (zCols.size() != checkCols.size()) { - return df; - } - // only one col to sort, no need to use z-order - if (zCols.size() == 1) { - return df.repartitionByRange(fieldNum, org.apache.spark.sql.functions.col(zCols.get(0))); - } - Map fieldMap = zCols - .stream().collect(Collectors.toMap(e -> Arrays.asList(df.schema().fields()).indexOf(columnsMap.get(e)), e -> columnsMap.get(e))); - // z-sort - JavaRDD sortedRdd = df.toJavaRDD().map(row -> { - List zBytesList = fieldMap.entrySet().stream().map(entry -> { - int index = entry.getKey(); - StructField field = entry.getValue(); - DataType dataType = field.dataType(); - if (dataType instanceof LongType) { - return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getLong(index)); - } else if (dataType instanceof DoubleType) { - return ZOrderingUtil.doubleTo8Byte(row.isNullAt(index) ? Double.MAX_VALUE : row.getDouble(index)); - } else if (dataType instanceof IntegerType) { - return ZOrderingUtil.intTo8Byte(row.isNullAt(index) ? Integer.MAX_VALUE : row.getInt(index)); - } else if (dataType instanceof FloatType) { - return ZOrderingUtil.doubleTo8Byte(row.isNullAt(index) ? Float.MAX_VALUE : row.getFloat(index)); - } else if (dataType instanceof StringType) { - return ZOrderingUtil.utf8To8Byte(row.isNullAt(index) ? "" : row.getString(index)); - } else if (dataType instanceof DateType) { - return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDate(index).getTime()); - } else if (dataType instanceof TimestampType) { - return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getTimestamp(index).getTime()); - } else if (dataType instanceof ByteType) { - return ZOrderingUtil.byteTo8Byte(row.isNullAt(index) ? Byte.MAX_VALUE : row.getByte(index)); - } else if (dataType instanceof ShortType) { - return ZOrderingUtil.intTo8Byte(row.isNullAt(index) ? Short.MAX_VALUE : row.getShort(index)); - } else if (dataType instanceof DecimalType) { - return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDecimal(index).longValue()); - } else if (dataType instanceof BooleanType) { - boolean value = row.isNullAt(index) ? false : row.getBoolean(index); - return ZOrderingUtil.intTo8Byte(value ? 1 : 0); - } else if (dataType instanceof BinaryType) { - return ZOrderingUtil.paddingTo8Byte(row.isNullAt(index) ? new byte[] {0} : (byte[]) row.get(index)); - } - return null; - }).filter(f -> f != null).collect(Collectors.toList()); - byte[][] zBytes = new byte[zBytesList.size()][]; - for (int i = 0; i < zBytesList.size(); i++) { - zBytes[i] = zBytesList.get(i); - } - List zVaules = new ArrayList<>(); - zVaules.addAll(scala.collection.JavaConverters.bufferAsJavaListConverter(row.toSeq().toBuffer()).asJava()); - zVaules.add(ZOrderingUtil.interleaving(zBytes, 8)); - return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(zVaules)); - }).sortBy(f -> new ZorderingBinarySort((byte[]) f.get(fieldNum)), true, fileNum); - - // create new StructType - List newFields = new ArrayList<>(); - newFields.addAll(Arrays.asList(df.schema().fields())); - newFields.add(new StructField("zIndex", BinaryType$.MODULE$, true, Metadata.empty())); - - // create new DataFrame - return df.sparkSession().createDataFrame(sortedRdd, StructType$.MODULE$.apply(newFields)).drop("zIndex"); - } - - public static Dataset createZIndexedDataFrameByMapValue(Dataset df, String zCols, int fileNum) { - if (zCols == null || zCols.isEmpty() || fileNum <= 0) { - return df; - } - return createZIndexedDataFrameByMapValue(df, - Arrays.stream(zCols.split(",")).map(f -> f.trim()).collect(Collectors.toList()), fileNum); - } - - public static Dataset createZIndexedDataFrameBySample(Dataset df, List zCols, int fileNum) { - return RangeSampleSort$.MODULE$.sortDataFrameBySample(df, JavaConversions.asScalaBuffer(zCols), fileNum); - } - - public static Dataset createZIndexedDataFrameBySample(Dataset df, String zCols, int fileNum) { - if (zCols == null || zCols.isEmpty() || fileNum <= 0) { - return df; - } - return createZIndexedDataFrameBySample(df, Arrays.stream(zCols.split(",")).map(f -> f.trim()).collect(Collectors.toList()), fileNum); - } - - /** - * Parse min/max statistics stored in parquet footers for z-sort cols. - * no support collect statistics from timeStampType, since parquet file has not collect the statistics for timeStampType. - * to do adapt for rfc-27 - * - * @param df a spark DataFrame holds parquet files to be read. - * @param cols z-sort cols - * @return a dataFrame holds all statistics info. - */ - public static Dataset getMinMaxValue(Dataset df, List cols) { - Map columnsMap = Arrays.stream(df.schema().fields()).collect(Collectors.toMap(e -> e.name(), e -> e.dataType())); - - List scanFiles = Arrays.asList(df.inputFiles()); - SparkContext sc = df.sparkSession().sparkContext(); - JavaSparkContext jsc = new JavaSparkContext(sc); - - SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration()); - int numParallelism = (scanFiles.size() / 3 + 1); - List> colMinMaxInfos = new ArrayList<>(); - String previousJobDescription = sc.getLocalProperty(SPARK_JOB_DESCRIPTION); - try { - String description = "Listing parquet column statistics"; - jsc.setJobDescription(description); - colMinMaxInfos = jsc.parallelize(scanFiles, numParallelism).mapPartitions(paths -> { - Configuration conf = serializableConfiguration.value(); - ParquetUtils parquetUtils = (ParquetUtils) BaseFileUtils.getInstance(HoodieFileFormat.PARQUET); - List>> results = new ArrayList<>(); - while (paths.hasNext()) { - String path = paths.next(); - results.add(parquetUtils.readRangeFromParquetMetadata(conf, new Path(path), cols)); - } - return results.stream().flatMap(f -> f.stream()).iterator(); - }).collect(); - } finally { - jsc.setJobDescription(previousJobDescription); - } - - Map>> fileToStatsListMap = colMinMaxInfos.stream().collect(Collectors.groupingBy(e -> e.getFilePath())); - JavaRDD allMetaDataRDD = jsc.parallelize(fileToStatsListMap.values().stream().collect(Collectors.toList()), 1).map(f -> { - int colSize = f.size(); - if (colSize == 0) { - return null; - } else { - List rows = new ArrayList<>(); - rows.add(f.get(0).getFilePath()); - cols.stream().forEach(col -> { - HoodieColumnRangeMetadata currentColRangeMetaData = - f.stream().filter(s -> s.getColumnName().trim().equalsIgnoreCase(col)).findFirst().orElse(null); - DataType colType = columnsMap.get(col); - if (currentColRangeMetaData == null || colType == null) { - throw new HoodieException(String.format("cannot collect min/max statistics for col: %s", col)); - } - if (colType instanceof IntegerType) { - rows.add(currentColRangeMetaData.getMinValue()); - rows.add(currentColRangeMetaData.getMaxValue()); - } else if (colType instanceof DoubleType) { - rows.add(currentColRangeMetaData.getMinValue()); - rows.add(currentColRangeMetaData.getMaxValue()); - } else if (colType instanceof StringType) { - String minString = new String(((Binary)currentColRangeMetaData.getMinValue()).getBytes()); - String maxString = new String(((Binary)currentColRangeMetaData.getMaxValue()).getBytes()); - rows.add(minString); - rows.add(maxString); - } else if (colType instanceof DecimalType) { - Double minDecimal = Double.parseDouble(currentColRangeMetaData.getStringifier().stringify(Long.valueOf(currentColRangeMetaData.getMinValue().toString()))); - Double maxDecimal = Double.parseDouble(currentColRangeMetaData.getStringifier().stringify(Long.valueOf(currentColRangeMetaData.getMaxValue().toString()))); - rows.add(BigDecimal.valueOf(minDecimal)); - rows.add(BigDecimal.valueOf(maxDecimal)); - } else if (colType instanceof DateType) { - rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getStringifier().stringify((int)currentColRangeMetaData.getMinValue()))); - rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getStringifier().stringify((int)currentColRangeMetaData.getMaxValue()))); - } else if (colType instanceof LongType) { - rows.add(currentColRangeMetaData.getMinValue()); - rows.add(currentColRangeMetaData.getMaxValue()); - } else if (colType instanceof ShortType) { - rows.add(Short.parseShort(currentColRangeMetaData.getMinValue().toString())); - rows.add(Short.parseShort(currentColRangeMetaData.getMaxValue().toString())); - } else if (colType instanceof FloatType) { - rows.add(currentColRangeMetaData.getMinValue()); - rows.add(currentColRangeMetaData.getMaxValue()); - } else if (colType instanceof BinaryType) { - rows.add(((Binary)currentColRangeMetaData.getMinValue()).getBytes()); - rows.add(((Binary)currentColRangeMetaData.getMaxValue()).getBytes()); - } else if (colType instanceof BooleanType) { - rows.add(currentColRangeMetaData.getMinValue()); - rows.add(currentColRangeMetaData.getMaxValue()); - } else if (colType instanceof ByteType) { - rows.add(Byte.valueOf(currentColRangeMetaData.getMinValue().toString())); - rows.add(Byte.valueOf(currentColRangeMetaData.getMaxValue().toString())); - } else { - throw new HoodieException(String.format("Not support type: %s", colType)); - } - rows.add(currentColRangeMetaData.getNumNulls()); - }); - return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(rows)); - } - }).filter(f -> f != null); - List allMetaDataSchema = new ArrayList<>(); - allMetaDataSchema.add(new StructField("file", StringType$.MODULE$, true, Metadata.empty())); - cols.forEach(col -> { - allMetaDataSchema.add(new StructField(col + "_minValue", columnsMap.get(col), true, Metadata.empty())); - allMetaDataSchema.add(new StructField(col + "_maxValue", columnsMap.get(col), true, Metadata.empty())); - allMetaDataSchema.add(new StructField(col + "_num_nulls", LongType$.MODULE$, true, Metadata.empty())); - }); - return df.sparkSession().createDataFrame(allMetaDataRDD, StructType$.MODULE$.apply(allMetaDataSchema)); - } - - public static Dataset getMinMaxValue(Dataset df, String cols) { - List rawCols = Arrays.asList(cols.split(",")).stream().map(f -> f.trim()).collect(Collectors.toList()); - return getMinMaxValue(df, rawCols); - } - - /** - * Update statistics info. - * this method will update old index table by full out join, - * and save the updated table into a new index table based on commitTime. - * old index table will be cleaned also. - * - * @param df a spark DataFrame holds parquet files to be read. - * @param cols z-sort cols. - * @param indexPath index store path. - * @param commitTime current operation commitTime. - * @param validateCommits all validate commits for current table. - * @return - */ - public static void saveStatisticsInfo(Dataset df, String cols, String indexPath, String commitTime, List validateCommits) { - Path savePath = new Path(indexPath, commitTime); - SparkSession spark = df.sparkSession(); - FileSystem fs = FSUtils.getFs(indexPath, spark.sparkContext().hadoopConfiguration()); - Dataset statisticsDF = ZCurveOptimizeHelper.getMinMaxValue(df, cols); - // try to find last validate index table from index path - try { - if (fs.exists(new Path(indexPath))) { - List allIndexTables = Arrays - .stream(fs.listStatus(new Path(indexPath))).filter(f -> f.isDirectory()).map(f -> f.getPath().getName()).collect(Collectors.toList()); - List candidateIndexTables = allIndexTables.stream().filter(f -> validateCommits.contains(f)).sorted().collect(Collectors.toList()); - List residualTables = allIndexTables.stream().filter(f -> !validateCommits.contains(f)).collect(Collectors.toList()); - Option latestIndexData = Option.empty(); - if (!candidateIndexTables.isEmpty()) { - latestIndexData = Option.of(spark.read().load(new Path(indexPath, candidateIndexTables.get(candidateIndexTables.size() - 1)).toString())); - // clean old index table, keep at most 1 index table. - candidateIndexTables.remove(candidateIndexTables.size() - 1); - candidateIndexTables.forEach(f -> { - try { - fs.delete(new Path(indexPath, f)); - } catch (IOException ie) { - throw new HoodieException(ie); - } - }); - } - - // clean residualTables - // retried cluster operations at the same instant time is also considered, - // the residual files produced by retried are cleaned up before save statistics - // save statistics info to index table which named commitTime - residualTables.forEach(f -> { - try { - fs.delete(new Path(indexPath, f)); - } catch (IOException ie) { - throw new HoodieException(ie); - } - }); - - if (latestIndexData.isPresent() && latestIndexData.get().schema().equals(statisticsDF.schema())) { - // update the statistics info - String originalTable = "indexTable_" + java.util.UUID.randomUUID().toString().replace("-", ""); - String updateTable = "updateTable_" + java.util.UUID.randomUUID().toString().replace("-", ""); - latestIndexData.get().registerTempTable(originalTable); - statisticsDF.registerTempTable(updateTable); - // update table by full out join - List columns = Arrays.asList(statisticsDF.schema().fieldNames()); - spark.sql(HoodieSparkUtils$ - .MODULE$.createMergeSql(originalTable, updateTable, JavaConversions.asScalaBuffer(columns))).repartition(1).write().save(savePath.toString()); - } - } else { - statisticsDF.repartition(1).write().mode("overwrite").save(savePath.toString()); - } - } catch (IOException e) { - throw new HoodieException(e); - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala index 11cc4959442c..2900f08cc168 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionHelper.scala @@ -20,7 +20,6 @@ package org.apache.hudi import java.nio.ByteBuffer import java.sql.{Date, Timestamp} -import java.util import org.apache.avro.Conversions.DecimalConversion import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis} @@ -318,7 +317,7 @@ object AvroConversionHelper { } else { val sourceArray = item.asInstanceOf[Seq[Any]] val sourceArraySize = sourceArray.size - val targetList = new util.ArrayList[Any](sourceArraySize) + val targetList = new java.util.ArrayList[Any](sourceArraySize) var idx = 0 while (idx < sourceArraySize) { targetList.add(elementConverter(sourceArray(idx))) @@ -336,7 +335,7 @@ object AvroConversionHelper { if (item == null) { null } else { - val javaMap = new util.HashMap[String, Any]() + val javaMap = new java.util.HashMap[String, Any]() item.asInstanceOf[Map[String, Any]].foreach { case (key, value) => javaMap.put(key, valueConverter(value)) } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index ce3984327581..3e5402565c15 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -22,7 +22,9 @@ import java.util.Properties import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord + import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.model.HoodieRecord @@ -30,9 +32,9 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator} + import org.apache.spark.SPARK_VERSION import org.apache.spark.rdd.RDD -import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal} import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex} @@ -45,8 +47,14 @@ import scala.collection.JavaConverters.asScalaBufferConverter object HoodieSparkUtils extends SparkAdapterSupport { + def isSpark2: Boolean = SPARK_VERSION.startsWith("2.") + def isSpark3: Boolean = SPARK_VERSION.startsWith("3.") + def isSpark3_0: Boolean = SPARK_VERSION.startsWith("3.0") + + def isSpark3_2: Boolean = SPARK_VERSION.startsWith("3.2") + def getMetaSchema: StructType = { StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => { StructField(col, StringType, nullable = true) @@ -137,13 +145,13 @@ object HoodieSparkUtils extends SparkAdapterSupport { def createRddInternal(df: DataFrame, writeSchema: Schema, latestTableSchema: Schema, structName: String, recordNamespace: String) : RDD[GenericRecord] = { // Use the write avro schema to derive the StructType which has the correct nullability information - val writeDataType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType] + val writeDataType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema) val encoder = RowEncoder.apply(writeDataType).resolveAndBind() val deserializer = sparkAdapter.createSparkRowSerDe(encoder) // if records were serialized with old schema, but an evolved schema was passed in with latestTableSchema, we need // latestTableSchema equivalent datatype to be passed in to AvroConversionHelper.createConverterToAvro() val reconciledDataType = - if (latestTableSchema != null) SchemaConverters.toSqlType(latestTableSchema).dataType.asInstanceOf[StructType] else writeDataType + if (latestTableSchema != null) AvroConversionUtils.convertAvroSchemaToStructType(latestTableSchema) else writeDataType // Note: deserializer.deserializeRow(row) is not capable of handling evolved schema. i.e. if Row was serialized in // old schema, but deserializer was created with an encoder with evolved schema, deserialization fails. // Hence we always need to deserialize in the same schema as serialized schema. @@ -285,43 +293,4 @@ object HoodieSparkUtils extends SparkAdapterSupport { s"${tableSchema.fieldNames.mkString(",")}") AttributeReference(columnName, field.get.dataType, field.get.nullable)() } - - /** - * Create merge sql to merge leftTable and right table. - * - * @param leftTable table name. - * @param rightTable table name. - * @param cols merged cols. - * @return merge sql. - */ - def createMergeSql(leftTable: String, rightTable: String, cols: Seq[String]): String = { - var selectsql = "" - for (i <- (0 to cols.size-1)) { - selectsql = selectsql + s" if (${leftTable}.${cols(0)} is null, ${rightTable}.${cols(i)}, ${leftTable}.${cols(i)}) as ${cols(i)} ," - } - "select " + selectsql.dropRight(1) + s" from ${leftTable} full join ${rightTable} on ${leftTable}.${cols(0)} = ${rightTable}.${cols(0)}" - } - - /** - * Collect min/max statistics for candidate cols. - * support all col types. - * - * @param df dataFrame holds read files. - * @param cols candidate cols to collect statistics. - * @return - */ - def getMinMaxValueSpark(df: DataFrame, cols: Seq[String]): DataFrame = { - val sqlContext = df.sparkSession.sqlContext - import sqlContext.implicits._ - - val values = cols.flatMap(c => Seq( min(col(c)).as(c + "_minValue"), max(col(c)).as(c + "_maxValue"), count(c).as(c + "_noNullCount"))) - val valueCounts = count("*").as("totalNum") - val projectValues = Seq(col("file")) ++ cols.flatMap(c => - Seq(col(c + "_minValue"), col(c + "_maxValue"), expr(s"totalNum - ${c + "_noNullCount"}").as(c + "_num_nulls"))) - - val result = df.select(input_file_name() as "file", col("*")) - .groupBy($"file") - .agg(valueCounts, values: _*).select(projectValues:_*) - result - } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkParsePartitionUtil.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkParsePartitionUtil.scala index fc2275bab139..626b3c6ef0d4 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkParsePartitionUtil.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkParsePartitionUtil.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.datasources import java.util.TimeZone import org.apache.hadoop.fs.Path -import org.apache.spark.sql.execution.datasources.PartitioningUtils.PartitionValues + +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.DataType trait SparkParsePartitionUtil extends Serializable { @@ -30,5 +31,5 @@ trait SparkParsePartitionUtil extends Serializable { typeInference: Boolean, basePaths: Set[Path], userSpecifiedDataTypes: Map[String, DataType], - timeZone: TimeZone): Option[PartitionValues] + timeZone: TimeZone): InternalRow } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/execution/RangeSample.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/execution/RangeSample.scala index da993b7545e5..7c39ce2546f2 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/execution/RangeSample.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/execution/RangeSample.scala @@ -18,18 +18,19 @@ package org.apache.spark.sql.hudi.execution -import java.util - +import org.apache.hudi.common.util.BinaryUtil import org.apache.hudi.config.HoodieClusteringConfig +import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy +import org.apache.hudi.optimize.HilbertCurveUtils import org.apache.spark.rdd.{PartitionPruningRDD, RDD} -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, BoundReference, SortOrder, UnsafeProjection, UnsafeRow} -import org.apache.hudi.optimize.ZOrderingUtil -import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, BoundReference, SortOrder, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.util.MutablePair import org.apache.spark.util.random.SamplingUtils +import org.davidmoten.hilbert.HilbertCurve import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -37,10 +38,10 @@ import scala.reflect.{ClassTag, classTag} import scala.util.hashing.byteswap32 class RangeSample[K: ClassTag, V]( - zEncodeNum: Int, - rdd: RDD[_ <: Product2[K, V]], - private var ascend: Boolean = true, - val samplePointsPerPartitionHint: Int = 20) extends Serializable { + zEncodeNum: Int, + rdd: RDD[_ <: Product2[K, V]], + private var ascend: Boolean = true, + val samplePointsPerPartitionHint: Int = 20) extends Serializable { // We allow zEncodeNum = 0, which happens when sorting an empty RDD under the default settings. require(zEncodeNum >= 0, s"Number of zEncodeNum cannot be negative but found $zEncodeNum.") @@ -197,22 +198,22 @@ class RawDecisionBound[K : Ordering : ClassTag](ordering: Ordering[K]) extends S // For primitive keys, we can use the natural ordering. Otherwise, use the Ordering comparator. classTag[K] match { case ClassTag.Float => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float]) case ClassTag.Double => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Double]], x.asInstanceOf[Double]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Double]], x.asInstanceOf[Double]) case ClassTag.Byte => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Byte]], x.asInstanceOf[Byte]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Byte]], x.asInstanceOf[Byte]) case ClassTag.Char => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Char]], x.asInstanceOf[Char]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Char]], x.asInstanceOf[Char]) case ClassTag.Short => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Short]], x.asInstanceOf[Short]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Short]], x.asInstanceOf[Short]) case ClassTag.Int => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Int]], x.asInstanceOf[Int]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Int]], x.asInstanceOf[Int]) case ClassTag.Long => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Long]], x.asInstanceOf[Long]) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[Long]], x.asInstanceOf[Long]) case _ => val comparator = ordering.asInstanceOf[java.util.Comparator[Any]] - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x, comparator) + (l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x, comparator) } } @@ -236,122 +237,31 @@ class RawDecisionBound[K : Ordering : ClassTag](ordering: Ordering[K]) extends S } } -case class ZorderingBinarySort(b: Array[Byte]) extends Ordered[ZorderingBinarySort] with Serializable { - override def compare(that: ZorderingBinarySort): Int = { +case class ByteArraySorting(b: Array[Byte]) extends Ordered[ByteArraySorting] with Serializable { + override def compare(that: ByteArraySorting): Int = { val len = this.b.length - ZOrderingUtil.compareTo(this.b, 0, len, that.b, 0, len) + BinaryUtil.compareTo(this.b, 0, len, that.b, 0, len) } } object RangeSampleSort { /** - * create z-order DataFrame by sample - * support all col types - */ - def sortDataFrameBySampleSupportAllTypes(df: DataFrame, zCols: Seq[String], fileNum: Int): DataFrame = { - val spark = df.sparkSession - val internalRdd = df.queryExecution.toRdd - val schema = df.schema - val outputAttributes = df.queryExecution.analyzed.output - val sortingExpressions = outputAttributes.filter(p => zCols.contains(p.name)) - if (sortingExpressions.length == 0 || sortingExpressions.length != zCols.size) { - df - } else { - val zOrderBounds = df.sparkSession.sessionState.conf.getConfString( - HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.key, - HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.defaultValue.toString).toInt - - val sampleRdd = internalRdd.mapPartitionsInternal { iter => - val projection = UnsafeProjection.create(sortingExpressions, outputAttributes) - val mutablePair = new MutablePair[InternalRow, Null]() - // Internally, RangePartitioner runs a job on the RDD that samples keys to compute - // partition bounds. To get accurate samples, we need to copy the mutable keys. - iter.map(row => mutablePair.update(projection(row).copy(), null)) - } - - val orderings = sortingExpressions.map(SortOrder(_, Ascending)).zipWithIndex.map { case (ord, i) => - ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) - } - - val lazyGeneratedOrderings = orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord))) - - val sample = new RangeSample(zOrderBounds, sampleRdd) - - val rangeBounds = sample.getRangeBounds() - - implicit val ordering1 = lazyGeneratedOrderings(0) - - val sampleBounds = sample.determineRowBounds(rangeBounds, math.min(zOrderBounds, rangeBounds.length), lazyGeneratedOrderings, sortingExpressions) - - val origin_orderings = sortingExpressions.map(SortOrder(_, Ascending)).map { ord => - ord.copy(child = BoundReference(0, ord.dataType, ord.nullable)) - } - - val origin_lazyGeneratedOrderings = origin_orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord))) - - // expand bounds. - // maybe it's better to use the value of "spark.zorder.bounds.number" as maxLength, - // however this will lead to extra time costs when all zorder cols distinct count values are less then "spark.zorder.bounds.number" - val maxLength = sampleBounds.map(_.length).max - val expandSampleBoundsWithFactor = sampleBounds.map { bound => - val fillFactor = maxLength / bound.size.toDouble - (bound, fillFactor) - } - - val boundBroadCast = spark.sparkContext.broadcast(expandSampleBoundsWithFactor) - - val indexRdd = internalRdd.mapPartitionsInternal { iter => - val boundsWithFactor = boundBroadCast.value - import java.util.concurrent.ThreadLocalRandom - val threadLocalRandom = ThreadLocalRandom.current - val maxBoundNum = boundsWithFactor.map(_._1.length).max - val origin_Projections = sortingExpressions.map { se => - UnsafeProjection.create(Seq(se), outputAttributes) - } - - iter.map { unsafeRow => - val interleaveValues = origin_Projections.zip(origin_lazyGeneratedOrderings).zipWithIndex.map { case ((rowProject, lazyOrdering), index) => - val row = rowProject(unsafeRow) - val decisionBound = new RawDecisionBound(lazyOrdering) - if (row.isNullAt(0)) { - maxBoundNum + 1 - } else { - val (bound, factor) = boundsWithFactor(index) - if (factor > 1) { - val currentRank = decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]]) - currentRank*factor.toInt + threadLocalRandom.nextInt(factor.toInt) - } else { - decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]]) - } - } - }.toArray.map(ZOrderingUtil.intTo8Byte(_)) - val zValues = ZOrderingUtil.interleaving(interleaveValues, 8) - val mutablePair = new MutablePair[InternalRow, Array[Byte]]() - - mutablePair.update(unsafeRow, zValues) - } - }.sortBy(x => ZorderingBinarySort(x._2), numPartitions = fileNum).map(_._1) - spark.internalCreateDataFrame(indexRdd, schema) - } - } - - /** - * create z-order DataFrame by sample - * first, sample origin data to get z-cols bounds, then create z-order DataFrame + * create optimize DataFrame by sample + * first, sample origin data to get order-cols bounds, then apply sort to produce DataFrame * support all type data. - * this method need more resource and cost more time than createZIndexedDataFrameByMapValue + * this method need more resource and cost more time than createOptimizedDataFrameByMapValue */ - def sortDataFrameBySample(df: DataFrame, zCols: Seq[String], fileNum: Int): DataFrame = { + def sortDataFrameBySample(df: DataFrame, layoutOptStrategy: LayoutOptimizationStrategy, orderByCols: Seq[String], targetPartitionsCount: Int): DataFrame = { val spark = df.sparkSession val columnsMap = df.schema.fields.map(item => (item.name, item)).toMap val fieldNum = df.schema.fields.length - val checkCols = zCols.filter(col => columnsMap(col) != null) + val checkCols = orderByCols.filter(col => columnsMap(col) != null) - if (zCols.isEmpty || checkCols.isEmpty) { + if (orderByCols.isEmpty || checkCols.isEmpty) { df } else { - val zFields = zCols.map { col => + val zFields = orderByCols.map { col => val newCol = columnsMap(col) if (newCol == null) { (-1, null) @@ -367,8 +277,8 @@ object RangeSampleSort { } }.filter(_._1 != -1) // Complex type found, use createZIndexedDataFrameByRange - if (zFields.length != zCols.length) { - return sortDataFrameBySampleSupportAllTypes(df, zCols, fieldNum) + if (zFields.length != orderByCols.length) { + return sortDataFrameBySampleSupportAllTypes(df, orderByCols, targetPartitionsCount) } val rawRdd = df.rdd @@ -464,6 +374,11 @@ object RangeSampleSort { } } + val hilbertCurve = if (layoutOptStrategy == LayoutOptimizationStrategy.HILBERT) + Some(HilbertCurve.bits(32).dimensions(zFields.length)) + else + None + iter.map { row => val values = zFields.zipWithIndex.map { case ((index, field), rawIndex) => field.dataType match { @@ -509,17 +424,115 @@ object RangeSampleSort { case _ => -1 } - }.filter(v => v != -1).map(ZOrderingUtil.intTo8Byte(_)).toArray - val zValues = ZOrderingUtil.interleaving(values, 8) - Row.fromSeq(row.toSeq ++ Seq(zValues)) + }.filter(v => v != -1) + + val mapValues = layoutOptStrategy match { + case LayoutOptimizationStrategy.HILBERT => + HilbertCurveUtils.indexBytes(hilbertCurve.get, values.map(_.toLong).toArray, 32) + case LayoutOptimizationStrategy.ZORDER => + BinaryUtil.interleaving(values.map(BinaryUtil.intTo8Byte(_)).toArray, 8) + } + + Row.fromSeq(row.toSeq ++ Seq(mapValues)) } - }.sortBy(x => ZorderingBinarySort(x.getAs[Array[Byte]](fieldNum)), numPartitions = fileNum) + }.sortBy(x => ByteArraySorting(x.getAs[Array[Byte]](fieldNum)), numPartitions = targetPartitionsCount) val newDF = df.sparkSession.createDataFrame(indexRdd, StructType( df.schema.fields ++ Seq( - StructField(s"zindex", + StructField(s"index", BinaryType, false)) )) - newDF.drop("zindex") + newDF.drop("index") + } + } + + /** + * create z-order DataFrame by sample + * support all col types + */ + def sortDataFrameBySampleSupportAllTypes(df: DataFrame, zCols: Seq[String], fileNum: Int): DataFrame = { + val spark = df.sparkSession + val internalRdd = df.queryExecution.toRdd + val schema = df.schema + val outputAttributes = df.queryExecution.analyzed.output + val sortingExpressions = outputAttributes.filter(p => zCols.contains(p.name)) + if (sortingExpressions.length == 0 || sortingExpressions.length != zCols.size) { + df + } else { + val zOrderBounds = df.sparkSession.sessionState.conf.getConfString( + HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.key, + HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.defaultValue.toString).toInt + + val sampleRdd = internalRdd.mapPartitionsInternal { iter => + val projection = UnsafeProjection.create(sortingExpressions, outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.map(row => mutablePair.update(projection(row).copy(), null)) + } + + val orderings = sortingExpressions.map(SortOrder(_, Ascending)).zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + + val lazyGeneratedOrderings = orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord))) + + val sample = new RangeSample(zOrderBounds, sampleRdd) + + val rangeBounds = sample.getRangeBounds() + + implicit val ordering1 = lazyGeneratedOrderings(0) + + val sampleBounds = sample.determineRowBounds(rangeBounds, math.min(zOrderBounds, rangeBounds.length), lazyGeneratedOrderings, sortingExpressions) + + val origin_orderings = sortingExpressions.map(SortOrder(_, Ascending)).map { ord => + ord.copy(child = BoundReference(0, ord.dataType, ord.nullable)) + } + + val origin_lazyGeneratedOrderings = origin_orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord))) + + // expand bounds. + // maybe it's better to use the value of "spark.zorder.bounds.number" as maxLength, + // however this will lead to extra time costs when all zorder cols distinct count values are less then "spark.zorder.bounds.number" + val maxLength = sampleBounds.map(_.length).max + val expandSampleBoundsWithFactor = sampleBounds.map { bound => + val fillFactor = maxLength / bound.size.toDouble + (bound, fillFactor) + } + + val boundBroadCast = spark.sparkContext.broadcast(expandSampleBoundsWithFactor) + + val indexRdd = internalRdd.mapPartitionsInternal { iter => + val boundsWithFactor = boundBroadCast.value + import java.util.concurrent.ThreadLocalRandom + val threadLocalRandom = ThreadLocalRandom.current + val maxBoundNum = boundsWithFactor.map(_._1.length).max + val origin_Projections = sortingExpressions.map { se => + UnsafeProjection.create(Seq(se), outputAttributes) + } + + iter.map { unsafeRow => + val interleaveValues = origin_Projections.zip(origin_lazyGeneratedOrderings).zipWithIndex.map { case ((rowProject, lazyOrdering), index) => + val row = rowProject(unsafeRow) + val decisionBound = new RawDecisionBound(lazyOrdering) + if (row.isNullAt(0)) { + maxBoundNum + 1 + } else { + val (bound, factor) = boundsWithFactor(index) + if (factor > 1) { + val currentRank = decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]]) + currentRank*factor.toInt + threadLocalRandom.nextInt(factor.toInt) + } else { + decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]]) + } + } + }.toArray.map(BinaryUtil.intTo8Byte(_)) + val zValues = BinaryUtil.interleaving(interleaveValues, 8) + val mutablePair = new MutablePair[InternalRow, Array[Byte]]() + + mutablePair.update(unsafeRow, zValues) + } + }.sortBy(x => ByteArraySorting(x._2), numPartitions = fileNum).map(_._1) + spark.internalCreateDataFrame(indexRdd, schema) } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index 32ac8689687c..08960d97d8cb 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -18,27 +18,39 @@ package org.apache.hudi.client; +import org.apache.hudi.avro.model.HoodieInstantInfo; +import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.testutils.FileCreateUtils; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -191,20 +203,31 @@ public void testRollbackCommit() throws Exception { put(p3, "id33"); } }; - HoodieTestTable testTable = HoodieTestTable.of(metaClient) - .withPartitionMetaFiles(p1, p2, p3) - .addCommit(commitTime1) - .withBaseFilesInPartitions(partitionAndFileId1) - .addCommit(commitTime2) - .withBaseFilesInPartitions(partitionAndFileId2) - .addInflightCommit(commitTime3) - .withBaseFilesInPartitions(partitionAndFileId3); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withRollbackUsingMarkers(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + + Map>> partitionToFilesNameLengthMap1 = new HashMap<>(); + partitionAndFileId1.forEach((k, v) -> partitionToFilesNameLengthMap1.put(k, Collections.singletonList(Pair.of(v, 100)))); + testTable.doWriteOperation(commitTime1, WriteOperationType.INSERT, Arrays.asList(p1, p2, p3), partitionToFilesNameLengthMap1, + false, false); + + Map>> partitionToFilesNameLengthMap2 = new HashMap<>(); + partitionAndFileId2.forEach((k, v) -> partitionToFilesNameLengthMap2.put(k, Collections.singletonList(Pair.of(v, 200)))); + testTable.doWriteOperation(commitTime2, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap2, + false, false); + + Map>> partitionToFilesNameLengthMap3 = new HashMap<>(); + partitionAndFileId3.forEach((k, v) -> partitionToFilesNameLengthMap3.put(k, Collections.singletonList(Pair.of(v, 300)))); + testTable.doWriteOperation(commitTime3, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap3, + false, true); + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { // Rollback commit3 @@ -247,10 +270,10 @@ public void testRollbackCommit() throws Exception { } /** - * Test auto-rollback of commits which are in flight. + * Test Cases for effects of rollbacking completed/inflight commits. */ @Test - public void testAutoRollbackInflightCommit() throws Exception { + public void testFailedRollbackCommit() throws Exception { // Let's create some commit files and base files final String p1 = "2016/05/01"; final String p2 = "2016/05/02"; @@ -283,17 +306,126 @@ public void testAutoRollbackInflightCommit() throws Exception { .withPartitionMetaFiles(p1, p2, p3) .addCommit(commitTime1) .withBaseFilesInPartitions(partitionAndFileId1) - .addInflightCommit(commitTime2) + .addCommit(commitTime2) .withBaseFilesInPartitions(partitionAndFileId2) .addInflightCommit(commitTime3) .withBaseFilesInPartitions(partitionAndFileId3); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withRollbackUsingMarkers(false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { + + // Rollback commit3 + client.rollback(commitTime3); + assertFalse(testTable.inflightCommitExists(commitTime3)); + assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3)); + assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2)); + + metaClient.reloadActiveTimeline(); + List rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + assertEquals(rollbackInstants.size(), 1); + HoodieInstant rollbackInstant = rollbackInstants.get(0); + + // delete rollback completed meta file and retry rollback. + FileCreateUtils.deleteRollbackCommit(basePath, rollbackInstant.getTimestamp()); + + // recreate actual commit files so that we can retry the rollback + testTable.addInflightCommit(commitTime3).withBaseFilesInPartitions(partitionAndFileId3); + + // retry rolling back the commit again. + client.rollback(commitTime3); + + // verify there are no extra rollback instants + metaClient.reloadActiveTimeline(); + rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + assertEquals(rollbackInstants.size(), 1); + assertEquals(rollbackInstants.get(0), rollbackInstant); + + final String commitTime4 = "20160507040601"; + final String commitTime5 = "20160507050611"; + + // add inflight compaction then rolls it back + testTable.addInflightCompaction(commitTime4, new HoodieCommitMetadata()); + HoodieRollbackPlan rollbackPlan = new HoodieRollbackPlan(); + rollbackPlan.setRollbackRequests(Collections.emptyList()); + rollbackPlan.setInstantToRollback(new HoodieInstantInfo(commitTime4, HoodieTimeline.COMPACTION_ACTION)); + testTable.addRequestedRollback(commitTime5, rollbackPlan); + + // the compaction instants should be excluded + metaClient.reloadActiveTimeline(); + assertEquals(0, client.getPendingRollbackInfos(metaClient).size()); + + // verify there is no extra rollback instants + client.rollback(commitTime4); + + metaClient.reloadActiveTimeline(); + rollbackInstants = metaClient.reloadActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()); + assertEquals(2, rollbackInstants.size()); + } + } + + /** + * Test auto-rollback of commits which are in flight. + */ + @Test + public void testAutoRollbackInflightCommit() throws Exception { + // Let's create some commit files and base files + final String p1 = "2016/05/01"; + final String p2 = "2016/05/02"; + final String p3 = "2016/05/06"; + final String commitTime1 = "20160501010101"; + final String commitTime2 = "20160502020601"; + final String commitTime3 = "20160506030611"; + Map partitionAndFileId1 = new HashMap() { + { + put(p1, "id11"); + put(p2, "id12"); + put(p3, "id13"); + } + }; + Map partitionAndFileId2 = new HashMap() { + { + put(p1, "id21"); + put(p2, "id22"); + put(p3, "id23"); + } + }; + Map partitionAndFileId3 = new HashMap() { + { + put(p1, "id31"); + put(p2, "id32"); + put(p3, "id33"); + } + }; + // Set Failed Writes rollback to LAZY HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()).build(); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + + Map>> partitionToFilesNameLengthMap1 = new HashMap<>(); + partitionAndFileId1.forEach((k, v) -> partitionToFilesNameLengthMap1.put(k, Collections.singletonList(Pair.of(v, 100)))); + testTable.doWriteOperation(commitTime1, WriteOperationType.INSERT, Arrays.asList(p1, p2, p3), partitionToFilesNameLengthMap1, + false, false); + + Map>> partitionToFilesNameLengthMap2 = new HashMap<>(); + partitionAndFileId2.forEach((k, v) -> partitionToFilesNameLengthMap2.put(k, Collections.singletonList(Pair.of(v, 200)))); + testTable.doWriteOperation(commitTime2, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap2, + false, true); + + Map>> partitionToFilesNameLengthMap3 = new HashMap<>(); + partitionAndFileId3.forEach((k, v) -> partitionToFilesNameLengthMap3.put(k, Collections.singletonList(Pair.of(v, 300)))); + testTable.doWriteOperation(commitTime3, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap3, + false, true); + final String commitTime4 = "20160506030621"; try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime(commitTime4); @@ -308,6 +440,7 @@ public void testAutoRollbackInflightCommit() throws Exception { // Set Failed Writes rollback to EAGER config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withRollbackUsingMarkers(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); final String commitTime5 = "20160506030631"; try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index c70a2cf6a583..5f8b26bac953 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -18,17 +18,16 @@ package org.apache.hudi.client; -import java.util.HashSet; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; @@ -42,21 +41,32 @@ import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; +import java.util.List; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; public class TestHoodieClientMultiWriter extends HoodieClientTestBase { @@ -84,49 +94,79 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws E } Properties properties = new Properties(); properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); - HoodieWriteConfig cfg = getConfigBuilder() + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "10"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "250"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY,"250"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY,"10"); + HoodieWriteConfig writeConfig = getConfigBuilder() .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + // Timeline-server-based markers are not used for multi-writer tests + .withMarkersType(MarkerType.DIRECT.name()) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) .build()).withAutoCommit(false).withProperties(properties).build(); + // Create the first commit - createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200); - try { - ExecutorService executors = Executors.newFixedThreadPool(2); - SparkRDDWriteClient client1 = getHoodieWriteClient(cfg); - SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); - Future future1 = executors.submit(() -> { - String newCommitTime = "004"; - int numRecords = 100; - String commitTimeBetweenPrevAndNew = "002"; - try { - createCommitWithUpserts(cfg, client1, "002", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); - } catch (Exception e1) { - Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); - throw new RuntimeException(e1); - } - }); - Future future2 = executors.submit(() -> { - String newCommitTime = "005"; - int numRecords = 100; - String commitTimeBetweenPrevAndNew = "002"; - try { - createCommitWithUpserts(cfg, client2, "002", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); - } catch (Exception e2) { - Assertions.assertTrue(e2 instanceof HoodieWriteConflictException); - throw new RuntimeException(e2); - } - }); - future1.get(); - future2.get(); - Assertions.fail("Should not reach here, this means concurrent writes were handled incorrectly"); - } catch (Exception e) { - // Expected to fail due to overlapping commits - } + createCommitWithInserts(writeConfig, getHoodieWriteClient(writeConfig), "000", "001", 200); + + final int threadCount = 2; + final ExecutorService executors = Executors.newFixedThreadPool(2); + final SparkRDDWriteClient client1 = getHoodieWriteClient(writeConfig); + final SparkRDDWriteClient client2 = getHoodieWriteClient(writeConfig); + + final CyclicBarrier cyclicBarrier = new CyclicBarrier(threadCount); + final AtomicBoolean writer1Completed = new AtomicBoolean(false); + final AtomicBoolean writer2Completed = new AtomicBoolean(false); + + Future future1 = executors.submit(() -> { + try { + final String nextCommitTime = "002"; + final JavaRDD writeStatusList = startCommitForUpdate(writeConfig, client1, nextCommitTime, 100); + + // Wait for the 2nd writer to start the commit + cyclicBarrier.await(60, TimeUnit.SECONDS); + + // Commit the update before the 2nd writer + assertDoesNotThrow(() -> { + client1.commit(nextCommitTime, writeStatusList); + }); + + // Signal the 2nd writer to go ahead for his commit + cyclicBarrier.await(60, TimeUnit.SECONDS); + writer1Completed.set(true); + } catch (Exception e) { + writer1Completed.set(false); + } + }); + + Future future2 = executors.submit(() -> { + try { + final String nextCommitTime = "003"; + + // Wait for the 1st writer to make progress with the commit + cyclicBarrier.await(60, TimeUnit.SECONDS); + final JavaRDD writeStatusList = startCommitForUpdate(writeConfig, client2, nextCommitTime, 100); + + // Wait for the 1st writer to complete the commit + cyclicBarrier.await(60, TimeUnit.SECONDS); + assertThrows(HoodieWriteConflictException.class, () -> { + client2.commit(nextCommitTime, writeStatusList); + }); + writer2Completed.set(true); + } catch (Exception e) { + writer2Completed.set(false); + } + }); + + future1.get(); + future2.get(); + + // both should have been completed successfully. I mean, we already assert for conflict for writer2 at L155. + assertTrue(writer1Completed.get() && writer2Completed.get()); } - @Disabled + @Test public void testMultiWriterWithAsyncTableServicesWithConflictCOW() throws Exception { testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.COPY_ON_WRITE); } @@ -136,6 +176,81 @@ public void testMultiWriterWithAsyncTableServicesWithConflictMOR() throws Except testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.MERGE_ON_READ); } + @ParameterizedTest + @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) + public void testMultiWriterWithInsertsToDistinctPartitions(HoodieTableType tableType) throws Exception { + if (tableType == HoodieTableType.MERGE_ON_READ) { + setUpMORTestTable(); + } + + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "10"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "3000"); + + HoodieWriteConfig cfg = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withInlineCompaction(false) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withMaxNumDeltaCommitsBeforeCompaction(2) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(FileSystemBasedLockProviderTestClass.class) + .build()) + .withAutoCommit(false) + // Timeline-server-based markers are not used for multi-writer tests + .withMarkersType(MarkerType.DIRECT.name()) + .withProperties(properties) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST) + .withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build()) + .build(); + + // Create the first commit + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + createCommitWithInsertsForPartition(cfg, client, "000", "001", 100, "2016/03/01"); + + int numConcurrentWriters = 5; + ExecutorService executors = Executors.newFixedThreadPool(numConcurrentWriters); + + List> futures = new ArrayList<>(numConcurrentWriters); + for (int loop = 0; loop < numConcurrentWriters; loop++) { + String newCommitTime = "00" + (loop + 2); + String partition = "2016/03/0" + (loop + 2); + futures.add(executors.submit(() -> { + try { + SparkRDDWriteClient writeClient = getHoodieWriteClient(cfg); + createCommitWithInsertsForPartition(cfg, writeClient, "001", newCommitTime, 100, partition); + } catch (Exception e) { + throw new RuntimeException(e); + } + })); + } + + futures.forEach(f -> { + try { + f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + /** + * Count down the latch and await for all the needed threads to join. + * + * @param latch - Count down latch + * @param waitTimeMillis - Max wait time in millis for waiting + */ + private void latchCountDownAndWait(CountDownLatch latch, long waitTimeMillis) { + latch.countDown(); + try { + latch.await(waitTimeMillis, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // + } + } + private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { // create inserts X 1 if (tableType == HoodieTableType.MERGE_ON_READ) { @@ -148,20 +263,22 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000"); // Disabling embedded timeline server, it doesn't work with multiwriter - HoodieWriteConfig cfg = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) - .withInlineCompaction(false).withAsyncClean(true) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) - .withMaxNumDeltaCommitsBeforeCompaction(2).build()) + HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) + .withInlineCompaction(false).withAsyncClean(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withMaxNumDeltaCommitsBeforeCompaction(2).build()) .withEmbeddedTimelineServerEnabled(false) + // Timeline-server-based markers are not used for multi-writer tests + .withMarkersType(MarkerType.DIRECT.name()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType( FileSystemViewStorageType.MEMORY).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) - .build()).withAutoCommit(false).withProperties(properties).build(); + .build()).withAutoCommit(false).withProperties(properties); Set validInstants = new HashSet<>(); // Create the first commit with inserts + HoodieWriteConfig cfg = writeConfigBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); createCommitWithInserts(cfg, client, "000", "001", 200); validInstants.add("001"); @@ -170,82 +287,108 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t createCommitWithUpserts(cfg, client, "002", "000", "003", 100); validInstants.add("002"); validInstants.add("003"); - ExecutorService executors = Executors.newFixedThreadPool(2); - SparkRDDWriteClient client1 = getHoodieWriteClient(cfg); - SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); + + // Three clients running actions in parallel + final int threadCount = 3; + final CountDownLatch scheduleCountDownLatch = new CountDownLatch(threadCount); + final ExecutorService executors = Executors.newFixedThreadPool(threadCount); + + // Write config with clustering enabled + final HoodieWriteConfig cfg2 = writeConfigBuilder + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClustering(true) + .withInlineClusteringNumCommits(1) + .build()) + .build(); + final SparkRDDWriteClient client1 = getHoodieWriteClient(cfg2); + final SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); + final SparkRDDWriteClient client3 = getHoodieWriteClient(cfg); + // Create upserts, schedule cleaning, schedule compaction in parallel Future future1 = executors.submit(() -> { - String newCommitTime = "004"; - int numRecords = 100; - String commitTimeBetweenPrevAndNew = "002"; - try { - createCommitWithUpserts(cfg, client1, "003", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); - if (tableType == HoodieTableType.MERGE_ON_READ) { - Assertions.fail("Conflicts not handled correctly"); - } - validInstants.add("004"); - } catch (Exception e1) { - if (tableType == HoodieTableType.MERGE_ON_READ) { - Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); - } + final String newCommitTime = "004"; + final int numRecords = 100; + final String commitTimeBetweenPrevAndNew = "002"; + + // We want the upsert to go through only after the compaction + // and cleaning schedule completion. So, waiting on latch here. + latchCountDownAndWait(scheduleCountDownLatch, 30000); + if (tableType == HoodieTableType.MERGE_ON_READ) { + // Since the compaction already went in, this upsert has + // to fail + assertThrows(IllegalArgumentException.class, () -> { + createCommitWithUpserts(cfg, client1, "003", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); + }); + } else { + // We don't have the compaction for COW and so this upsert + // has to pass + assertDoesNotThrow(() -> { + createCommitWithUpserts(cfg, client1, "003", commitTimeBetweenPrevAndNew, newCommitTime, numRecords); + }); + validInstants.add(newCommitTime); } }); + Future future2 = executors.submit(() -> { - try { - client2.scheduleTableService("005", Option.empty(), TableServiceType.COMPACT); - } catch (Exception e2) { - if (tableType == HoodieTableType.MERGE_ON_READ) { - throw new RuntimeException(e2); - } + if (tableType == HoodieTableType.MERGE_ON_READ) { + assertDoesNotThrow(() -> { + client2.scheduleTableService("005", Option.empty(), TableServiceType.COMPACT); + }); } + latchCountDownAndWait(scheduleCountDownLatch, 30000); }); + Future future3 = executors.submit(() -> { - try { - client2.scheduleTableService("006", Option.empty(), TableServiceType.CLEAN); - } catch (Exception e2) { - throw new RuntimeException(e2); - } + assertDoesNotThrow(() -> { + latchCountDownAndWait(scheduleCountDownLatch, 30000); + client3.scheduleTableService("006", Option.empty(), TableServiceType.CLEAN); + }); }); future1.get(); future2.get(); future3.get(); + + CountDownLatch runCountDownLatch = new CountDownLatch(threadCount); // Create inserts, run cleaning, run compaction in parallel future1 = executors.submit(() -> { - String newCommitTime = "007"; - int numRecords = 100; - try { + final String newCommitTime = "007"; + final int numRecords = 100; + latchCountDownAndWait(runCountDownLatch, 30000); + assertDoesNotThrow(() -> { createCommitWithInserts(cfg, client1, "003", newCommitTime, numRecords); validInstants.add("007"); - } catch (Exception e1) { - throw new RuntimeException(e1); - } + }); }); + future2 = executors.submit(() -> { - try { - JavaRDD writeStatusJavaRDD = (JavaRDD) client2.compact("005"); - client2.commitCompaction("005", writeStatusJavaRDD, Option.empty()); - validInstants.add("005"); - } catch (Exception e2) { - if (tableType == HoodieTableType.MERGE_ON_READ) { - throw new RuntimeException(e2); - } + latchCountDownAndWait(runCountDownLatch, 30000); + if (tableType == HoodieTableType.MERGE_ON_READ) { + assertDoesNotThrow(() -> { + JavaRDD writeStatusJavaRDD = (JavaRDD) client2.compact("005"); + client2.commitCompaction("005", writeStatusJavaRDD, Option.empty()); + validInstants.add("005"); + }); } }); + future3 = executors.submit(() -> { - try { - client2.clean("006", false); + latchCountDownAndWait(runCountDownLatch, 30000); + assertDoesNotThrow(() -> { + client3.clean("006", false); validInstants.add("006"); - } catch (Exception e2) { - throw new RuntimeException(e2); - } + }); }); future1.get(); future2.get(); future3.get(); - Set completedInstants = metaClient.getActiveTimeline().getCommitsTimeline() + + validInstants.addAll( + metaClient.reloadActiveTimeline().getCompletedReplaceTimeline() + .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet())); + Set completedInstants = metaClient.reloadActiveTimeline().getCommitsTimeline() .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toSet()); - Assertions.assertTrue(validInstants.containsAll(completedInstants)); + assertTrue(validInstants.containsAll(completedInstants)); } @ParameterizedTest @@ -256,13 +399,20 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) } Properties properties = new Properties(); properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); - HoodieWriteConfig cfg = getConfigBuilder() + HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + // Timeline-server-based markers are not used for multi-writer tests + .withMarkersType(MarkerType.DIRECT.name()) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class) - .build()).withAutoCommit(false).withProperties(properties).build(); + .build()).withAutoCommit(false).withProperties(properties); + HoodieWriteConfig cfg = writeConfigBuilder.build(); + HoodieWriteConfig cfg2 = writeConfigBuilder.build(); + HoodieWriteConfig cfg3 = writeConfigBuilder + .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClustering(true).withInlineClusteringNumCommits(1).build()) + .build(); + // Create the first commit createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200); // Start another inflight commit @@ -276,24 +426,32 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) // Start and finish another commit while the previous writer for commit 003 is running newCommitTime = "004"; SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); - JavaRDD result2 = updateBatch(cfg, client2, newCommitTime, "001", + JavaRDD result2 = updateBatch(cfg2, client2, newCommitTime, "001", Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, numRecords, 200, 2); client2.commit(newCommitTime, result2); // Schedule and run clustering while previous writer for commit 003 is running - SparkRDDWriteClient client3 = getHoodieWriteClient(cfg); + SparkRDDWriteClient client3 = getHoodieWriteClient(cfg3); // schedule clustering Option clusterInstant = client3.scheduleTableService(Option.empty(), TableServiceType.CLUSTER); - client3.cluster(clusterInstant.get(), true); + assertTrue(clusterInstant.isPresent()); // Attempt to commit the inflight commit 003 try { client1.commit("003", result1); - Assertions.fail("Should have thrown a concurrent conflict exception"); + fail("Should have thrown a concurrent conflict exception"); } catch (Exception e) { // Expected } } + private void createCommitWithInsertsForPartition(HoodieWriteConfig cfg, SparkRDDWriteClient client, + String prevCommitTime, String newCommitTime, int numRecords, + String partition) throws Exception { + JavaRDD result = insertBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::insert, + false, false, numRecords, numRecords, 1, Option.of(partition)); + assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); + } + private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommitTime, String newCommitTime, int numRecords) throws Exception { // Finish first base commmit @@ -311,4 +469,32 @@ private void createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient client.commit(newCommitTime, result); } + /** + * Start the commit for an update operation with given number of records + * + * @param writeConfig - Write config + * @param writeClient - Write client for starting the commit + * @param newCommitTime - Commit time for the update + * @param numRecords - Number of records to update + * @return RDD of write status from the update + * @throws Exception + */ + private JavaRDD startCommitForUpdate(HoodieWriteConfig writeConfig, SparkRDDWriteClient writeClient, + String newCommitTime, int numRecords) throws Exception { + // Start the new commit + writeClient.startCommitWithTime(newCommitTime); + + // Prepare update records + final Function2, String, Integer> recordGenFunction = + generateWrapRecordsFn(false, writeConfig, dataGen::generateUniqueUpdates); + final List records = recordGenFunction.apply(newCommitTime, numRecords); + final JavaRDD writeRecords = jsc.parallelize(records, 1); + + // Write updates + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn = SparkRDDWriteClient::upsert; + JavaRDD result = writeFn.apply(writeClient, writeRecords, newCommitTime); + List statuses = result.collect(); + assertNoWriteErrors(statuses); + return result; + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 29bad0d949c3..dda396a13567 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -303,7 +303,7 @@ public void testMORTable() throws Exception { // Insert with original schema is allowed now insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert, - false, false, 0, 0, 0); + false, false, 0, 0, 0, Option.empty()); checkLatestDeltaCommit("009"); checkReadRecords("000", 3 * numRecords); } @@ -316,7 +316,7 @@ public void testCopyOnWriteTable() throws Exception { .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); - HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); + HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA).withRollbackUsingMarkers(false).build(); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA @@ -438,7 +438,7 @@ public void testCopyOnWriteTable() throws Exception { // Insert with original schema is allowed now insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, 2 * numRecords, 1); + false, true, numRecords, 2 * numRecords, 1, Option.empty()); checkReadRecords("000", 2 * numRecords); // Update with original schema is allowed now @@ -507,11 +507,14 @@ private List convertToSchema(List records, String sc } private HoodieWriteConfig getWriteConfig(String schema) { + return getWriteConfigBuilder(schema).build(); + } + + private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { return getConfigBuilder(schema) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) - .withAvroSchemaValidate(true) - .build(); + .withAvroSchemaValidate(true); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index 96782f49428c..00e65a67c08e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.BaseFileUtils; @@ -64,6 +65,7 @@ public void setUp() throws Exception { HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); initSparkContexts("TestUpdateSchemaEvolution"); initFileSystem(); + initTimelineService(); } @AfterEach @@ -228,6 +230,9 @@ public void testSchemaEvolutionOnUpdateMisMatchWithChangeColumnType() throws Exc private HoodieWriteConfig makeHoodieClientConfig(String name) { Schema schema = getSchemaFromResource(getClass(), name); - return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build(); + return HoodieWriteConfig.newBuilder().withPath(basePath) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) + .withSchema(schema.toString()).build(); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkRecentDaysClusteringPlanStrategy.java deleted file mode 100644 index a9c7b7154edc..000000000000 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkRecentDaysClusteringPlanStrategy.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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 org.apache.hudi.client.clustering.plan.strategy; - -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertSame; - -public class TestSparkRecentDaysClusteringPlanStrategy { - @Mock - HoodieSparkCopyOnWriteTable table; - @Mock - HoodieSparkEngineContext context; - HoodieWriteConfig hoodieWriteConfig; - - @BeforeEach - public void setUp() { - this.hoodieWriteConfig = HoodieWriteConfig - .newBuilder() - .withPath("Fake_Table_Path") - .withClusteringConfig(HoodieClusteringConfig - .newBuilder() - .withClusteringSkipPartitionsFromLatest(1) - .withClusteringTargetPartitions(1) - .build()) - .build(); - } - - @Test - public void testFilterPartitionPaths() { - SparkRecentDaysClusteringPlanStrategy sg = new SparkRecentDaysClusteringPlanStrategy(table, context, hoodieWriteConfig); - ArrayList fakeTimeBasedPartitionsPath = new ArrayList<>(); - fakeTimeBasedPartitionsPath.add("20210718"); - fakeTimeBasedPartitionsPath.add("20210716"); - fakeTimeBasedPartitionsPath.add("20210719"); - List list = sg.filterPartitionPaths(fakeTimeBasedPartitionsPath); - assertEquals(1, list.size()); - assertSame("20210718", list.get(0)); - } -} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java index db17ceae92af..b35fee015310 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java @@ -119,7 +119,11 @@ public static void init() throws Exception { @AfterAll public static void clean() throws Exception { - utility.shutdownMiniCluster(); + utility.shutdownMiniHBaseCluster(); + utility.shutdownMiniDFSCluster(); + utility.shutdownMiniMapReduceCluster(); + // skip shutdownZkCluster due to localhost connection refused issue + utility = null; } @BeforeEach @@ -278,7 +282,7 @@ public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws E final int numRecords = 10; final String oldPartitionPath = "1970/01/01"; final String emptyHoodieRecordPayloadClasssName = EmptyHoodieRecordPayload.class.getName(); - HoodieWriteConfig config = getConfig(true, true); + HoodieWriteConfig config = getConfigBuilder(100, true, true).withRollbackUsingMarkers(false).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) { @@ -337,6 +341,7 @@ public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws E public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Load to memory HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withRollbackUsingMarkers(false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); @@ -383,7 +388,7 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { @Test public void testSimpleTagLocationWithInvalidCommit() throws Exception { // Load to memory - HoodieWriteConfig config = getConfig(); + HoodieWriteConfig config = getConfigBuilder(100, false, false).withRollbackUsingMarkers(false).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); @@ -425,6 +430,7 @@ public void testSimpleTagLocationWithInvalidCommit() throws Exception { public void testEnsureTagLocationUsesCommitTimeline() throws Exception { // Load to memory HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withRollbackUsingMarkers(false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 9ece523ff472..392d687349e7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -18,6 +18,7 @@ package org.apache.hudi.client.functional; +import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -29,6 +30,8 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -41,6 +44,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -50,6 +54,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.FileCreateUtils; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.HoodieTimer; @@ -60,12 +65,18 @@ import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsConfig; +import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.hudi.metadata.FileSystemBackedTableMetadata; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieMetadataMetrics; +import org.apache.hudi.metadata.HoodieMetadataPayload; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator; import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; @@ -75,9 +86,13 @@ import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.hudi.testutils.MetadataMergeWriteStatus; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.util.Time; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -89,12 +104,15 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -104,6 +122,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static java.util.Arrays.asList; @@ -116,10 +135,13 @@ import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @Tag("functional") @@ -127,7 +149,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase { private static final Logger LOG = LogManager.getLogger(TestHoodieBackedMetadata.class); - public static List bootstrapAndTableOperationTestArgs() { + public static List tableTypeAndEnableOperationArgs() { return asList( Arguments.of(COPY_ON_WRITE, true), Arguments.of(COPY_ON_WRITE, false), @@ -140,7 +162,7 @@ public static List bootstrapAndTableOperationTestArgs() { * Metadata Table bootstrap scenarios. */ @ParameterizedTest - @MethodSource("bootstrapAndTableOperationTestArgs") + @MethodSource("tableTypeAndEnableOperationArgs") public void testMetadataTableBootstrap(HoodieTableType tableType, boolean addRollback) throws Exception { init(tableType, false); // bootstrap with few commits @@ -221,9 +243,9 @@ public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Excep * Test various table operations sync to Metadata Table correctly. */ @ParameterizedTest - @MethodSource("bootstrapAndTableOperationTestArgs") + @MethodSource("tableTypeAndEnableOperationArgs") public void testTableOperations(HoodieTableType tableType, boolean enableFullScan) throws Exception { - init(tableType, true, enableFullScan); + init(tableType, true, enableFullScan, false, false); doWriteInsertAndUpsert(testTable); // trigger an upsert @@ -246,6 +268,43 @@ public void testTableOperations(HoodieTableType tableType, boolean enableFullSca validateMetadata(testTable, emptyList(), true); } + @Test + public void testMetadataTableArchival() throws Exception { + init(COPY_ON_WRITE, false); + writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .enableFullScan(true) + .enableMetrics(false) + .withMaxNumDeltaCommitsBeforeCompaction(3) + .archiveCommitsWith(3, 4) + .retainCommits(1) + .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()).build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + + AtomicInteger commitTime = new AtomicInteger(1); + // trigger 2 regular writes(1 bootstrap commit). just 1 before archival can get triggered. + int i = 1; + for (; i <= 2; i++) { + doWriteOperation(testTable, "000000" + (commitTime.getAndIncrement()), INSERT); + } + // expected num commits = 1 (bootstrap) + 2 (writes) + 1 compaction. + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), 4); + + // trigger a async table service, archival should not kick in, even though conditions are met. + doCluster(testTable, "000000" + commitTime.getAndIncrement()); + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), 5); + + // trigger a regular write operation. archival should kick in. + doWriteOperation(testTable, "000000" + (commitTime.getAndIncrement()), INSERT); + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), 3); + } + @ParameterizedTest @EnumSource(HoodieTableType.class) public void testMetadataInsertUpsertClean(HoodieTableType tableType) throws Exception { @@ -260,6 +319,16 @@ public void testMetadataInsertUpsertClean(HoodieTableType tableType) throws Exce validateMetadata(testTable, emptyList(), true); } + @Test + public void testMetadataInsertUpsertCleanNonPartitioned() throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType); + doWriteOperationNonPartitioned(testTable, "0000001", INSERT); + doWriteOperationNonPartitioned(testTable, "0000002", UPSERT); + testTable.doCleanBasedOnCommits("0000003", Arrays.asList("0000001")); + validateMetadata(testTable, emptyList(), true); + } + @ParameterizedTest @EnumSource(HoodieTableType.class) public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception { @@ -275,6 +344,7 @@ public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception /** * Tests that table services in data table won't trigger table services in metadata table. + * * @throws Exception */ @Test @@ -304,18 +374,150 @@ public void testMetadataTableServices() throws Exception { assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000004001"); } + + /** + * Tests that virtual key configs are honored in base files after compaction in metadata table. + * + * @throws Exception + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testVirtualKeysInBaseFiles(boolean populateMetaFields) throws Exception { + HoodieTableType tableType = MERGE_ON_READ; + init(tableType, false); + writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .enableFullScan(true) + .enableMetrics(false) + .withPopulateMetaFields(populateMetaFields) + .withMaxNumDeltaCommitsBeforeCompaction(2) + .build()).build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + + doWriteOperation(testTable, "0000001", INSERT); + doClean(testTable, "0000003", Arrays.asList("0000001")); + // this should have triggered compaction in metadata table + doWriteOperation(testTable, "0000004", UPSERT); + + HoodieTableMetadata tableMetadata = metadata(writeConfig, context); + assertTrue(tableMetadata.getLatestCompactionTime().isPresent()); + assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000004001"); + + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + HoodieWriteConfig metadataTableWriteConfig = getMetadataWriteConfig(writeConfig); + metadataMetaClient.reloadActiveTimeline(); + + HoodieTable table = HoodieSparkTable.create(metadataTableWriteConfig, context, metadataMetaClient); + table.getHoodieView().sync(); + List fileSlices = table.getSliceView().getLatestFileSlices("files").collect(Collectors.toList()); + HoodieBaseFile baseFile = fileSlices.get(0).getBaseFile().get(); + HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()), + new CacheConfig(context.getHadoopConf().get())); + List> records = hoodieHFileReader.readAllRecords(); + records.forEach(entry -> { + if (populateMetaFields) { + assertNotNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); + } else { + assertNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)); + } + }); + } + + /** + * Tests that virtual key configs are honored in base files after compaction in metadata table. + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompaction) throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType, false); + writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .enableFullScan(true) + .enableMetrics(false) + .withMaxNumDeltaCommitsBeforeCompaction(3) + .build()).build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + + doWriteOperation(testTable, "0000001", INSERT); + // create an inflight compaction in metadata table. + // not easy to create an inflight in metadata table directly, hence letting compaction succeed and then deleting the completed instant. + // this new write is expected to trigger metadata table compaction + String commitInstant = "0000002"; + doWriteOperation(testTable, commitInstant, INSERT); + + HoodieTableMetadata tableMetadata = metadata(writeConfig, context); + String metadataCompactionInstant = commitInstant + "001"; + assertTrue(tableMetadata.getLatestCompactionTime().isPresent()); + assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant); + + validateMetadata(testTable); + // Fetch compaction Commit file and rename to some other file. completed compaction meta file should have some serialized info that table interprets + // for future upserts. so, renaming the file here to some temp name and later renaming it back to same name. + java.nio.file.Path parentPath = Paths.get(metadataTableBasePath, HoodieTableMetaClient.METAFOLDER_NAME); + java.nio.file.Path metaFilePath = parentPath.resolve(metadataCompactionInstant + HoodieTimeline.COMMIT_EXTENSION); + java.nio.file.Path tempFilePath = FileCreateUtils.renameFileToTemp(metaFilePath, metadataCompactionInstant); + metaClient.reloadActiveTimeline(); + testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + // this validation will exercise the code path where a compaction is inflight in metadata table, but still metadata based file listing should match non + // metadata based file listing. + validateMetadata(testTable); + + if (simulateFailedCompaction) { + // this should retry the compaction in metadata table. + doWriteOperation(testTable, "0000003", INSERT); + } else { + // let the compaction succeed in metadata and validation should succeed. + FileCreateUtils.renameTempToMetaFile(tempFilePath, metaFilePath); + } + + validateMetadata(testTable); + + // add few more write and validate + doWriteOperation(testTable, "0000004", INSERT); + doWriteOperation(testTable, "0000005", UPSERT); + validateMetadata(testTable); + + if (simulateFailedCompaction) { + //trigger another compaction failure. + metadataCompactionInstant = "0000005001"; + tableMetadata = metadata(writeConfig, context); + assertTrue(tableMetadata.getLatestCompactionTime().isPresent()); + assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant); + + // Fetch compaction Commit file and rename to some other file. completed compaction meta file should have some serialized info that table interprets + // for future upserts. so, renaming the file here to some temp name and later renaming it back to same name. + parentPath = Paths.get(metadataTableBasePath, HoodieTableMetaClient.METAFOLDER_NAME); + metaFilePath = parentPath.resolve(metadataCompactionInstant + HoodieTimeline.COMMIT_EXTENSION); + tempFilePath = FileCreateUtils.renameFileToTemp(metaFilePath, metadataCompactionInstant); + + validateMetadata(testTable); + + // this should retry the failed compaction in metadata table. + doWriteOperation(testTable, "0000006", INSERT); + + validateMetadata(testTable); + + // add few more write and validate + doWriteOperation(testTable, "0000007", INSERT); + doWriteOperation(testTable, "0000008", UPSERT); + validateMetadata(testTable); + } + } + /** * Test rollback of various table operations sync to Metadata Table correctly. */ - //@ParameterizedTest - //@EnumSource(HoodieTableType.class) - @Disabled + @ParameterizedTest + @EnumSource(HoodieTableType.class) public void testRollbackOperations(HoodieTableType tableType) throws Exception { init(tableType); doWriteInsertAndUpsert(testTable); // trigger an upsert - doWriteOperationAndValidate(testTable, "0000003"); + doWriteOperation(testTable, "0000003", UPSERT); // trigger a commit and rollback doWriteOperation(testTable, "0000004"); @@ -355,12 +557,34 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { validateMetadata(testTable, true); } + @Test + public void testRollbackOperationsNonPartitioned() throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType); + doWriteInsertAndUpsertNonPartitioned(testTable); + + // trigger an upsert + doWriteOperationNonPartitioned(testTable, "0000003", UPSERT); + + // trigger a commit and rollback + doWriteOperationNonPartitioned(testTable, "0000004", UPSERT); + doRollback(testTable, "0000004", "0000005"); + validateMetadata(testTable); + + // trigger few upserts and validate + for (int i = 6; i < 10; i++) { + doWriteOperationNonPartitioned(testTable, "000000" + i, UPSERT); + } + validateMetadata(testTable); + } + /** * Test that manual rollbacks work correctly and enough timeline history is maintained on the metadata table * timeline. */ - @Test - public void testManualRollbacks() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testManualRollbacks(final boolean populateMateFields) throws Exception { HoodieTableType tableType = COPY_ON_WRITE; init(tableType, false); // Setting to archive more aggressively on the Metadata Table than the Dataset @@ -370,13 +594,15 @@ public void testManualRollbacks() throws Exception { writeConfig = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) .archiveCommitsWith(minArchiveCommitsMetadata, minArchiveCommitsMetadata + 1).retainCommits(1) - .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction) + .withPopulateMetaFields(populateMateFields) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) .build(); initWriteConfigAndMetatableWriter(writeConfig, true); - doWriteInsertAndUpsert(testTable, "000001", "000002"); + doWriteInsertAndUpsert(testTable, "000001", "000002", false); for (int i = 3; i < 10; i++) { doWriteOperation(testTable, "00000" + i); @@ -462,29 +688,45 @@ public void testSync(HoodieTableType tableType) throws Exception { validateMetadata(testTable, emptyList(), true); } + /** + * Fetches next commit time in seconds from current one. + * + * @param curCommitTime current commit time. + * @return the next valid commit time. + */ + private Long getNextCommitTime(long curCommitTime) { + if ((curCommitTime + 1) % 1000000000000L >= 60) { // max seconds is 60 and hence + return Long.parseLong(HoodieActiveTimeline.createNewInstantTime()); + } else { + return curCommitTime + 1; + } + } + @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType) throws Exception { - init(tableType); + @MethodSource("tableTypeAndEnableOperationArgs") + public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType, boolean nonPartitionedDataset) throws Exception { + init(tableType, true, true, true, false); + long baseCommitTime = Long.parseLong(HoodieActiveTimeline.createNewInstantTime()); for (int i = 1; i < 25; i += 7) { - String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i; - String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1); - String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2); - String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3); - String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4); - String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5); - String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6); - doWriteOperation(testTable, commitTime1, INSERT); - doWriteOperation(testTable, commitTime2); - doClean(testTable, commitTime3, Arrays.asList(commitTime1)); - doWriteOperation(testTable, commitTime4); + long commitTime1 = getNextCommitTime(baseCommitTime); + long commitTime2 = getNextCommitTime(commitTime1); + long commitTime3 = getNextCommitTime(commitTime2); + long commitTime4 = getNextCommitTime(commitTime3); + long commitTime5 = getNextCommitTime(commitTime4); + long commitTime6 = getNextCommitTime(commitTime5); + long commitTime7 = getNextCommitTime(commitTime6); + baseCommitTime = commitTime7; + doWriteOperation(testTable, Long.toString(commitTime1), INSERT, nonPartitionedDataset); + doWriteOperation(testTable, Long.toString(commitTime2), UPSERT, nonPartitionedDataset); + doClean(testTable, Long.toString(commitTime3), Arrays.asList(Long.toString(commitTime1))); + doWriteOperation(testTable, Long.toString(commitTime4), UPSERT, nonPartitionedDataset); if (tableType == MERGE_ON_READ) { - doCompaction(testTable, commitTime5); + doCompaction(testTable, Long.toString(commitTime5), nonPartitionedDataset); } - doWriteOperation(testTable, commitTime6); - doRollback(testTable, commitTime6, commitTime7); + doWriteOperation(testTable, Long.toString(commitTime6), UPSERT, nonPartitionedDataset); + doRollback(testTable, Long.toString(commitTime6), Long.toString(commitTime7)); } - validateMetadata(testTable, emptyList(), true); + validateMetadata(testTable, emptyList(), nonPartitionedDataset); } // Some operations are not feasible with test table infra. hence using write client to test those cases. @@ -522,6 +764,35 @@ public void testFirstCommitRollback(HoodieTableType tableType) throws Exception } } + /** + * Tests the metadata payload spurious deletes. + * Lets say a commit was applied to metadata table, and later was explicitly got rolledback. Due to spark task failures, there could be more files in rollback + * metadata when compared to the original commit metadata. When payload consistency check is enabled, it will throw exception. If not, it will succeed. + * + * @throws Exception + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testMetadataPayloadSpuriousDeletes(boolean ignoreSpuriousDeletes) throws Exception { + tableType = COPY_ON_WRITE; + init(tableType, true, true, false, ignoreSpuriousDeletes); + doWriteInsertAndUpsert(testTable); + // trigger an upsert + doWriteOperationAndValidate(testTable, "0000003"); + + // trigger a commit and rollback + doWriteOperation(testTable, "0000004"); + // add extra files in rollback to check for payload consistency + Map> extraFiles = new HashMap<>(); + extraFiles.put("p1", Collections.singletonList("f10")); + extraFiles.put("p2", Collections.singletonList("f12")); + testTable.doRollbackWithExtraFiles("0000004", "0000005", extraFiles); + if (!ignoreSpuriousDeletes) { + assertThrows(HoodieMetadataException.class, () -> validateMetadata(testTable)); + } else { + validateMetadata(testTable); + } + } /** * Test several table operations with restore. This test uses SparkRDDWriteClient. @@ -854,7 +1125,7 @@ public void testReader() throws Exception { *

* Metadata Table should be automatically compacted as per config. */ - @Test + @Disabled public void testCleaningArchivingAndCompaction() throws Exception { init(HoodieTableType.COPY_ON_WRITE, false); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); @@ -1071,18 +1342,72 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte assertTrue(oldStatus.getModificationTime() < newStatus.getModificationTime()); } + /** + * Tests rollback of a commit which has new partitions which is not present in hudi table prior to the commit being rolledback. + * + * @throws Exception + */ + @Test + public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false, false).build(), + true)) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 10); + List upsertRecords = new ArrayList<>(); + for (HoodieRecord entry : records) { + if (entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + || entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) { + upsertRecords.add(entry); + } + } + List writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed + // instant so that only the inflight is left over. + String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime); + assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + commitInstantFileName), false)); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false, false).build(), + true)) { + String newCommitTime = client.startCommit(); + // Next insert + List records = dataGen.generateInserts(newCommitTime, 20); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + } + /** * Test various error scenarios. */ - //@Test - @Disabled + @Test public void testErrorCases() throws Exception { init(HoodieTableType.COPY_ON_WRITE); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table // should be rolled back to last valid commit. - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false, false).build(), + true)) { String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 10); @@ -1095,6 +1420,7 @@ public void testErrorCases() throws Exception { records = dataGen.generateInserts(newCommitTime, 5); writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); + validateMetadata(client); // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. @@ -1103,7 +1429,9 @@ public void testErrorCases() throws Exception { commitInstantFileName), false)); } - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false, false).build(), + true)) { String newCommitTime = client.startCommit(); // Next insert List records = dataGen.generateInserts(newCommitTime, 5); @@ -1163,6 +1491,95 @@ public void testMetadataMetrics() throws Exception { } } + /** + * Fetching WriteConfig for metadata table from Data table's writeConfig is not trivial and the method is not public in source code. so, for now, + * using this method which mimics source code. + * @param writeConfig + * @return + */ + private HoodieWriteConfig getMetadataWriteConfig(HoodieWriteConfig writeConfig) { + int parallelism = writeConfig.getMetadataInsertParallelism(); + + int minCommitsToKeep = Math.max(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMinCommitsToKeep()); + int maxCommitsToKeep = Math.max(writeConfig.getMetadataMaxCommitsToKeep(), writeConfig.getMaxCommitsToKeep()); + + // Create the write config for the metadata table by borrowing options from the main write config. + HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() + .withConsistencyCheckEnabled(writeConfig.getConsistencyGuardConfig().isConsistencyCheckEnabled()) + .withInitialConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getInitialConsistencyCheckIntervalMs()) + .withMaxConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getMaxConsistencyCheckIntervalMs()) + .withMaxConsistencyChecks(writeConfig.getConsistencyGuardConfig().getMaxConsistencyChecks()) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.SINGLE_WRITER) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).withFileListingParallelism(writeConfig.getFileListingParallelism()).build()) + .withAutoCommit(true) + .withAvroSchemaValidate(true) + .withEmbeddedTimelineServerEnabled(false) + .withMarkersType(MarkerType.DIRECT.name()) + .withRollbackUsingMarkers(false) + .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) + .withSchema(HoodieMetadataRecord.getClassSchema().toString()) + .forTable(writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withAsyncClean(writeConfig.isMetadataAsyncClean()) + // we will trigger cleaning manually, to control the instant times + .withAutoClean(false) + .withCleanerParallelism(parallelism) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) + .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) + // we will trigger compaction manually, to control the instant times + .withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()).build()) + .withParallelism(parallelism, parallelism) + .withDeleteParallelism(parallelism) + .withRollbackParallelism(parallelism) + .withFinalizeWriteParallelism(parallelism) + .withAllowMultiWriteOnSameInstant(true) + .withKeyGenerator(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) + .withPopulateMetaFields(writeConfig.getMetadataConfig().populateMetaFields()); + + // RecordKey properties are needed for the metadata table records + final Properties properties = new Properties(); + properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY); + properties.put("hoodie.datasource.write.recordkey.field", HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY); + builder.withProperties(properties); + + if (writeConfig.isMetricsOn()) { + builder.withMetricsConfig(HoodieMetricsConfig.newBuilder() + .withReporterType(writeConfig.getMetricsReporterType().toString()) + .withExecutorMetrics(writeConfig.isExecutorMetricsEnabled()) + .on(true).build()); + switch (writeConfig.getMetricsReporterType()) { + case GRAPHITE: + builder.withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() + .onGraphitePort(writeConfig.getGraphiteServerPort()) + .toGraphiteHost(writeConfig.getGraphiteServerHost()) + .usePrefix(writeConfig.getGraphiteMetricPrefix()).build()); + break; + case JMX: + builder.withMetricsJmxConfig(HoodieMetricsJmxConfig.newBuilder() + .onJmxPort(writeConfig.getJmxPort()) + .toJmxHost(writeConfig.getJmxHost()) + .build()); + break; + case DATADOG: + case PROMETHEUS: + case PROMETHEUS_PUSHGATEWAY: + case CONSOLE: + case INMEMORY: + case CLOUDWATCH: + break; + default: + throw new HoodieMetadataException("Unsupported Metrics Reporter type " + writeConfig.getMetricsReporterType()); + } + } + return builder.build(); + } + private void doPreBootstrapOperations(HoodieTestTable testTable) throws Exception { doPreBootstrapOperations(testTable, "0000001", "0000002"); } @@ -1175,8 +1592,12 @@ private void doPreBootstrapOperations(HoodieTestTable testTable, String commit1, validateMetadata(testTable); } + private void doWriteInsertAndUpsertNonPartitioned(HoodieTestTable testTable) throws Exception { + doWriteInsertAndUpsert(testTable, "0000001", "0000002", true); + } + private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { - doWriteInsertAndUpsert(testTable, "0000001", "0000002"); + doWriteInsertAndUpsert(testTable, "0000001", "0000002", false); } private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) { @@ -1210,6 +1631,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex. .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server + .withRemoteServerPort(timelineServicePort) .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 5242e9f33766..d6f151e34255 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -18,10 +18,12 @@ package org.apache.hudi.client.functional; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -29,6 +31,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.io.IOException; import java.util.ArrayList; @@ -57,7 +61,7 @@ public void testTableOperations() throws Exception { } private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { - doWriteInsertAndUpsert(testTable, "0000001", "0000002"); + doWriteInsertAndUpsert(testTable, "0000001", "0000002", false); } private void verifyBaseMetadataTable() throws IOException { @@ -90,4 +94,33 @@ private void verifyBaseMetadataTable() throws IOException { }); } + /** + * Verify if the Metadata table is constructed with table properties including + * the right key generator class name. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throws Exception { + init(tableType); + + HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, + writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + + assertEquals(HoodieTableMetadataKeyGenerator.class.getCanonicalName(), + tableMetadata.getMetadataMetaClient().getTableConfig().getKeyGeneratorClassName()); + } + + /** + * [HUDI-2852] Table metadata returns empty for non-exist partition. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testNotExistPartition(final HoodieTableType tableType) throws Exception { + init(tableType); + HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, + writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + FileStatus[] allFilesInPartition = + tableMetadata.getAllFilesInPartition(new Path(writeConfig.getBasePath() + "dummy")); + assertEquals(allFilesInPartition.length, 0); + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index caffb476b840..566db224e61e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -47,11 +47,13 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -61,6 +63,7 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -129,6 +132,14 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; import static org.apache.hudi.common.testutils.FileCreateUtils.getBaseFileCountsForPaths; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; @@ -495,7 +506,7 @@ void assertNodupesInPartition(List records) { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testUpserts(boolean populateMetaFields) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withRollbackUsingMarkers(true); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsert, false); } @@ -506,7 +517,7 @@ public void testUpserts(boolean populateMetaFields) throws Exception { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testUpsertsPrepped(boolean populateMetaFields) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withRollbackUsingMarkers(true); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsertPreppedRecords, true); } @@ -523,6 +534,7 @@ private void testUpsertsInternal(HoodieWriteConfig config, throws Exception { // Force using older timeline layout HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withRollbackUsingMarkers(true) .withProps(config.getProps()).withTimelineLayoutVersion( VERSION_0).build(); @@ -585,16 +597,16 @@ private void testUpsertsInternal(HoodieWriteConfig config, HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false); List instants = activeTimeline.getCommitTimeline().getInstants().collect(Collectors.toList()); assertEquals(5, instants.size()); - assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"), + assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"), instants.get(0)); - assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "004"), + assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "004"), instants.get(1)); // New Format should have all states of instants - assertEquals(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "006"), + assertEquals(new HoodieInstant(REQUESTED, COMMIT_ACTION, "006"), instants.get(2)); - assertEquals(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "006"), + assertEquals(new HoodieInstant(INFLIGHT, COMMIT_ACTION, "006"), instants.get(3)); - assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "006"), + assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "006"), instants.get(4)); final HoodieWriteConfig cfg = hoodieWriteConfig; @@ -1402,15 +1414,14 @@ public void testPendingClusteringRollback(boolean populateMetaFields) throws Exc HoodieInstant pendingClusteringInstant = pendingClusteringPlans.get(0).getLeft(); // complete another commit after pending clustering - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(EAGER); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig config = cfgBuilder.build(); SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); String commitTime = HoodieActiveTimeline.createNewInstantTime(); allRecords.addAll(dataGen.generateInserts(commitTime, 200)); - writeAndVerifyBatch(client, allRecords, commitTime, populateMetaFields); - + assertThrows(HoodieUpsertException.class, () -> writeAndVerifyBatch(client, allRecords, commitTime, populateMetaFields)); // verify pending clustering can be rolled back (even though there is a completed commit greater than pending clustering) client.rollback(pendingClusteringInstant.getTimestamp()); metaClient.reloadActiveTimeline(); @@ -1418,6 +1429,41 @@ public void testPendingClusteringRollback(boolean populateMetaFields) throws Exc assertEquals(0, ClusteringUtils.getAllPendingClusteringPlans(metaClient).count()); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testInflightClusteringRollbackWhenUpdatesAllowed(boolean rollbackPendingClustering) throws Exception { + // setup clustering config with update strategy to allow updates during ingestion + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder() + .withClusteringMaxNumGroups(10).withClusteringTargetPartitions(0) + .withClusteringUpdatesStrategy("org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy") + .withRollbackPendingClustering(rollbackPendingClustering) + .withInlineClustering(true).withInlineClusteringNumCommits(1).build(); + + // start clustering, but don't commit keep it inflight + List allRecords = testInsertAndClustering(clusteringConfig, true, false); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); + List> pendingClusteringPlans = + ClusteringUtils.getAllPendingClusteringPlans(metaClient).collect(Collectors.toList()); + assertEquals(1, pendingClusteringPlans.size()); + HoodieInstant pendingClusteringInstant = pendingClusteringPlans.get(0).getLeft(); + assertEquals(pendingClusteringInstant.getState(), INFLIGHT); + + // make an update to a filegroup within the partition that is pending clustering + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(EAGER); + addConfigsForPopulateMetaFields(cfgBuilder, true); + cfgBuilder.withClusteringConfig(clusteringConfig); + HoodieWriteConfig config = cfgBuilder.build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + allRecords.addAll(dataGen.generateUpdates(commitTime, 200)); + writeAndVerifyBatch(client, allRecords, commitTime, true); + + // verify inflight clustering was rolled back + metaClient.reloadActiveTimeline(); + pendingClusteringPlans = ClusteringUtils.getAllPendingClusteringPlans(metaClient).collect(Collectors.toList()); + assertEquals(config.isRollbackPendingClustering() ? 0 : 1, pendingClusteringPlans.size()); + } + @Test public void testClusteringWithFailingValidator() throws Exception { // setup clustering config. @@ -1621,7 +1667,7 @@ private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int // Do Insert Overwrite String commitTime2 = "002"; - client.startCommitWithTime(commitTime2, HoodieTimeline.REPLACE_COMMIT_ACTION); + client.startCommitWithTime(commitTime2, REPLACE_COMMIT_ACTION); List inserts2 = dataGen.generateInserts(commitTime2, batch2RecordsCount); List insertsAndUpdates2 = new ArrayList<>(); insertsAndUpdates2.addAll(inserts2); @@ -1677,7 +1723,7 @@ private Set insertPartitionRecordsWithCommit(SparkRDDWriteClient client, } private Set deletePartitionWithCommit(SparkRDDWriteClient client, String commitTime, List deletePartitionPath) { - client.startCommitWithTime(commitTime, HoodieTimeline.REPLACE_COMMIT_ACTION); + client.startCommitWithTime(commitTime, REPLACE_COMMIT_ACTION); HoodieWriteResult writeResult = client.deletePartitions(deletePartitionPath, commitTime); Set deletePartitionReplaceFileIds = writeResult.getPartitionToReplaceFileIds().entrySet() @@ -2036,7 +2082,7 @@ public void testConsistencyCheckDuringFinalize(boolean enableOptimisticConsisten private void testRollbackAfterConsistencyCheckFailureUsingFileList(boolean rollbackUsingMarkers, boolean enableOptimisticConsistencyGuard, boolean populateMetaFields) throws Exception { - String instantTime = "000"; + String instantTime = "00000000000010"; HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); Properties properties = new Properties(); @@ -2102,14 +2148,20 @@ public void testRollbackAfterConsistencyCheckFailureUsingMarkers(boolean enableO @MethodSource("rollbackFailedCommitsParams") public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) throws Exception { HoodieTestUtils.init(hadoopConf, basePath); - // Perform 2 failed writes to table SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); + + // perform 1 successfull commit writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, true); + + // Perform 2 failed writes to table + writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "100", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", + writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); @@ -2117,24 +2169,24 @@ public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningP dataGen = new HoodieTestDataGenerator(); // Perform 1 successful write client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", + writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, true); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); assertTrue(metaClient.getActiveTimeline().getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 0); assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); - assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); + assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 2); // Await till enough time passes such that the first 2 failed commits heartbeats are expired boolean conditionMet = false; while (!conditionMet) { - conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300"); Thread.sleep(2000); } client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); // Perform 1 successful write - writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", + writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, true); client.clean(); @@ -2142,29 +2194,29 @@ public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningP if (cleaningPolicy.isLazy()) { assertTrue( timeline - .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)) + .getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION)) .countInstants() == 2); // Since we write rollbacks not clean, there should be no clean action on the timeline assertTrue( timeline - .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)) + .getTimelineOfActions(CollectionUtils.createSet(CLEAN_ACTION)) .countInstants() == 0); - assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3); } else if (cleaningPolicy.isNever()) { assertTrue( timeline - .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)) + .getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION)) .countInstants() == 0); // There should be no clean or rollback action on the timeline assertTrue( timeline - .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)) + .getTimelineOfActions(CollectionUtils.createSet(CLEAN_ACTION)) .countInstants() == 0); - assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3); } } @@ -2172,10 +2224,15 @@ public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningP @MethodSource("populateMetaFieldsParams") public void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMetaFields) throws Exception { HoodieTestUtils.init(hadoopConf, basePath); - HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; + HoodieFailedWritesCleaningPolicy cleaningPolicy = EAGER; SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - // Perform 1 failed writes to table + // Perform 1 successful writes to table writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, true); + + // Perform 1 failed writes to table + writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); @@ -2183,44 +2240,44 @@ public void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMetaFi cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; // Perform 2 failed writes to table client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", + writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", + writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); - // Await till enough time passes such that the first 2 failed commits heartbeats are expired + // Await till enough time passes such that the 2 failed commits heartbeats are expired boolean conditionMet = false; while (!conditionMet) { - conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300"); + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("400"); Thread.sleep(2000); } client.clean(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 3); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 3); // Perform 2 failed commits client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", + writeBatch(client, "500", "400", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); - writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500", + writeBatch(client, "600", "500", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, 0, false); client.close(); // Toggle cleaning policy to EAGER - cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; + cleaningPolicy = EAGER; client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); client.startCommit(); timeline = metaClient.getActiveTimeline().reload(); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 5); - assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 5); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 1); } @Test @@ -2228,14 +2285,19 @@ public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception { HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; ExecutorService service = Executors.newFixedThreadPool(2); HoodieTestUtils.init(hadoopConf, basePath); - // Perform 2 failed writes to table SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); + // perform 1 successfull write writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, + 0, true); + + // Perform 2 failed writes to table + writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); client.close(); client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); - writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200", + writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); client.close(); @@ -2243,35 +2305,35 @@ public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception { dataGen = new HoodieTestDataGenerator(); // Create a succesful commit Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)), - "300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts, + "400", "300", Option.of(Arrays.asList("400")), "300", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); commit3.get(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); assertTrue(metaClient.getActiveTimeline().getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 0); assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); - assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); + assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 2); client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); // Await till enough time passes such that the first 2 failed commits heartbeats are expired boolean conditionMet = false; while (!conditionMet) { - conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300"); Thread.sleep(2000); } Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)), - "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, + "500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)).clean()); commit4.get(); clean1.get(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 2); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 2); // Since we write rollbacks not clean, there should be no clean action on the timeline assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).countInstants() == 0); - assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + CollectionUtils.createSet(CLEAN_ACTION)).countInstants() == 0); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3); } private Pair> testConsistencyCheck(HoodieTableMetaClient metaClient, String instantTime, boolean enableOptimisticConsistencyGuard) @@ -2293,13 +2355,22 @@ private Pair> testConsistencyCheck(HoodieTableMetaCli // Create a dummy marker file to simulate the case that a marker file was created without data file. // This should fail the commit - String partitionPath = Arrays - .stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", metaClient.getMarkerFolderPath(instantTime))), - path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN))) - .limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0); + String partitionPath; + String markerFolderPath = metaClient.getMarkerFolderPath(instantTime); + if (cfg.getMarkersType() == MarkerType.TIMELINE_SERVER_BASED) { + String markerName = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem( + markerFolderPath, fs, context, 1).values().stream() + .flatMap(Collection::stream).findFirst().get(); + partitionPath = new Path(markerFolderPath, markerName).getParent().toString(); + } else { + partitionPath = Arrays + .stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", markerFolderPath)), + path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN))) + .limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0); + } Option markerFilePath = WriteMarkersFactory.get( - cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime) + cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime) .create(partitionPath, FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()), IOType.MERGE); @@ -2431,7 +2502,7 @@ protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient meta HoodieClusteringPlan clusteringPlan = ClusteringUtils.createClusteringPlan(EXECUTION_STRATEGY_CLASS_NAME.defaultValue(), STRATEGY_PARAMS, fileSlices, Collections.emptyMap()); - HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime); + HoodieInstant clusteringInstant = new HoodieInstant(REQUESTED, REPLACE_COMMIT_ACTION, clusterTime); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); metaClient.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); @@ -2446,6 +2517,8 @@ private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleani .withAutoClean(false).build()) .withTimelineLayoutVersion(1) .withHeartbeatIntervalInMs(3 * 1000) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withAutoCommit(false) .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 5c75bb565a31..6cd25f399225 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; @@ -35,12 +36,16 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLayoutConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner; import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; @@ -86,7 +91,8 @@ private static Stream indexTypeParams() { {IndexType.SIMPLE, true}, {IndexType.GLOBAL_SIMPLE, true}, {IndexType.SIMPLE, false}, - {IndexType.GLOBAL_SIMPLE, false} + {IndexType.GLOBAL_SIMPLE, false}, + {IndexType.BUCKET, false} }; return Stream.of(data).map(Arguments::of); } @@ -98,10 +104,10 @@ private static Stream indexTypeParams() { private HoodieWriteConfig config; private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields, true); + setUp(indexType, populateMetaFields, true, true); } - private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata) throws Exception { + private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata, boolean rollbackUsingMarkers) throws Exception { this.indexType = indexType; initPath(); initSparkContexts(); @@ -109,10 +115,16 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean enab initFileSystem(); metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, populateMetaFields ? new Properties() : getPropertiesForKeyGen()); + HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(indexType) + .fromProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) + .withIndexType(indexType); config = getConfigBuilder() .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) - .build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build(); + .withRollbackUsingMarkers(rollbackUsingMarkers) + .withIndexConfig(indexBuilder + .build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()) + .withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps()) + .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()).build(); writeClient = getHoodieWriteClient(config); this.index = writeClient.getIndex(); } @@ -225,7 +237,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul @ParameterizedTest @MethodSource("indexTypeParams") public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields, false); + setUp(indexType, populateMetaFields, true, false); String newCommitTime = writeClient.startCommit(); int totalRecords = 20 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); @@ -235,7 +247,7 @@ public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, bool // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); Assertions.assertNoWriteErrors(writeStatues.collect()); - + List fileIds = writeStatues.map(WriteStatus::getFileId).collect(); // commit this upsert writeClient.commit(newCommitTime, writeStatues); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); @@ -245,7 +257,6 @@ public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, bool assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords); // check tagged records are tagged with correct fileIds - List fileIds = writeStatues.map(WriteStatus::getFileId).collect(); assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0); List taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect(); @@ -373,11 +384,18 @@ public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() thro .withGlobalSimpleIndexUpdatePartitionPath(true) .withBloomIndexUpdatePartitionPath(true) .build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); + .withMetadataConfig( + HoodieMetadataConfig.newBuilder().enable(true).build()) + .build(); writeClient = getHoodieWriteClient(config); index = writeClient.getIndex(); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( + writeClient.getEngineContext().getHadoopConf().get(), config, writeClient.getEngineContext()); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable.getMetaClient(), + SCHEMA, metadataWriter); + final String p1 = "2016/01/31"; final String p2 = "2016/02/28"; @@ -414,8 +432,14 @@ public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() thro new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()), incomingPayloadSamePartition); + final String file1P1C0 = UUID.randomUUID().toString(); + Map>> c1PartitionToFilesNameLengthMap = new HashMap<>(); + c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, 100))); + testTable.doWriteOperation("1000", WriteOperationType.INSERT, Arrays.asList(p1), + c1PartitionToFilesNameLengthMap, false, false); + // We have some records to be tagged (two different partitions) - testTable.addCommit("1000").getFileIdWithInserts(p1, originalRecord); + testTable.withInserts(p1, file1P1C0, originalRecord); // test against incoming record with a different partition JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord)); @@ -457,7 +481,6 @@ private HoodieWriteConfig.Builder getConfigBuilder() { .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index cf261cc8994a..56c9f016bcc6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -75,20 +75,22 @@ public void init(HoodieTableType tableType) throws IOException { } public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException { - init(tableType, enableMetadataTable, true); + init(tableType, enableMetadataTable, true, false, false); } - public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan) throws IOException { + public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean + validateMetadataPayloadStateConsistency) throws IOException { this.tableType = tableType; initPath(); initSparkContexts("TestHoodieMetadata"); initFileSystem(); fs.mkdirs(new Path(basePath)); + initTimelineService(); initMetaClient(tableType); initTestDataGenerator(); metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); - writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, false, - enableFullScan).build(); + writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, + enableFullScan, true, validateMetadataPayloadStateConsistency).build(); initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable); } @@ -107,10 +109,10 @@ public void clean() throws Exception { cleanupResources(); } - protected void doWriteInsertAndUpsert(HoodieTestTable testTable, String commit1, String commit2) throws Exception { - testTable.doWriteOperation(commit1, INSERT, asList("p1", "p2"), asList("p1", "p2"), + protected void doWriteInsertAndUpsert(HoodieTestTable testTable, String commit1, String commit2, boolean nonPartitioned) throws Exception { + testTable.doWriteOperation(commit1, INSERT, nonPartitioned ? asList("") : asList("p1", "p2"), nonPartitioned ? asList("") : asList("p1", "p2"), 4, false); - testTable.doWriteOperation(commit2, UPSERT, asList("p1", "p2"), + testTable.doWriteOperation(commit2, UPSERT, nonPartitioned ? asList("") : asList("p1", "p2"), 4, false); validateMetadata(testTable); } @@ -133,6 +135,18 @@ protected void doWriteOperationAndValidate(HoodieTestTable testTable, String com validateMetadata(testTable); } + protected void doWriteOperationNonPartitioned(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception { + testTable.doWriteOperation(commitTime, operationType, emptyList(), asList(""), 3); + } + + protected void doWriteOperation(HoodieTestTable testTable, String commitTime, WriteOperationType operationType, boolean nonPartitioned) throws Exception { + if (nonPartitioned) { + doWriteOperationNonPartitioned(testTable, commitTime, operationType); + } else { + doWriteOperation(testTable, commitTime, operationType); + } + } + protected void doWriteOperation(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception { testTable.doWriteOperation(commitTime, operationType, emptyList(), asList("p1", "p2"), 3); } @@ -152,16 +166,28 @@ private void doCleanInternal(HoodieTestTable testTable, String commitTime, List< } } + protected void doCompactionNonPartitioned(HoodieTestTable testTable, String commitTime) throws Exception { + doCompactionInternal(testTable, commitTime, false, true); + } + + protected void doCompaction(HoodieTestTable testTable, String commitTime, boolean nonPartitioned) throws Exception { + doCompactionInternal(testTable, commitTime, false, nonPartitioned); + } + protected void doCompaction(HoodieTestTable testTable, String commitTime) throws Exception { - doCompactionInternal(testTable, commitTime, false); + doCompactionInternal(testTable, commitTime, false, false); + } + + protected void doCompactionNonPartitionedAndValidate(HoodieTestTable testTable, String commitTime) throws Exception { + doCompactionInternal(testTable, commitTime, true, true); } protected void doCompactionAndValidate(HoodieTestTable testTable, String commitTime) throws Exception { - doCompactionInternal(testTable, commitTime, true); + doCompactionInternal(testTable, commitTime, true, false); } - private void doCompactionInternal(HoodieTestTable testTable, String commitTime, boolean validate) throws Exception { - testTable.doCompaction(commitTime, asList("p1", "p2")); + private void doCompactionInternal(HoodieTestTable testTable, String commitTime, boolean validate, boolean nonPartitioned) throws Exception { + testTable.doCompaction(commitTime, nonPartitioned ? asList("") : asList("p1", "p2")); if (validate) { validateMetadata(testTable); } @@ -266,11 +292,12 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, bo protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { - return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true); + return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true, true, false); } protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, - boolean enableMetrics, boolean enableFullScan) { + boolean enableMetrics, boolean enableFullScan, boolean useRollbackUsingMarkers, + boolean validateMetadataPayloadConsistency) { Properties properties = new Properties(); properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName()); return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) @@ -288,11 +315,15 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(useFileListingMetadata) .enableFullScan(enableFullScan) - .enableMetrics(enableMetrics).build()) + .enableMetrics(enableMetrics) + .withPopulateMetaFields(HoodieMetadataConfig.POPULATE_META_FIELDS.defaultValue()) + .ignoreSpuriousDeletes(validateMetadataPayloadConsistency) + .build()) .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) .withExecutorMetrics(true).build()) .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() .usePrefix("unit-test").build()) + .withRollbackUsingMarkers(useRollbackUsingMarkers) .withProperties(properties); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java index 12c8410c35e0..baf5e7437dc3 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -234,13 +235,15 @@ private void bootstrapAndVerifyFailure() throws Exception { } private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { - doWriteInsertAndUpsert(testTable, "0000100", "0000101"); + doWriteInsertAndUpsert(testTable, "0000100", "0000101", false); } private HoodieWriteConfig getWriteConfig(int minArchivalCommits, int maxArchivalCommits) throws Exception { return HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 2fb364187598..665e3a6a8e4a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -32,6 +32,7 @@ import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; +import org.apache.hudi.index.bucket.HoodieBucketIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; import org.apache.hudi.index.simple.HoodieSimpleIndex; @@ -60,7 +61,7 @@ public void setUp(@TempDir Path tempDir) { } @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE"}) + @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "BUCKET"}) public void testCreateIndex(IndexType indexType) throws Exception { HoodieWriteConfig config; HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); @@ -93,6 +94,11 @@ public void testCreateIndex(IndexType indexType) throws Exception { .build(); assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkHoodieHBaseIndex); break; + case BUCKET: + config = clientConfigBuilder.withPath(basePath) + .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET).build()).build(); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBucketIndex); + break; default: // no -op. just for checkstyle errors } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java new file mode 100644 index 000000000000..6cf29df5140a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java @@ -0,0 +1,65 @@ +/* + * 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 org.apache.hudi.index.bucket; + +import java.util.Arrays; +import java.util.List; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.keygen.KeyGenUtils; +import org.apache.hudi.testutils.KeyGeneratorTestUtilities; +import org.junit.jupiter.api.Test; + +public class TestBucketIdentifier { + + @Test + public void testBucketFileId() { + for (int i = 0; i < 1000; i++) { + String bucketId = BucketIdentifier.bucketIdStr(i); + String fileId = BucketIdentifier.newBucketFileIdPrefix(bucketId); + assert BucketIdentifier.bucketIdFromFileId(fileId) == i; + } + } + + @Test + public void testBucketIdWithSimpleRecordKey() { + String recordKeyField = "_row_key"; + String indexKeyField = "_row_key"; + GenericRecord record = KeyGeneratorTestUtilities.getRecord(); + HoodieRecord hoodieRecord = new HoodieRecord( + new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null); + int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); + assert bucketId == BucketIdentifier.getBucketId( + Arrays.asList(record.get(indexKeyField).toString()), 8); + } + + @Test + public void testBucketIdWithComplexRecordKey() { + List recordKeyField = Arrays.asList("_row_key","ts_ms"); + String indexKeyField = "_row_key"; + GenericRecord record = KeyGeneratorTestUtilities.getRecord(); + HoodieRecord hoodieRecord = new HoodieRecord( + new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null); + int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); + assert bucketId == BucketIdentifier.getBucketId( + Arrays.asList(record.get(indexKeyField).toString()), 8); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java new file mode 100644 index 000000000000..c79f9aec773e --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java @@ -0,0 +1,147 @@ +/* + * 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 org.apache.hudi.index.bucket; + +import org.apache.avro.Schema; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Properties; +import java.util.UUID; + +import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieBucketIndex extends HoodieClientTestHarness { + + private static final Logger LOG = LogManager.getLogger(TestHoodieBucketIndex.class); + private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBucketIndex.class, "/exampleSchema.avsc", true); + private static final int NUM_BUCKET = 8; + + @BeforeEach + public void setUp() throws Exception { + initSparkContexts(); + initPath(); + initFileSystem(); + // We have some records to be tagged (two different partitions) + initMetaClient(); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + @Test + public void testBucketIndexValidityCheck() { + Properties props = new Properties(); + props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "_row_key"); + assertThrows(HoodieIndexException.class, () -> { + HoodieIndexConfig.newBuilder().fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build(); + }); + props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "uuid"); + HoodieIndexConfig.newBuilder().fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build(); + } + + @Test + public void testTagLocation() throws Exception { + String rowKey1 = UUID.randomUUID().toString(); + String rowKey2 = UUID.randomUUID().toString(); + String rowKey3 = UUID.randomUUID().toString(); + String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); + HoodieRecord record1 = new HoodieRecord( + new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); + HoodieRecord record2 = new HoodieRecord( + new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); + HoodieRecord record3 = new HoodieRecord( + new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); + HoodieRecord record4 = new HoodieRecord( + new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); + + HoodieWriteConfig config = makeConfig(); + HoodieTable table = HoodieSparkTable.create(config, context, metaClient); + HoodieBucketIndex bucketIndex = new HoodieBucketIndex(config); + HoodieData taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table); + assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown())); + + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA); + testTable.addCommit("001").withInserts("2016/01/31", getRecordFileId(record1), record1); + testTable.addCommit("002").withInserts("2016/01/31", getRecordFileId(record2), record2); + testTable.addCommit("003").withInserts("2016/01/31", getRecordFileId(record3), record3); + taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, + HoodieSparkTable.create(config, context, metaClient)); + assertFalse(taggedRecordRDD.collectAsList().stream().filter(r -> r.isCurrentLocationKnown()) + .filter(r -> BucketIdentifier.bucketIdFromFileId(r.getCurrentLocation().getFileId()) + != getRecordBucketId(r)).findAny().isPresent()); + assertTrue(taggedRecordRDD.collectAsList().stream().filter(r -> r.getPartitionPath().equals("2015/01/31") + && !r.isCurrentLocationKnown()).count() == 1L); + } + + private HoodieWriteConfig makeConfig() { + Properties props = new Properties(); + props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString()) + .withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withIndexKeyField("_row_key") + .withBucketNum(String.valueOf(NUM_BUCKET)).build()).build(); + } + + private String getRecordFileId(HoodieRecord record) { + return BucketIdentifier.bucketIdStr( + BucketIdentifier.getBucketId(record, "_row_key", NUM_BUCKET)); + } + + private int getRecordBucketId(HoodieRecord record) { + return BucketIdentifier + .getBucketId(record, "_row_key", NUM_BUCKET); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/columnstats/TestColumnStatsIndexHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/columnstats/TestColumnStatsIndexHelper.java new file mode 100644 index 000000000000..3901a9378390 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/columnstats/TestColumnStatsIndexHelper.java @@ -0,0 +1,41 @@ +/* + * 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 org.apache.hudi.index.columnstats; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestColumnStatsIndexHelper { + + @Test + public void testMergeSql() { + String q = ColumnStatsIndexHelper.createIndexMergeSql("old", "new", Arrays.asList("file", "a", "b")); + assertEquals( + "SELECT " + + "if (new.file is null, old.file, new.file) AS file, " + + "if (new.a is null, old.a, new.a) AS a, " + + "if (new.b is null, old.b, new.b) AS b " + + "FROM old FULL JOIN new ON old.file = new.file", q); + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index 7cb9740a8c6c..faa06210d941 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -18,15 +18,10 @@ package org.apache.hudi.io; -import org.apache.hudi.avro.model.HoodieActionInstant; -import org.apache.hudi.avro.model.HoodieCleanMetadata; -import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.utils.MetadataConversionUtils; -import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; @@ -35,12 +30,11 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; -import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -69,11 +63,9 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -94,6 +86,7 @@ public void init() throws Exception { public void init(HoodieTableType tableType) throws Exception { initPath(); initSparkContexts(); + initTimelineService(); initMetaClient(); hadoopConf = context.getHadoopConf().get(); metaClient.getFs().mkdirs(new Path(basePath)); @@ -126,6 +119,8 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata) .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build()) .forTable("test-trip-table").build(); @@ -210,6 +205,8 @@ public void testArchiveCommitSavepointNoHole() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); @@ -237,6 +234,25 @@ public void testArchiveCommitSavepointNoHole() throws Exception { "Archived commits should always be safe"); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPendingClusteringWillBlockArchival(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 5, 2); + HoodieTestDataGenerator.createPendingReplaceFile(basePath, "00000000", wrapperFs.getConf()); + for (int i = 1; i < 8; i++) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2); + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + } + + HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertEquals(7, timeline.countInstants(), + "Since we have a pending clustering instant at 00000000, we should never archive any commit after 00000000"); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void testArchiveRollbacksTestTable(boolean enableMetadata) throws Exception { @@ -328,6 +344,8 @@ public void testArchiveCommitTimeline() throws Exception { HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -475,8 +493,9 @@ public void testArchiveRollbacksAndCleanTestTable() throws Exception { HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival); } - @Test - public void testArchiveCompletedRollbackAndClean() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testArchiveCompletedRollbackAndClean(boolean isEmpty) throws Exception { init(); int minInstantsToKeep = 2; int maxInstantsToKeep = 10; @@ -484,17 +503,19 @@ public void testArchiveCompletedRollbackAndClean() throws Exception { HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); int startInstant = 1; for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant++) { - createCleanMetadata(startInstant + "", false); + createCleanMetadata(startInstant + "", false, isEmpty || i % 2 == 0); } for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant += 2) { - createCommitAndRollbackFile(startInstant + 1 + "", startInstant + "", false); + createCommitAndRollbackFile(startInstant + 1 + "", startInstant + "", false, isEmpty || i % 2 == 0); } HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); @@ -519,6 +540,8 @@ public void testArchiveInflightClean() throws Exception { HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -670,31 +693,16 @@ private List getActiveCommitInstants(List commitTimes, St return allInstants; } - private HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly) throws IOException { - HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), - CleanPlanV2MigrationHandler.VERSION, new HashMap<>()); - if (inflightOnly) { - HoodieTestTable.of(metaClient).addInflightClean(instantTime, cleanerPlan); - } else { - HoodieCleanStat cleanStats = new HoodieCleanStat( - HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, - HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEFAULT_PARTITION_PATHS.length)], - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - instantTime); - HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats)); - HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata); - } - return new HoodieInstant(inflightOnly, "clean", instantTime); + private void createCommitAndRollbackFile(String commitToRollback, String rollbackTIme, boolean isRollbackInflight) throws IOException { + createCommitAndRollbackFile(commitToRollback, rollbackTIme, isRollbackInflight, false); } - private void createCommitAndRollbackFile(String commitToRollback, String rollbackTIme, boolean isRollbackInflight) throws IOException { + private void createCommitAndRollbackFile(String commitToRollback, String rollbackTIme, boolean isRollbackInflight, boolean isEmpty) throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, commitToRollback, wrapperFs.getConf()); - createRollbackMetadata(rollbackTIme, commitToRollback, isRollbackInflight); + createRollbackMetadata(rollbackTIme, commitToRollback, isRollbackInflight, isEmpty); } - private HoodieInstant createRollbackMetadata(String rollbackTime, String commitToRollback, boolean inflight) throws IOException { + private HoodieInstant createRollbackMetadata(String rollbackTime, String commitToRollback, boolean inflight, boolean isEmpty) throws IOException { if (inflight) { HoodieTestTable.of(metaClient).addInflightRollback(rollbackTime); } else { @@ -707,7 +715,7 @@ private HoodieInstant createRollbackMetadata(String rollbackTime, String commitT .setPartitionMetadata(Collections.emptyMap()) .setInstantsRollback(Collections.emptyList()) .build(); - HoodieTestTable.of(metaClient).addRollback(rollbackTime, hoodieRollbackMetadata); + HoodieTestTable.of(metaClient).addRollback(rollbackTime, hoodieRollbackMetadata, isEmpty); } return new HoodieInstant(inflight, "rollback", rollbackTime); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java index de5555543465..dd509a86b4e9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java @@ -18,8 +18,6 @@ package org.apache.hudi.io.storage.row; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -27,6 +25,9 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.SparkDatasetTestUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -66,7 +67,8 @@ public void tearDown() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception { - HoodieWriteConfig.Builder writeConfigBuilder = SparkDatasetTestUtils.getConfigBuilder(basePath); + HoodieWriteConfig.Builder writeConfigBuilder = + SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort); for (int i = 0; i < 5; i++) { // init write support and parquet config HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java index 76a91ef124bb..0f308425bc1c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java @@ -28,8 +28,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; - import org.apache.hudi.testutils.SparkDatasetTestUtils; + import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; @@ -65,6 +65,7 @@ public void setUp() throws Exception { initFileSystem(); initTestDataGenerator(); initMetaClient(); + initTimelineService(); } @AfterEach @@ -75,7 +76,8 @@ public void tearDown() throws Exception { @Test public void testRowCreateHandle() throws Exception { // init config and table - HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); + HoodieWriteConfig cfg = + SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); List fileNames = new ArrayList<>(); List fileAbsPaths = new ArrayList<>(); @@ -116,7 +118,8 @@ public void testRowCreateHandle() throws Exception { @Test public void testGlobalFailure() throws Exception { // init config and table - HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); + HoodieWriteConfig cfg = + SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; @@ -124,7 +127,8 @@ public void testGlobalFailure() throws Exception { String fileId = UUID.randomUUID().toString(); String instantTime = "000"; - HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); + HoodieRowCreateHandle handle = + new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); int totalFailures = 5; // Generate first batch of valid rows @@ -169,7 +173,8 @@ public void testGlobalFailure() throws Exception { @Test public void testInstantiationFailure() throws IOException { // init config and table - HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .withPath("/dummypath/abc/").build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index cb468e903e59..8a1f4abd29ce 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -18,8 +18,6 @@ package org.apache.hudi.table; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; @@ -62,6 +60,7 @@ import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV1MigrationHandler; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -81,6 +80,9 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -89,7 +91,6 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; -import scala.Tuple3; import java.io.File; import java.io.IOException; @@ -111,6 +112,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import scala.Tuple3; + import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; @@ -256,7 +259,7 @@ public void testBulkInsertPreppedAndCleanByVersions() throws Exception { * @param insertFn Insert API to be tested * @param upsertFn Upsert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use unique record-gen APIs) * @throws Exception in case of errors */ private void testInsertAndCleanByVersions( @@ -271,7 +274,7 @@ private void testInsertAndCleanByVersions( .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) .build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { final Function2, String, Integer> recordInsertGenWrappedFunction = generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); @@ -426,7 +429,7 @@ public void testBulkInsertAndCleanByCommits() throws Exception { * @param insertFn Insert API to be tested * @param upsertFn Upsert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use unique record-gen APIs) * @throws Exception in case of errors */ private void testInsertAndCleanByCommits( @@ -547,10 +550,10 @@ private void testFailedInsertAndCleanByCommits( HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); assertTrue(timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().countInstants() == 3); - Option rolleBackInstantForFailedCommit = timeline.getTimelineOfActions( + Option rollBackInstantForFailedCommit = timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.getInstantDetails(rolleBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + timeline.getInstantDetails(rollBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); // Rollback of one of the failed writes should have deleted 3 files assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); } @@ -747,6 +750,59 @@ public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throw assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); } + @Test + public void testCleanEmptyInstants() throws Exception { + HoodieWriteConfig config = + HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) + .build(); + metaClient = HoodieTableMetaClient.reload(metaClient); + + int commitCount = 20; + int cleanCount = 20; + + int startInstant = 1; + for (int i = 0; i < commitCount; i++, startInstant++) { + String commitTime = makeNewCommitTime(startInstant); + HoodieTestTable.of(metaClient).addCommit(commitTime); + } + + for (int i = 0; i < cleanCount; i++, startInstant++) { + String commitTime = makeNewCommitTime(startInstant); + createCleanMetadata(commitTime + "", false, true); + } + + List cleanStats = runCleaner(config); + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + + assertEquals(0, cleanStats.size(), "Must not clean any files"); + assertEquals(1, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); + assertEquals(0, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); + assertEquals(--cleanCount, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--startInstant))); + + cleanStats = runCleaner(config); + timeline = metaClient.reloadActiveTimeline(); + + assertEquals(0, cleanStats.size(), "Must not clean any files"); + assertEquals(1, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); + assertEquals(0, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); + assertEquals(--cleanCount, timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--startInstant))); + } + + /** * Test HoodieTable.clean() Cleaning by versions logic for MOR table with Log files. */ @@ -1295,20 +1351,23 @@ private Map> generateBootstrapIndexAndSourceD @Test public void testCleanMarkerDataFilesOnRollback() throws Exception { HoodieTestTable testTable = HoodieTestTable.of(metaClient) - .addRequestedCommit("000") + .addRequestedCommit("001") .withMarkerFiles("default", 10, IOType.MERGE); final int numTempFilesBefore = testTable.listAllFilesInTempFolder().length; assertEquals(10, numTempFilesBefore, "Some marker files are created."); - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) + .withPath(basePath).build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); table.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"), Option.empty()); metaClient.reloadActiveTimeline(); - HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"); - table.scheduleRollback(context, "001", rollbackInstant, false); - table.rollback(context, "001", rollbackInstant, true, false); + HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"); + table.scheduleRollback(context, "002", rollbackInstant, false, config.shouldRollbackUsingMarkers()); + table.rollback(context, "002", rollbackInstant, true, false); final int numTempFilesAfter = testTable.listAllFilesInTempFolder().length; assertEquals(0, numTempFilesAfter, "All temp files are deleted."); } @@ -1419,7 +1478,7 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { * * @param insertFn Insert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use unique record-gen APIs) * @throws Exception in case of errors */ private void testInsertAndCleanFailedWritesByVersions( @@ -1435,7 +1494,7 @@ private void testInsertAndCleanFailedWritesByVersions( .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { final Function2, String, Integer> recordInsertGenWrappedFunction = generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); @@ -1468,10 +1527,10 @@ private void testInsertAndCleanFailedWritesByVersions( HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); assertTrue(timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().countInstants() == 3); - Option rolleBackInstantForFailedCommit = timeline.getTimelineOfActions( + Option rollBackInstantForFailedCommit = timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.getInstantDetails(rolleBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + timeline.getInstantDetails(rollBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); // Rollback of one of the failed writes should have deleted 3 files assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 595d4df2a93a..7674c3489072 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -34,13 +34,14 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.Transformations; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.action.deltacommit.AbstractSparkDeltaCommitActionExecutor; import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor; import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; @@ -63,6 +64,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -190,11 +192,13 @@ public void testUpsertPartitioner(boolean populateMetaFields) throws Exception { } } + // TODO: Enable metadata virtual keys in this test once the feature HUDI-2593 is completed @ParameterizedTest - @ValueSource(booleans = {true, false}) + @ValueSource(booleans = {true}) public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws Exception { // insert 100 records - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig config = cfgBuilder.build(); @@ -208,37 +212,40 @@ public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws // Update all the 100 records newCommitTime = "101"; - writeClient.startCommitWithTime(newCommitTime); - List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc().parallelize(updatedRecords, 1); HoodieReadClient readClient = new HoodieReadClient(context(), config); - updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect(); + JavaRDD updatedTaggedRecordsRDD = readClient.tagLocation(updatedRecordsRDD); + + writeClient.startCommitWithTime(newCommitTime); + writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); // Write them to corresponding avro logfiles metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(config, context(), metaClient); - HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) - .withLogAppends(updatedRecords); - // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state - ((SyncableFileSystemView) (table.getSliceView())).reset(); + + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( + writeClient.getEngineContext().getHadoopConf().get(), config, writeClient.getEngineContext()); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable + .of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); + + Set allPartitions = updatedRecords.stream() + .map(record -> record.getPartitionPath()) + .collect(Collectors.groupingBy(partitionPath -> partitionPath)) + .keySet(); + assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length); // Verify that all data file has one log file + HoodieTable table = HoodieSparkTable.create(config, context(), metaClient, true); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); for (FileSlice fileSlice : groupedLogFiles) { - assertEquals(1, fileSlice.getLogFiles().count(), "There should be 1 log file written for every data file"); + assertEquals(1, fileSlice.getLogFiles().count(), + "There should be 1 log file written for the latest data file - " + fileSlice); } } - // Mark 2nd delta-instant as completed - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.INFLIGHT, - HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime)); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); - // Do a compaction String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); JavaRDD result = (JavaRDD) writeClient.compact(compactionInstantTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestSparkClusteringPlanPartitionFilter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestSparkClusteringPlanPartitionFilter.java new file mode 100644 index 000000000000..a68a9e33601e --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestSparkClusteringPlanPartitionFilter.java @@ -0,0 +1,107 @@ +/* + * 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 org.apache.hudi.table.action.cluster.strategy; + +import org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; + +public class TestSparkClusteringPlanPartitionFilter { + @Mock + HoodieSparkCopyOnWriteTable table; + @Mock + HoodieSparkEngineContext context; + HoodieWriteConfig.Builder hoodieWriteConfigBuilder; + + @BeforeEach + public void setUp() { + this.hoodieWriteConfigBuilder = HoodieWriteConfig + .newBuilder() + .withPath("Fake_Table_Path"); + } + + @Test + public void testFilterPartitionNoFilter() { + HoodieWriteConfig config = hoodieWriteConfigBuilder.withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringPlanPartitionFilterMode(ClusteringPlanPartitionFilterMode.NONE) + .build()) + .build(); + + PartitionAwareClusteringPlanStrategy sg = new SparkSizeBasedClusteringPlanStrategy(table, context, config); + ArrayList fakeTimeBasedPartitionsPath = new ArrayList<>(); + fakeTimeBasedPartitionsPath.add("20210718"); + fakeTimeBasedPartitionsPath.add("20210716"); + fakeTimeBasedPartitionsPath.add("20210719"); + List list = sg.filterPartitionPaths(fakeTimeBasedPartitionsPath); + assertEquals(3, list.size()); + } + + @Test + public void testFilterPartitionRecentDays() { + HoodieWriteConfig config = hoodieWriteConfigBuilder.withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringSkipPartitionsFromLatest(1) + .withClusteringTargetPartitions(1) + .withClusteringPlanPartitionFilterMode(ClusteringPlanPartitionFilterMode.RECENT_DAYS) + .build()) + .build(); + + PartitionAwareClusteringPlanStrategy sg = new SparkSizeBasedClusteringPlanStrategy(table, context, config); + ArrayList fakeTimeBasedPartitionsPath = new ArrayList<>(); + fakeTimeBasedPartitionsPath.add("20210718"); + fakeTimeBasedPartitionsPath.add("20210716"); + fakeTimeBasedPartitionsPath.add("20210719"); + List list = sg.filterPartitionPaths(fakeTimeBasedPartitionsPath); + assertEquals(1, list.size()); + assertSame("20210718", list.get(0)); + } + + @Test + public void testFilterPartitionSelectedPartitions() { + HoodieWriteConfig config = hoodieWriteConfigBuilder.withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringPartitionFilterBeginPartition("20211222") + .withClusteringPartitionFilterEndPartition("20211223") + .withClusteringPlanPartitionFilterMode(ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS) + .build()) + .build(); + + PartitionAwareClusteringPlanStrategy sg = new SparkSizeBasedClusteringPlanStrategy(table, context, config); + ArrayList fakeTimeBasedPartitionsPath = new ArrayList<>(); + fakeTimeBasedPartitionsPath.add("20211220"); + fakeTimeBasedPartitionsPath.add("20211221"); + fakeTimeBasedPartitionsPath.add("20211222"); + fakeTimeBasedPartitionsPath.add("20211224"); + List list = sg.filterPartitionPaths(fakeTimeBasedPartitionsPath); + assertEquals(1, list.size()); + assertSame("20211222", list.get(0)); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 40df1af898ea..cba77b0c7e55 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -26,20 +26,26 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.testutils.Transformations; import org.apache.hudi.common.util.BaseFileUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLayoutConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.HoodieCreateHandle; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.MetadataMergeWriteStatus; @@ -57,6 +63,8 @@ import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.File; @@ -66,7 +74,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.UUID; +import java.util.stream.Stream; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; @@ -82,6 +92,13 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase { private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class); private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.avsc"); + private static final Stream indexType() { + HoodieIndex.IndexType[] data = new HoodieIndex.IndexType[] { + HoodieIndex.IndexType.BLOOM, + HoodieIndex.IndexType.BUCKET + }; + return Stream.of(data).map(Arguments::of); + } @Test public void testMakeNewPath() { @@ -112,14 +129,34 @@ private HoodieWriteConfig makeHoodieClientConfig() { private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() { // Prepare the AvroParquetIO - return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString()); + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()); + } + + private Properties makeIndexConfig(HoodieIndex.IndexType indexType) { + Properties props = new Properties(); + HoodieIndexConfig.Builder indexConfig = HoodieIndexConfig.newBuilder() + .withIndexType(indexType); + props.putAll(indexConfig.build().getProps()); + if (indexType.equals(HoodieIndex.IndexType.BUCKET)) { + props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + indexConfig.fromProperties(props).withIndexKeyField("_row_key").withBucketNum("1"); + props.putAll(indexConfig.build().getProps()); + props.putAll(HoodieLayoutConfig.newBuilder().fromProperties(props) + .withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name()) + .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build().getProps()); + } + return props; } // TODO (weiy): Add testcases for crossing file writing. - @Test - public void testUpdateRecords() throws Exception { + @ParameterizedTest + @MethodSource("indexType") + public void testUpdateRecords(HoodieIndex.IndexType indexType) throws Exception { // Prepare the AvroParquetIO - HoodieWriteConfig config = makeHoodieClientConfig(); + HoodieWriteConfig config = makeHoodieClientConfigBuilder() + .withProps(makeIndexConfig(indexType)).build(); String firstCommitTime = makeNewCommitTime(); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); writeClient.startCommitWithTime(firstCommitTime); @@ -165,7 +202,6 @@ public void testUpdateRecords() throws Exception { GenericRecord newRecord; int index = 0; for (GenericRecord record : fileRecords) { - //System.out.println("Got :" + record.get("_row_key").toString() + ", Exp :" + records.get(index).getRecordKey()); assertEquals(records.get(index).getRecordKey(), record.get("_row_key").toString()); index++; } @@ -405,8 +441,10 @@ public void testFileSizeUpsertRecords() throws Exception { public void testInsertUpsertWithHoodieAvroPayload() throws Exception { Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt"); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()) - .withStorageConfig(HoodieStorageConfig.newBuilder() - .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); String instantTime = "000"; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index 7b5cc27d3728..1e5f8029a714 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -56,6 +56,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -336,7 +337,6 @@ public void testUpsertPartitionerWithSmallFileHandlingWithInflightCompactionWith HoodieCompactionPlan plan = CompactionTestUtils.createCompactionPlan(metaClient, "001", "002", 1, true, false); FileCreateUtils.createRequestedCompactionCommit(basePath, "002", plan); // Simulate one more commit so that inflight compaction is considered when building file groups in file system view - // FileCreateUtils.createBaseFile(basePath, testPartitionPath, "003", "2", 1); FileCreateUtils.createCommit(basePath, "003"); @@ -434,6 +434,49 @@ public void testUpsertPartitionerWithSmallFileHandlingWithCanIndexLogFiles() thr "Insert should be assigned to fg1"); } + @Test + public void testUpsertPartitionerWithSmallFileHandlingPickingMultipleCandidates() throws Exception { + final String partitionPath = DEFAULT_PARTITION_PATHS[0]; + + HoodieWriteConfig config = + makeHoodieClientConfigBuilder() + .withMergeSmallFileGroupCandidatesLimit(3) + .withStorageConfig( + HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(2048) + .build() + ) + .build(); + + // Bootstrap base files ("small-file targets") + FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-1", 1024); + FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-2", 1024); + FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-3", 1024); + + FileCreateUtils.createCommit(basePath, "002"); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {partitionPath}); + // Default estimated record size will be 1024 based on last file group created. + // Only 1 record can be added to small file + WorkloadProfile profile = + new WorkloadProfile(buildProfile(jsc.parallelize(dataGenerator.generateInserts("003", 3)))); + + HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(this.metaClient); + + HoodieSparkTable table = HoodieSparkTable.create(config, context, reloadedMetaClient); + + SparkUpsertDeltaCommitPartitioner partitioner = new SparkUpsertDeltaCommitPartitioner<>(profile, context, table, config); + + assertEquals(3, partitioner.numPartitions()); + assertEquals( + Arrays.asList( + new BucketInfo(BucketType.UPDATE, "fg-1", partitionPath), + new BucketInfo(BucketType.UPDATE, "fg-2", partitionPath), + new BucketInfo(BucketType.UPDATE, "fg-3", partitionPath) + ), + partitioner.getBucketInfos()); + } + private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() { // Prepare the AvroParquetIO return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 6b837e3178ee..454c289dbd6d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -46,7 +46,6 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; @@ -57,9 +56,6 @@ import java.util.List; import java.util.stream.Collectors; -import static org.apache.hudi.common.testutils.FileCreateUtils.createDeltaCommit; -import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit; -import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -163,7 +159,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfigBuilder() .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) .build(); try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { String newCommitTime = "100"; @@ -176,19 +172,14 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Update all the 100 records HoodieTable table = HoodieSparkTable.create(config, context); newCommitTime = "101"; - writeClient.startCommitWithTime(newCommitTime); List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); HoodieIndex index = new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance()); - updatedRecords = tagLocation(index, updatedRecordsRDD, table).collect(); + JavaRDD updatedTaggedRecordsRDD = tagLocation(index, updatedRecordsRDD, table); - // Write them to corresponding avro logfiles. Also, set the state transition properly. - HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) - .withLogAppends(updatedRecords); - metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, - HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); - writeClient.commit(newCommitTime, jsc.emptyRDD(), Option.empty()); + writeClient.startCommitWithTime(newCommitTime); + writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); metaClient.reloadActiveTimeline(); // Verify that all data file has one log file @@ -200,9 +191,6 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { assertEquals(1, fileSlice.getLogFiles().count(), "There should be 1 log file written for every data file"); } } - createDeltaCommit(basePath, newCommitTime); - createRequestedDeltaCommit(basePath, newCommitTime); - createInflightDeltaCommit(basePath, newCommitTime); // Do a compaction table = HoodieSparkTable.create(config, context); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java index 823eac407dde..dee1fadd73d5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java @@ -99,19 +99,20 @@ public void testLogFileSizeCompactionSimple() { sizesMap.put(90 * MB, Collections.singletonList(1024 * MB)); LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy(); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(1205) + .withLogFileSizeThresholdBasedCompaction(100 * 1024 * 1024).build()) .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>()); assertTrue(returned.size() < operations.size(), "LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions"); - assertEquals(1, returned.size(), "LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction"); + assertEquals(2, returned.size(), "LogFileSizeBasedCompactionStrategy should have resulted in 2 compaction"); // Total size of all the log files Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) .map(Double::longValue).reduce(Long::sum).orElse(0L); - assertEquals(1204, (long) returnedSize, - "Should chose the first 2 compactions which should result in a total IO of 690 MB"); + assertEquals(1594, (long) returnedSize, + "Should chose the first 2 compactions which should result in a total IO of 1594 MB"); } @Test diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java index 3225dcd04ea3..1bee6ac0ac62 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java @@ -86,7 +86,8 @@ public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() // execute CopyOnWriteRollbackActionExecutor with filelisting mode BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, false); + new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, false, + table.getConfig().shouldRollbackUsingMarkers()); HoodieRollbackPlan rollbackPlan = (HoodieRollbackPlan) copyOnWriteRollbackPlanActionExecutor.execute().get(); CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, true, false); @@ -168,7 +169,8 @@ private void performRollbackAndValidate(boolean isUsingMarkers, HoodieWriteConfi } BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", commitInstant, false); + new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, "003", commitInstant, false, + table.getConfig().shouldRollbackUsingMarkers()); HoodieRollbackPlan hoodieRollbackPlan = (HoodieRollbackPlan) copyOnWriteRollbackPlanActionExecutor.execute().get(); CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(context, cfg, table, "003", commitInstant, false, false); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index 38be873e57ad..4e98b220f361 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -91,7 +91,8 @@ public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws //2. rollback HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = - new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false); + new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false, + cfg.shouldRollbackUsingMarkers()); mergeOnReadRollbackPlanActionExecutor.execute().get(); MergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new MergeOnReadRollbackActionExecutor( context, @@ -155,13 +156,14 @@ public void testFailForCompletedInstants() { } /** - * Test Cases for rollbacking when has not base file. + * Test Cases for rolling back when there is no base file. */ @Test public void testRollbackWhenFirstCommitFail() throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); - + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withRollbackUsingMarkers(false) + .withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime("001"); client.insert(jsc.emptyRDD(), "001"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java new file mode 100644 index 000000000000..1c6602348fba --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java @@ -0,0 +1,103 @@ +/* + * 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 org.apache.hudi.table.functional; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static org.apache.hudi.testutils.HoodieClientTestUtils.countRecordsOptionallySince; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@Tag("functional") +public class TestHoodieSparkCopyOnWriteTableArchiveWithReplace extends SparkClientFunctionalTestHarness { + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDeletePartitionAndArchive(boolean metadataEnabled) throws IOException { + HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.COPY_ON_WRITE); + HoodieWriteConfig writeConfig = getConfigBuilder(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(DEFAULT_PARTITION_PATHS)) { + + // 1st write batch; 3 commits for 3 partitions + String instantTime1 = HoodieActiveTimeline.createNewInstantTime(1000); + client.startCommitWithTime(instantTime1); + client.insert(jsc().parallelize(dataGen.generateInsertsForPartition(instantTime1, 10, DEFAULT_FIRST_PARTITION_PATH), 1), instantTime1); + String instantTime2 = HoodieActiveTimeline.createNewInstantTime(2000); + client.startCommitWithTime(instantTime2); + client.insert(jsc().parallelize(dataGen.generateInsertsForPartition(instantTime2, 10, DEFAULT_SECOND_PARTITION_PATH), 1), instantTime2); + String instantTime3 = HoodieActiveTimeline.createNewInstantTime(3000); + client.startCommitWithTime(instantTime3); + client.insert(jsc().parallelize(dataGen.generateInsertsForPartition(instantTime3, 1, DEFAULT_THIRD_PARTITION_PATH), 1), instantTime3); + + final HoodieTimeline timeline1 = metaClient.getCommitsTimeline().filterCompletedInstants(); + assertEquals(21, countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline1, Option.empty())); + + // delete the 1st and the 2nd partition; 1 replace commit + final String instantTime4 = HoodieActiveTimeline.createNewInstantTime(4000); + client.startCommitWithTime(instantTime4, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); + client.deletePartitions(Arrays.asList(DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH), instantTime4); + + // 2nd write batch; 3 commits for the 3rd partition; the 3rd commit to trigger archiving the replace commit + for (int i = 5; i < 8; i++) { + String instantTime = HoodieActiveTimeline.createNewInstantTime(i * 1000); + client.startCommitWithTime(instantTime); + client.insert(jsc().parallelize(dataGen.generateInsertsForPartition(instantTime, 1, DEFAULT_THIRD_PARTITION_PATH), 1), instantTime); + } + + // verify archived timeline + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTimeline archivedTimeline = metaClient.getArchivedTimeline(); + assertTrue(archivedTimeline.containsInstant(instantTime1)); + assertTrue(archivedTimeline.containsInstant(instantTime2)); + assertTrue(archivedTimeline.containsInstant(instantTime3)); + assertTrue(archivedTimeline.containsInstant(instantTime4), "should contain the replace commit."); + + // verify records + final HoodieTimeline timeline2 = metaClient.getCommitTimeline().filterCompletedInstants(); + assertEquals(4, countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline2, Option.empty()), + "should only have the 4 records from the 3rd partition."); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java index 8f7a500d13f1..a0ec0de37147 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -45,6 +45,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.List; @@ -52,6 +53,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Tag("functional") class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTestHarness { @@ -111,7 +113,8 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 400); - insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + Stream dataFiles = insertRecordsToMORTable(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); /* * Write 2 (more inserts to create new files) @@ -119,7 +122,8 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese // we already set small file size to small number to force inserts to go into new file. newCommitTime = "002"; client.startCommitWithTime(newCommitTime); - insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + dataFiles = insertRecordsToMORTable(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); if (doUpdates) { /* @@ -128,7 +132,7 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese newCommitTime = "003"; client.startCommitWithTime(newCommitTime); records = dataGen.generateUpdates(newCommitTime, 100); - updateRecords(metaClient, records, client, cfg, newCommitTime); + updateRecordsInMORTable(metaClient, records, client, cfg, newCommitTime); } HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); @@ -144,28 +148,101 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); // Do the clustering and validate - client.cluster(clusteringCommitTime, true); + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); + } + } - metaClient = HoodieTableMetaClient.reload(metaClient); - final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); - clusteredTable.getHoodieView().sync(); - Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) - .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); - // verify there should be only one base file per partition after clustering. - assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); - - HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); - assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), - "Expecting a single commit."); - assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); - assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); - if (cfg.populateMetaFields()) { - assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), - "Must contain 200 records"); - } else { - assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty())); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testClusteringWithNoBaseFiles(boolean doUpdates) throws Exception { + // set low compaction small File Size to generate more file groups. + HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder() + .forTable("test-trip-table") + .withPath(basePath()) + .withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withDeleteParallelism(2) + .withAutoCommit(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(10L) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(1024 * 1024 * 1024) + .parquetMaxFileSize(1024 * 1024 * 1024).build()) + .withEmbeddedTimelineServerEnabled(true) + .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withEnableBackupForRemoteFileSystemView(false).build()) + // set index type to INMEMORY so that log files can be indexed, and it is safe to send + // inserts straight to the log to produce file slices with only log files and no data files + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0) + .withInlineClustering(true) + .withInlineClusteringNumCommits(1).build()) + .withRollbackUsingMarkers(false); + HoodieWriteConfig cfg = cfgBuilder.build(); + HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, cfg.getProps()); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { + // test 2 inserts + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 400); + Stream dataFiles = insertRecordsToMORTable(metaClient, records.subList(0, 200), client, cfg, newCommitTime); + assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files"); + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + dataFiles = insertRecordsToMORTable(metaClient, records.subList(200, 400), client, cfg, newCommitTime); + assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files"); + // run updates + if (doUpdates) { + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 100); + updateRecordsInMORTable(metaClient, records, client, cfg, newCommitTime); } + + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + hoodieTable.getHoodieView().sync(); + FileStatus[] allBaseFiles = listAllBaseFilesInPath(hoodieTable); + // expect 0 base files for each partition + assertEquals(0, allBaseFiles.length); + + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + // verify log files are included in clustering plan for each partition. + assertEquals(dataGen.getPartitionPaths().length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); + + // do the clustering and validate + doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen); } } + private void doClusteringAndValidate(SparkRDDWriteClient client, + String clusteringCommitTime, + HoodieTableMetaClient metaClient, + HoodieWriteConfig cfg, + HoodieTestDataGenerator dataGen) { + client.cluster(clusteringCommitTime, true); + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); + clusteredTable.getHoodieView().sync(); + Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) + .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); + assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); + assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), + "Expecting a single commit."); + assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); + if (cfg.populateMetaFields()) { + assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), + "Must contain 200 records"); + } else { + assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty())); + } + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java index fd2f63a26c63..c80374b64f4a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -57,6 +58,7 @@ import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -94,7 +96,8 @@ public void testIncrementalReadsWithCompaction() throws Exception { client.startCommitWithTime(commitTime1); List records001 = dataGen.generateInserts(commitTime1, 200); - insertRecords(metaClient, records001, client, cfg, commitTime1); + Stream dataFiles = insertRecordsToMORTable(metaClient, records001, client, cfg, commitTime1); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); // verify only one base file shows up with commit time 001 FileStatus[] snapshotROFiles = getROSnapshotFiles(partitionPath); @@ -115,7 +118,7 @@ public void testIncrementalReadsWithCompaction() throws Exception { String updateTime = "004"; client.startCommitWithTime(updateTime); List records004 = dataGen.generateUpdates(updateTime, 100); - updateRecords(metaClient, records004, client, cfg, updateTime); + updateRecordsInMORTable(metaClient, records004, client, cfg, updateTime); // verify RO incremental reads - only one base file shows up because updates to into log files incrementalROFiles = getROIncrementalFiles(partitionPath, false); @@ -142,7 +145,8 @@ public void testIncrementalReadsWithCompaction() throws Exception { String insertsTime = "006"; List records006 = dataGen.generateInserts(insertsTime, 200); client.startCommitWithTime(insertsTime); - insertRecords(metaClient, records006, client, cfg, insertsTime); + dataFiles = insertRecordsToMORTable(metaClient, records006, client, cfg, insertsTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); // verify new write shows up in snapshot mode even though there is pending compaction snapshotROFiles = getROSnapshotFiles(partitionPath); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java index fb44c14f59ad..62ce00749668 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java @@ -95,7 +95,8 @@ public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean popul client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - insertRecords(metaClient, records, client, cfg, newCommitTime); + Stream dataFiles = insertRecordsToMORTable(metaClient, records, client, cfg, newCommitTime); + assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit"); /* * Write 2 (updates) @@ -103,7 +104,7 @@ public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean popul newCommitTime = "004"; client.startCommitWithTime(newCommitTime); records = dataGen.generateUpdates(newCommitTime, 100); - updateRecords(metaClient, records, client, cfg, newCommitTime); + updateRecordsInMORTable(metaClient, records, client, cfg, newCommitTime); String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index 6bbb0f655bb8..5eee26276484 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -24,13 +24,16 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; @@ -139,7 +142,8 @@ void testCOWToMORConvertedTableRollback(boolean rollbackUsingMarkers) throws Exc @ParameterizedTest @ValueSource(booleans = {true, false}) void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); addConfigsForPopulateMetaFields(cfgBuilder, true); HoodieWriteConfig cfg = cfgBuilder.build(); @@ -294,7 +298,10 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro @Test void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { boolean populateMetaFields = true; - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false) + // Timeline-server-based markers are not used for multi-rollback tests + .withMarkersType(MarkerType.DIRECT.name()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig cfg = cfgBuilder.build(); @@ -344,7 +351,9 @@ void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { newCommitTime = "002"; // WriteClient with custom config (disable small file handling) HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); + // Timeline-server-based markers are not used for multi-rollback tests + .withMarkersType(MarkerType.DIRECT.name()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) { nClient.startCommitWithTime(newCommitTime); @@ -465,6 +474,8 @@ private HoodieWriteConfig.Builder getHoodieWriteConfigWithSmallFileHandlingOffBu .withAutoCommit(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .withEmbeddedTimelineServerEnabled(true) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table"); @@ -559,7 +570,7 @@ void testInsertsGeneratedIntoLogFilesRollback(boolean rollbackUsingMarkers) thro } } if (rollbackUsingMarkers) { - metaClient.getFs().copyFromLocalFile(markerDir, + metaClient.getFs().copyFromLocalFile(new Path(markerDir, lastCommitTime), new Path(metaClient.getMarkerFolderPath(lastCommitTime))); } Thread.sleep(1000); @@ -628,6 +639,122 @@ void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(boolean rollbackUsi } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLazyRollbackOfFailedCommit(boolean rollbackUsingMarkers) throws Exception { + Properties properties = new Properties(); + properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString()); + HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties); + + HoodieWriteConfig cfg = getWriteConfig(true, rollbackUsingMarkers); + HoodieWriteConfig autoCommitFalseCfg = getWriteConfig(false, rollbackUsingMarkers); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + // commit 1 + List records = insertRecords(client, dataGen, "001"); + // commit 2 to create log files + List updates1 = updateRecords(client, dataGen, "002", records, metaClient, cfg, true); + + // trigger a inflight commit 3 which will be later be rolled back explicitly. + SparkRDDWriteClient autoCommitFalseClient = getHoodieWriteClient(autoCommitFalseCfg); + List updates2 = updateRecords(autoCommitFalseClient, dataGen, "003", records, metaClient, autoCommitFalseCfg, false); + + // commit 4 successful (mimic multi-writer scenario) + List updates3 = updateRecords(client, dataGen, "004", records, metaClient, cfg, false); + + // trigger compaction + long numLogFiles = getNumLogFilesInLatestFileSlice(metaClient, cfg, dataGen); + doCompaction(autoCommitFalseClient, metaClient, cfg, numLogFiles); + long numLogFilesAfterCompaction = getNumLogFilesInLatestFileSlice(metaClient, cfg, dataGen); + assertNotEquals(numLogFiles, numLogFilesAfterCompaction); + + // rollback 3rd commit. + client.rollback("003"); + long numLogFilesAfterRollback = getNumLogFilesInLatestFileSlice(metaClient, cfg, dataGen); + // lazy rollback should have added the rollback block to previous file slice and not the latest. And so the latest slice's log file count should + // remain the same. + assertEquals(numLogFilesAfterRollback, numLogFilesAfterCompaction); + } + + private List insertRecords(SparkRDDWriteClient client, HoodieTestDataGenerator dataGen, String commitTime) { + /* + * Write 1 (only inserts, written as base file) + */ + client.startCommitWithTime(commitTime); + + List records = dataGen.generateInserts(commitTime, 20); + JavaRDD writeRecords = jsc().parallelize(records, 1); + + List statuses = client.upsert(writeRecords, commitTime).collect(); + assertNoWriteErrors(statuses); + return records; + } + + private List updateRecords(SparkRDDWriteClient client, HoodieTestDataGenerator dataGen, String commitTime, + List records, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, + boolean assertLogFiles) throws IOException { + client.startCommitWithTime(commitTime); + + records = dataGen.generateUpdates(commitTime, records); + JavaRDD writeRecords = jsc().parallelize(records, 1); + List statuses = client.upsert(writeRecords, commitTime).collect(); + assertNoWriteErrors(statuses); + + if (assertLogFiles) { + HoodieTable table = HoodieSparkTable.create(cfg, context(), metaClient); + table.getHoodieView().sync(); + TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView(); + + long numLogFiles = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + List allSlices = tableRTFileSystemView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(1, allSlices.stream().filter(fileSlice -> fileSlice.getBaseFile().isPresent()).count()); + assertTrue(allSlices.stream().anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); + numLogFiles += allSlices.stream().filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); + } + assertTrue(numLogFiles > 0); + } + return records; + } + + private long doCompaction(SparkRDDWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, long numLogFiles) throws IOException { + // Do a compaction + String instantTime = client.scheduleCompaction(Option.empty()).get().toString(); + JavaRDD writeStatuses = (JavaRDD) client.compact(instantTime); + + metaClient.reloadActiveTimeline(); + HoodieTable table = HoodieSparkTable.create(cfg, context(), metaClient); + String extension = table.getBaseFileExtension(); + assertEquals(numLogFiles, writeStatuses.map(status -> status.getStat().getPath().contains(extension)).count()); + assertEquals(numLogFiles, writeStatuses.count()); + client.commitCompaction(instantTime, writeStatuses, Option.empty()); + return numLogFiles; + } + + private long getNumLogFilesInLatestFileSlice(HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, HoodieTestDataGenerator dataGen) { + metaClient.reloadActiveTimeline(); + HoodieTable table = HoodieSparkTable.create(cfg, context(), metaClient); + table.getHoodieView().sync(); + TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView(); + + long numLogFiles = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + List allSlices = tableRTFileSystemView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + numLogFiles += allSlices.stream().filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); + } + return numLogFiles; + } + + private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean rollbackUsingMarkers) { + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(autoCommit).withRollbackUsingMarkers(rollbackUsingMarkers) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024L) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(3) + .withAutoClean(false) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()); + return cfgBuilder.build(); + } + private SyncableFileSystemView getFileSystemViewWithUnCommittedSlices(HoodieTableMetaClient metaClient) { try { return new HoodieTableFileSystemView(metaClient, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestDirectWriteMarkers.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestDirectWriteMarkers.java index 0817d6317b15..fa6df3ba73df 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestDirectWriteMarkers.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestDirectWriteMarkers.java @@ -59,15 +59,18 @@ public void cleanup() { } @Override - void verifyMarkersInFileSystem() throws IOException { + void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException { List markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath) .stream().filter(status -> status.getPath().getName().contains(".marker")) .sorted().collect(Collectors.toList()); assertEquals(3, markerFiles.size()); assertIterableEquals(CollectionUtils.createImmutableList( - "file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE", - "file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND", - "file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"), + "file:" + markerFolderPath.toString() + + (isTablePartitioned ? "/2020/06/01" : "") + "/file1.marker.MERGE", + "file:" + markerFolderPath.toString() + + (isTablePartitioned ? "/2020/06/02" : "") + "/file2.marker.APPEND", + "file:" + markerFolderPath.toString() + + (isTablePartitioned ? "/2020/06/03" : "") + "/file3.marker.CREATE"), markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList()) ); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestTimelineServerBasedWriteMarkers.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestTimelineServerBasedWriteMarkers.java index 583883ccb493..61ee844b1917 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestTimelineServerBasedWriteMarkers.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestTimelineServerBasedWriteMarkers.java @@ -93,18 +93,20 @@ public void cleanup() { } @Override - void verifyMarkersInFileSystem() throws IOException { + void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException { // Verifies the markers List allMarkers = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem( - markerFolderPath.toString(), fs, context, 1) + markerFolderPath.toString(), fs, context, 1) .values().stream().flatMap(Collection::stream).sorted() .collect(Collectors.toList()); assertEquals(3, allMarkers.size()); - assertIterableEquals(CollectionUtils.createImmutableList( - "2020/06/01/file1.marker.MERGE", - "2020/06/02/file2.marker.APPEND", - "2020/06/03/file3.marker.CREATE"), - allMarkers); + List expectedMarkers = isTablePartitioned + ? CollectionUtils.createImmutableList( + "2020/06/01/file1.marker.MERGE", "2020/06/02/file2.marker.APPEND", + "2020/06/03/file3.marker.CREATE") + : CollectionUtils.createImmutableList( + "file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE"); + assertIterableEquals(expectedMarkers, allMarkers); // Verifies the marker type file Path markerTypeFilePath = new Path(markerFolderPath, MarkerUtils.MARKER_TYPE_FILENAME); assertTrue(MarkerUtils.doesMarkerTypeFileExist(fs, markerFolderPath.toString())); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersBase.java index 0298ed37a638..5f96041b372d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersBase.java @@ -31,8 +31,11 @@ import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaSparkContext; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; +import java.util.List; import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -48,10 +51,10 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness { protected JavaSparkContext jsc; protected HoodieSparkEngineContext context; - private void createSomeMarkers() { - writeMarkers.create("2020/06/01", "file1", IOType.MERGE); - writeMarkers.create("2020/06/02", "file2", IOType.APPEND); - writeMarkers.create("2020/06/03", "file3", IOType.CREATE); + private void createSomeMarkers(boolean isTablePartitioned) { + writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE); + writeMarkers.create(isTablePartitioned ? "2020/06/02" : "", "file2", IOType.APPEND); + writeMarkers.create(isTablePartitioned ? "2020/06/03" : "", "file3", IOType.CREATE); } private void createInvalidFile(String partitionPath, String invalidFileName) { @@ -64,22 +67,24 @@ private void createInvalidFile(String partitionPath, String invalidFileName) { } } - abstract void verifyMarkersInFileSystem() throws IOException; + abstract void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException; - @Test - public void testCreation() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testCreation(boolean isTablePartitioned) throws Exception { // when - createSomeMarkers(); + createSomeMarkers(isTablePartitioned); // then assertTrue(fs.exists(markerFolderPath)); - verifyMarkersInFileSystem(); + verifyMarkersInFileSystem(isTablePartitioned); } - @Test - public void testDeletionWhenMarkerDirExists() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDeletionWhenMarkerDirExists(boolean isTablePartitioned) throws IOException { //when - writeMarkers.create("2020/06/01", "file1", IOType.MERGE); + writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE); // then assertTrue(writeMarkers.doesMarkerDirExist()); @@ -95,32 +100,40 @@ public void testDeletionWhenMarkerDirNotExists() throws IOException { assertFalse(writeMarkers.deleteMarkerDir(context, 2)); } - @Test - public void testDataPathsWhenCreatingOrMerging() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDataPathsWhenCreatingOrMerging(boolean isTablePartitioned) throws IOException { // add markfiles - createSomeMarkers(); + createSomeMarkers(isTablePartitioned); // add invalid file - createInvalidFile("2020/06/01", "invalid_file3"); + createInvalidFile(isTablePartitioned ? "2020/06/01" : "", "invalid_file3"); long fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).stream() .filter(fileStatus -> !fileStatus.getPath().getName().contains(MarkerUtils.MARKER_TYPE_FILENAME)) .count(); assertEquals(fileSize, 4); + List expectedPaths = isTablePartitioned + ? CollectionUtils.createImmutableList("2020/06/01/file1", "2020/06/03/file3") + : CollectionUtils.createImmutableList("file1", "file3"); // then - assertIterableEquals(CollectionUtils.createImmutableList( - "2020/06/01/file1", "2020/06/03/file3"), + assertIterableEquals(expectedPaths, writeMarkers.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList()) ); } - @Test - public void testAllMarkerPaths() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testAllMarkerPaths(boolean isTablePartitioned) throws IOException { // given - createSomeMarkers(); + createSomeMarkers(isTablePartitioned); + List expectedPaths = isTablePartitioned + ? CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE", + "2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE") + : CollectionUtils.createImmutableList( + "file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE"); // then - assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE", - "2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"), + assertIterableEquals(expectedPaths, writeMarkers.allMarkerFilePaths().stream() .filter(path -> !path.contains(MarkerUtils.MARKER_TYPE_FILENAME)) .sorted().collect(Collectors.toList()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index 19ec4e6d0654..5f5dfdec5dce 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -326,9 +326,11 @@ public void testDowngrade(boolean deletePartialMarkerFiles, HoodieTableType tabl new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance()) .run(toVersion, null); - // assert marker files - assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE); - + if (fromVersion == HoodieTableVersion.TWO) { + // assert marker files + assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE); + } + // verify hoodie.table.version got downgraded metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 0a010dde5b63..ee3c309b30f2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -152,6 +152,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType in .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server + .withRemoteServerPort(timelineServicePort) .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } @@ -244,6 +245,18 @@ private Function2, String, Integer> wrapRecordsGenFunctionFor }; } + private Function3, String, Integer, String> wrapRecordsGenFunctionForPreppedCalls( + final HoodieWriteConfig writeConfig, final Function3, String, Integer, String> recordGenFunction) { + return (commit, numRecords, partition) -> { + final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig); + List records = recordGenFunction.apply(commit, numRecords, partition); + final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient); + JavaRDD taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table); + return taggedRecords.collect(); + }; + } + /** * Helper to generate delete keys generation function for testing Prepped version of API. Prepped APIs expect the keys * to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is @@ -285,6 +298,15 @@ public Function2, String, Integer> generateWrapRecordsFn(bool } } + public Function3, String, Integer, String> generateWrapRecordsForPartitionFn(boolean isPreppedAPI, + HoodieWriteConfig writeConfig, Function3, String, Integer, String> wrapped) { + if (isPreppedAPI) { + return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped); + } else { + return wrapped; + } + } + /** * Generate wrapper for delete key generation function for testing Prepped APIs. * @@ -355,12 +377,22 @@ public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, Spar public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { - final Function2, String, Integer> recordGenFunction = - generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, Option partition) throws Exception { - return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + if (partition.isPresent()) { + final Function3, String, Integer, String> recordGenFunction = + generateWrapRecordsForPartitionFn(isPreppedAPI, writeConfig, dataGen::generateInsertsForPartition); + + return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, + partition.get()); + } else { + final Function2, String, Integer> recordGenFunction = + generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); + + return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + } } public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, @@ -453,6 +485,16 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); } + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function3, String, Integer, String> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + boolean doCommit, String partition) throws Exception { + return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition); + } + /** * Helper to insert/upsert batch of records and do regular assertions on the state after successful completion. * @@ -478,10 +520,35 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws Exception { + List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); + return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, + numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, + expTotalCommits, doCommit, filterForCommitTimeWithAssert); + } + + public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function3, String, Integer, String> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, + boolean filterForCommitTimeWithAssert, + String partition) throws Exception { + + List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit, partition); + return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, + numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, + expTotalCommits, doCommit, filterForCommitTimeWithAssert); + } + + private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, + int numRecordsInThisCommit, List records, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws IOException { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); - List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); JavaRDD writeRecords = jsc.parallelize(records, 1); JavaRDD result = writeFn.apply(client, writeRecords, newCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index 9ed98b15cb7a..906f13d7a63b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -17,13 +17,18 @@ package org.apache.hudi.testutils; +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieRecord; @@ -31,7 +36,10 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; @@ -56,6 +64,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.timeline.service.TimelineService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -69,6 +78,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -83,12 +93,14 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.stream.Collectors; import scala.Tuple2; +import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -102,9 +114,12 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class); + protected static int timelineServicePort = + FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue(); private String testMethodName; protected transient JavaSparkContext jsc = null; protected transient HoodieSparkEngineContext context = null; + protected transient SparkSession sparkSession = null; protected transient Configuration hadoopConf = null; protected transient SQLContext sqlContext; protected transient FileSystem fs; @@ -113,6 +128,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im protected transient SparkRDDWriteClient writeClient; protected transient HoodieReadClient readClient; protected transient HoodieTableFileSystemView tableView; + protected transient TimelineService timelineService; protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier(); @@ -145,12 +161,14 @@ public void initResources() throws IOException { initTestDataGenerator(); initFileSystem(); initMetaClient(); + initTimelineService(); } /** * Cleanups resource group for the subclasses of {@link HoodieClientTestBase}. */ public void cleanupResources() throws IOException { + cleanupTimelineService(); cleanupClients(); cleanupSparkContexts(); cleanupTestDataGenerator(); @@ -175,6 +193,7 @@ protected void initSparkContexts(String appName) { sqlContext = new SQLContext(jsc); context = new HoodieSparkEngineContext(jsc); hadoopConf = context.getHadoopConf().get(); + sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate(); } /** @@ -245,6 +264,7 @@ protected void cleanupFileSystem() throws IOException { * * @throws IOException */ + @Override protected void initMetaClient() throws IOException { initMetaClient(getTableType()); } @@ -272,6 +292,28 @@ protected void initMetaClient(HoodieTableType tableType, Properties properties) metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties); } + /** + * Initializes timeline service based on the write config. + */ + protected void initTimelineService() { + timelineService = HoodieClientTestUtils.initTimelineService( + context, basePath, incrementTimelineServicePortToUse()); + timelineServicePort = timelineService.getServerPort(); + } + + protected void cleanupTimelineService() { + if (timelineService != null) { + timelineService.close(); + } + } + + protected int incrementTimelineServicePortToUse() { + // Increment the timeline service port for each individual test + // to avoid port reuse causing failures + timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024; + return timelineServicePort; + } + protected Properties getPropertiesForKeyGen() { Properties properties = new Properties(); properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); @@ -283,13 +325,26 @@ protected Properties getPropertiesForKeyGen() { return properties; } - protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) { + protected Properties getPropertiesForMetadataTable() { + Properties properties = new Properties(); + properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); + properties.put("hoodie.datasource.write.recordkey.field", "key"); + properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "key"); + return properties; + } + + protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields, + boolean isMetadataTable) { if (!populateMetaFields) { - configBuilder.withProperties(getPropertiesForKeyGen()) + configBuilder.withProperties((isMetadataTable ? getPropertiesForMetadataTable() : getPropertiesForKeyGen())) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.SIMPLE).build()); } } + protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) { + addConfigsForPopulateMetaFields(configBuilder, populateMetaFields, false); + } + /** * Cleanups hoodie clients. */ @@ -489,6 +544,9 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom List fsPartitionPaths = testTable.getAllPartitionPaths(); List fsPartitions = new ArrayList<>(); fsPartitionPaths.forEach(entry -> fsPartitions.add(entry.getFileName().toString())); + if (fsPartitions.isEmpty()) { + fsPartitions.add(""); + } List metadataPartitions = tableMetadata.getAllPartitionPaths(); Collections.sort(fsPartitions); @@ -575,7 +633,7 @@ protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableM } } } - assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length); + assertEquals(fsStatuses.length, partitionToFilesMap.get(partitionPath.toString()).length); // Block sizes should be valid Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getBlockSize() > 0)); @@ -625,7 +683,7 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); // Metadata table should automatically compact and clean - // versions are +1 as autoclean / compaction happens end of commits + // versions are +1 as autoClean / compaction happens end of commits int numFileVersions = metadataWriteConfig.getCleanerFileVersionsRetained() + 1; HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline()); metadataTablePartitions.forEach(partition -> { @@ -636,4 +694,27 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab + numFileVersions + " but was " + latestSlices.size()); }); } + + public HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly) throws IOException { + return createCleanMetadata(instantTime, inflightOnly, false); + } + + public HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly, boolean isEmpty) throws IOException { + HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), + CleanPlanV2MigrationHandler.VERSION, new HashMap<>()); + if (inflightOnly) { + HoodieTestTable.of(metaClient).addInflightClean(instantTime, cleanerPlan); + } else { + HoodieCleanStat cleanStats = new HoodieCleanStat( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, + HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEFAULT_PARTITION_PATHS.length)], + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + instantTime); + HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats)); + HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata, isEmpty); + } + return new HoodieInstant(inflightOnly, "clean", instantTime); + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 59172c55a806..6dffd535b914 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.HoodieReadClient; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -30,13 +31,18 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.timeline.service.TimelineService; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -257,6 +263,36 @@ public static Stream readHFile(JavaSparkContext jsc, String[] pat return valuesAsList.stream(); } + /** + * Initializes timeline service based on the write config. + * + * @param context {@link HoodieEngineContext} instance to use. + * @param basePath Base path of the table. + * @param timelineServicePort Port number to use for timeline service. + * @return started {@link TimelineService} instance. + */ + public static TimelineService initTimelineService( + HoodieEngineContext context, String basePath, int timelineServicePort) { + try { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) + .build(); + TimelineService timelineService = new TimelineService(context, new Configuration(), + TimelineService.Config.builder().enableMarkerRequests(true) + .serverPort(config.getViewStorageConfig().getRemoteViewServerPort()).build(), + FileSystem.get(new Configuration()), + FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), + config.getViewStorageConfig(), config.getCommonConfig())); + timelineService.startService(); + LOG.info("Timeline service server port: " + timelineServicePort); + return timelineService; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + public static Option getCommitMetadataForLatestInstant(HoodieTableMetaClient metaClient) { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); if (timeline.lastInstant().isPresent()) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java index 8e37c92d34a1..ca7bb4e0117c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; @@ -39,12 +40,20 @@ public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable { private static final Logger LOG = LogManager.getLogger(HoodieSparkWriteableTestTable.class); - private HoodieSparkWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { - super(basePath, fs, metaClient, schema, filter); + private HoodieSparkWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, + BloomFilter filter, HoodieTableMetadataWriter metadataWriter) { + super(basePath, fs, metaClient, schema, filter, metadataWriter); } public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { - return new HoodieSparkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter); + return new HoodieSparkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), + metaClient, schema, filter, null); + } + + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, + HoodieTableMetadataWriter metadataWriter) { + return new HoodieSparkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), + metaClient, schema, filter, metadataWriter); } public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { @@ -53,6 +62,13 @@ public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, return of(metaClient, schema, filter); } + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetadataWriter metadataWriter) { + BloomFilter filter = BloomFilterFactory + .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); + return of(metaClient, schema, filter, metadataWriter); + } + public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); return of(metaClient, schema); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 79fbdcaad93a..1ecdd336464d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -49,6 +49,7 @@ import org.apache.hudi.testutils.providers.HoodieMetaClientProvider; import org.apache.hudi.testutils.providers.HoodieWriteClientProvider; import org.apache.hudi.testutils.providers.SparkProvider; +import org.apache.hudi.timeline.service.TimelineService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -81,10 +82,13 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMetaClientProvider, HoodieWriteClientProvider { + protected static int timelineServicePort = + FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue(); private static transient SparkSession spark; private static transient SQLContext sqlContext; private static transient JavaSparkContext jsc; private static transient HoodieSparkEngineContext context; + private static transient TimelineService timelineService; /** * An indicator of the initialization status. @@ -174,6 +178,9 @@ public synchronized void runBeforeEach() { sqlContext = spark.sqlContext(); jsc = new JavaSparkContext(spark.sparkContext()); context = new HoodieSparkEngineContext(jsc); + timelineService = HoodieClientTestUtils.initTimelineService( + context, basePath(), incrementTimelineServicePortToUse()); + timelineServicePort = timelineService.getServerPort(); } } @@ -189,6 +196,9 @@ public static synchronized void resetSpark() { spark.close(); spark = null; } + if (timelineService != null) { + timelineService.close(); + } } protected JavaRDD tagLocation( @@ -203,7 +213,8 @@ protected JavaRDD updateLocation( index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table)); } - protected void insertRecords(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { + protected Stream insertRecordsToMORTable(HoodieTableMetaClient metaClient, List records, + SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient); JavaRDD writeRecords = jsc().parallelize(records, 1); @@ -228,11 +239,10 @@ protected void insertRecords(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { + protected void updateRecordsInMORTable(HoodieTableMetaClient metaClient, List records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException { HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient); Map recordsMap = new HashMap<>(); @@ -312,9 +322,17 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withRemoteServerPort(timelineServicePort) .withEnableBackupForRemoteFileSystemView(false).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) .withClusteringConfig(clusteringConfig) .withRollbackUsingMarkers(rollbackUsingMarkers); } + + protected int incrementTimelineServicePortToUse() { + // Increment the timeline service port for each individual test + // to avoid port reuse causing failures + timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024; + return timelineServicePort; + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java index 2e3b546de403..fd8ece1e06b8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.testutils; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; @@ -176,12 +177,14 @@ public static InternalRow getInternalRowWithError(String partitionPath) { return new GenericInternalRow(values); } - public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) { + public static HoodieWriteConfig.Builder getConfigBuilder(String basePath, int timelineServicePort) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2) .withDeleteParallelism(2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withBulkInsertParallelism(2); diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml index f81dec4d7243..33c98144ccab 100644 --- a/hudi-client/pom.xml +++ b/hudi-client/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index bbf2b2972910..c20ff22f8c9e 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index a602b452c324..fdaa466c3cc9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -18,8 +18,10 @@ package org.apache.hudi.avro; +import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -57,7 +59,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.sql.Timestamp; import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; @@ -229,10 +230,10 @@ public static Schema removeMetadataFields(Schema schema) { public static Schema removeFields(Schema schema, List fieldsToRemove) { List filteredFields = schema.getFields() - .stream() - .filter(field -> !fieldsToRemove.contains(field.name())) - .map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal())) - .collect(Collectors.toList()); + .stream() + .filter(field -> !fieldsToRemove.contains(field.name())) + .map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal())) + .collect(Collectors.toList()); Schema filteredSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false); filteredSchema.setFields(filteredFields); return filteredSchema; @@ -289,7 +290,7 @@ public static Schema getSchemaForFields(Schema fileSchema, List fields) } public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath, - String fileName) { + String fileName) { record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName); record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath); record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey); @@ -540,8 +541,6 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object fieldValue) { if (fieldSchema.getLogicalType() == LogicalTypes.date()) { return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString())); - } else if (fieldSchema.getLogicalType() == LogicalTypes.timestampMicros()) { - return new Timestamp(Long.parseLong(fieldValue.toString()) / 1000); } else if (fieldSchema.getLogicalType() instanceof LogicalTypes.Decimal) { Decimal dc = (Decimal) fieldSchema.getLogicalType(); DecimalConversion decimalConversion = new DecimalConversion(); @@ -551,7 +550,7 @@ private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object } else if (fieldSchema.getType() == Schema.Type.BYTES) { ByteBuffer byteBuffer = (ByteBuffer) fieldValue; BigDecimal convertedValue = decimalConversion.fromBytes(byteBuffer, fieldSchema, - LogicalTypes.decimal(dc.getPrecision(), dc.getScale())); + LogicalTypes.decimal(dc.getPrecision(), dc.getScale())); byteBuffer.rewind(); return convertedValue; } @@ -570,9 +569,51 @@ public static Schema getNullSchema() { * @return sanitized name */ public static String sanitizeName(String name) { - if (name.substring(0,1).matches(INVALID_AVRO_FIRST_CHAR_IN_NAMES)) { + if (name.substring(0, 1).matches(INVALID_AVRO_FIRST_CHAR_IN_NAMES)) { name = name.replaceFirst(INVALID_AVRO_FIRST_CHAR_IN_NAMES, MASK_FOR_INVALID_CHARS_IN_NAMES); } return name.replaceAll(INVALID_AVRO_CHARS_IN_NAMES, MASK_FOR_INVALID_CHARS_IN_NAMES); } + + /** + * Gets record column values into one object. + * + * @param record Hoodie record. + * @param columns Names of the columns to get values. + * @param schema {@link Schema} instance. + * @return Column value if a single column, or concatenated String values by comma. + */ + public static Object getRecordColumnValues(HoodieRecord record, + String[] columns, + Schema schema) { + try { + GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + if (columns.length == 1) { + return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true); + } else { + StringBuilder sb = new StringBuilder(); + for (String col : columns) { + sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true)); + } + + return sb.toString(); + } + } catch (IOException e) { + throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e); + } + } + + /** + * Gets record column values into one object. + * + * @param record Hoodie record. + * @param columns Names of the columns to get values. + * @param schema {@link SerializableSchema} instance. + * @return Column value if a single column, or concatenated String values by comma. + */ + public static Object getRecordColumnValues(HoodieRecord record, + String[] columns, + SerializableSchema schema) { + return getRecordColumnValues(record, columns, schema.get()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/HoodiePendingRollbackInfo.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodiePendingRollbackInfo.java new file mode 100644 index 000000000000..c53babf35010 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodiePendingRollbackInfo.java @@ -0,0 +1,44 @@ +/* + * 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 org.apache.hudi.common; + +import org.apache.hudi.avro.model.HoodieRollbackPlan; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +/** + * Holds rollback instant and rollback plan for a pending rollback. + */ +public class HoodiePendingRollbackInfo { + + private final HoodieInstant rollbackInstant; + private final HoodieRollbackPlan rollbackPlan; + + public HoodiePendingRollbackInfo(HoodieInstant rollbackInstant, HoodieRollbackPlan rollbackPlan) { + this.rollbackInstant = rollbackInstant; + this.rollbackPlan = rollbackPlan; + } + + public HoodieInstant getRollbackInstant() { + return rollbackInstant; + } + + public HoodieRollbackPlan getRollbackPlan() { + return rollbackPlan; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java index d4a77b082284..3700d01a60ea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -291,13 +291,13 @@ private HFile.Reader fileIdIndexReader() { @Override public List getIndexedPartitionPaths() { - HFileScanner scanner = partitionIndexReader().getScanner(true, true); + HFileScanner scanner = partitionIndexReader().getScanner(true, false); return getAllKeys(scanner, HFileBootstrapIndex::getPartitionFromKey); } @Override public List getIndexedFileGroupIds() { - HFileScanner scanner = fileIdIndexReader().getScanner(true, true); + HFileScanner scanner = fileIdIndexReader().getScanner(true, false); return getAllKeys(scanner, HFileBootstrapIndex::getFileGroupFromKey); } @@ -319,7 +319,7 @@ private List getAllKeys(HFileScanner scanner, Function convert @Override public List getSourceFileMappingForPartition(String partition) { try { - HFileScanner scanner = partitionIndexReader().getScanner(true, true); + HFileScanner scanner = partitionIndexReader().getScanner(true, false); KeyValue keyValue = new KeyValue(Bytes.toBytes(getPartitionKey(partition)), new byte[0], new byte[0], HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]); if (scanner.seekTo(keyValue) == 0) { @@ -352,7 +352,7 @@ public Map getSourceFileMappingForFileI List fileGroupIds = new ArrayList<>(ids); Collections.sort(fileGroupIds); try { - HFileScanner scanner = fileIdIndexReader().getScanner(true, true); + HFileScanner scanner = fileIdIndexReader().getScanner(true, false); for (HoodieFileGroupId fileGroupId : fileGroupIds) { KeyValue keyValue = new KeyValue(Bytes.toBytes(getFileGroupKey(fileGroupId)), new byte[0], new byte[0], HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java index 08e1bb4a87a5..fef00389d8c5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java @@ -31,7 +31,8 @@ public enum Names { WRITE_CLIENT("Write Client Configs"), METRICS("Metrics Configs"), RECORD_PAYLOAD("Record Payload Config"), - KAFKA_CONNECT("Kafka Connect Configs"); + KAFKA_CONNECT("Kafka Connect Configs"), + AWS("Amazon Web Services Configs"); public final String name; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java index 7870d8551e09..961291430358 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java @@ -23,6 +23,8 @@ import org.apache.hudi.exception.HoodieException; import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import java.util.function.Function; import java.util.Objects; @@ -93,8 +95,8 @@ Option>> getInferFunc() { return inferFunction; } - public String[] getAlternatives() { - return alternatives; + public List getAlternatives() { + return Arrays.asList(alternatives); } public ConfigProperty withDocumentation(String doc) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java index f97e4add462d..afce8f6ec454 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/DFSPropertiesConfiguration.java @@ -18,14 +18,26 @@ package org.apache.hudi.common.config; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.io.InputStreamReader; +import java.net.URI; import java.util.HashSet; import java.util.Set; @@ -43,72 +55,111 @@ public class DFSPropertiesConfiguration { private static final Logger LOG = LogManager.getLogger(DFSPropertiesConfiguration.class); - private final FileSystem fs; + public static final String DEFAULT_PROPERTIES_FILE = "hudi-defaults.conf"; + public static final String CONF_FILE_DIR_ENV_NAME = "HUDI_CONF_DIR"; + public static final String DEFAULT_CONF_FILE_DIR = "file:/etc/hudi/conf"; + public static final Path DEFAULT_PATH = new Path(DEFAULT_CONF_FILE_DIR + "/" + DEFAULT_PROPERTIES_FILE); + + // props read from hudi-defaults.conf + private static TypedProperties GLOBAL_PROPS = loadGlobalProps(); - private final Path rootFile; + @Nullable + private final Configuration hadoopConfig; - private final TypedProperties props; + private Path currentFilePath; + + // props read from user defined configuration file or input stream + private final HoodieConfig hoodieConfig; // Keep track of files visited, to detect loops - private final Set visitedFiles; - - public DFSPropertiesConfiguration(FileSystem fs, Path rootFile, TypedProperties defaults) { - this.fs = fs; - this.rootFile = rootFile; - this.props = defaults; - this.visitedFiles = new HashSet<>(); - visitFile(rootFile); + private final Set visitedFilePaths; + + public DFSPropertiesConfiguration(@Nonnull Configuration hadoopConf, @Nonnull Path filePath) { + this.hadoopConfig = hadoopConf; + this.currentFilePath = filePath; + this.hoodieConfig = new HoodieConfig(); + this.visitedFilePaths = new HashSet<>(); + addPropsFromFile(filePath); } - public DFSPropertiesConfiguration(FileSystem fs, Path rootFile) { - this(fs, rootFile, new TypedProperties()); + public DFSPropertiesConfiguration() { + this.hadoopConfig = null; + this.currentFilePath = null; + this.hoodieConfig = new HoodieConfig(); + this.visitedFilePaths = new HashSet<>(); } - public DFSPropertiesConfiguration() { - this.fs = null; - this.rootFile = null; - this.props = new TypedProperties(); - this.visitedFiles = new HashSet<>(); + /** + * Load global props from hudi-defaults.conf which is under CONF_FILE_DIR_ENV_NAME. + * @return Typed Properties + */ + public static TypedProperties loadGlobalProps() { + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(); + Option defaultConfPath = getConfPathFromEnv(); + if (defaultConfPath.isPresent()) { + conf.addPropsFromFile(defaultConfPath.get()); + } else { + conf.addPropsFromFile(DEFAULT_PATH); + } + return conf.getProps(); } - private String[] splitProperty(String line) { - int ind = line.indexOf('='); - String k = line.substring(0, ind).trim(); - String v = line.substring(ind + 1).trim(); - return new String[] {k, v}; + public static void refreshGlobalProps() { + GLOBAL_PROPS = loadGlobalProps(); + } + + public static void clearGlobalProps() { + GLOBAL_PROPS = new TypedProperties(); } - private void visitFile(Path file) { + /** + * Add properties from external configuration files. + * + * @param filePath File path for configuration file + */ + public void addPropsFromFile(Path filePath) { + if (visitedFilePaths.contains(filePath.toString())) { + throw new IllegalStateException("Loop detected; file " + filePath + " already referenced"); + } + + FileSystem fs = FSUtils.getFs( + filePath.toString(), + Option.ofNullable(hadoopConfig).orElseGet(Configuration::new) + ); try { - if (visitedFiles.contains(file.getName())) { - throw new IllegalStateException("Loop detected; file " + file + " already referenced"); + if (filePath.equals(DEFAULT_PATH) && !fs.exists(filePath)) { + LOG.warn("Properties file " + filePath + " not found. Ignoring to load props file"); + return; } - visitedFiles.add(file.getName()); - BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file))); - addProperties(reader); + + BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath))); + visitedFilePaths.add(filePath.toString()); + currentFilePath = filePath; + addPropsFromStream(reader); } catch (IOException ioe) { - LOG.error("Error reading in properies from dfs", ioe); - throw new IllegalArgumentException("Cannot read properties from dfs", ioe); + LOG.error("Error reading in properties from dfs from file " + filePath); + throw new HoodieIOException("Cannot read properties from dfs from file " + filePath, ioe); } } /** - * Add properties from input stream. - * + * Add properties from buffered reader. + * * @param reader Buffered Reader * @throws IOException */ - public void addProperties(BufferedReader reader) throws IOException { + public void addPropsFromStream(BufferedReader reader) throws IOException { try { reader.lines().forEach(line -> { - if (line.startsWith("#") || line.equals("") || !line.contains("=")) { + if (!isValidLine(line)) { return; } String[] split = splitProperty(line); if (line.startsWith("include=") || line.startsWith("include =")) { - visitFile(new Path(rootFile.getParent(), split[1])); + Path includeFilePath = new Path(currentFilePath.getParent(), split[1]); + addPropsFromFile(includeFilePath); } else { - props.setProperty(split[0], split[1]); + hoodieConfig.setValue(split[0], split[1]); } }); @@ -117,7 +168,46 @@ public void addProperties(BufferedReader reader) throws IOException { } } - public TypedProperties getConfig() { - return props; + public static TypedProperties getGlobalProps() { + final TypedProperties globalProps = new TypedProperties(); + globalProps.putAll(GLOBAL_PROPS); + return globalProps; + } + + public TypedProperties getProps() { + return new TypedProperties(hoodieConfig.getProps()); + } + + public TypedProperties getProps(boolean includeGlobalProps) { + return new TypedProperties(hoodieConfig.getProps(includeGlobalProps)); + } + + private static Option getConfPathFromEnv() { + String confDir = System.getenv(CONF_FILE_DIR_ENV_NAME); + if (confDir == null) { + LOG.warn("Cannot find " + CONF_FILE_DIR_ENV_NAME + ", please set it as the dir of " + DEFAULT_PROPERTIES_FILE); + return Option.empty(); + } + if (StringUtils.isNullOrEmpty(URI.create(confDir).getScheme())) { + confDir = "file://" + confDir; + } + return Option.of(new Path(confDir + File.separator + DEFAULT_PROPERTIES_FILE)); + } + + private String[] splitProperty(String line) { + line = line.replaceAll("\\s+", " "); + String delimiter = line.contains("=") ? "=" : " "; + int ind = line.indexOf(delimiter); + String k = line.substring(0, ind).trim(); + String v = line.substring(ind + 1).trim(); + return new String[] {k, v}; + } + + private boolean isValidLine(String line) { + ValidationUtils.checkArgument(line != null, "passed line is null"); + if (line.startsWith("#") || line.equals("")) { + return false; + } + return line.contains("=") || line.matches(".*\\s.*"); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index ed2b90eeae4f..997f8a31e82f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -44,20 +44,28 @@ public static HoodieConfig create(FSDataInputStream inputStream) throws IOExcept return config; } - protected Properties props; + protected TypedProperties props; public HoodieConfig() { - this.props = new Properties(); + this.props = new TypedProperties(); } public HoodieConfig(Properties props) { - this.props = props; + this.props = new TypedProperties(props); } public void setValue(ConfigProperty cfg, String val) { props.setProperty(cfg.key(), val); } + public void setValue(String key, String val) { + props.setProperty(key, val); + } + + public void setAll(Properties properties) { + props.putAll(properties); + } + public void setDefaultValue(ConfigProperty configProperty) { if (!contains(configProperty)) { Option inferValue = Option.empty(); @@ -82,7 +90,7 @@ public boolean contains(ConfigProperty configProperty) { if (props.containsKey(configProperty.key())) { return true; } - return Arrays.stream(configProperty.getAlternatives()).anyMatch(props::containsKey); + return configProperty.getAlternatives().stream().anyMatch(props::containsKey); } private Option getRawValue(ConfigProperty configProperty) { @@ -131,6 +139,12 @@ public Integer getInt(ConfigProperty configProperty) { return rawValue.map(v -> Integer.parseInt(v.toString())).orElse(null); } + public Integer getIntOrDefault(ConfigProperty configProperty) { + Option rawValue = getRawValue(configProperty); + return rawValue.map(v -> Integer.parseInt(v.toString())) + .orElse((Integer) configProperty.defaultValue()); + } + public Boolean getBoolean(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Boolean.parseBoolean(v.toString())).orElse(null); @@ -139,7 +153,7 @@ public Boolean getBoolean(ConfigProperty configProperty) { public boolean getBooleanOrDefault(ConfigProperty configProperty) { Option rawValue = getRawValue(configProperty); return rawValue.map(v -> Boolean.parseBoolean(v.toString())) - .orElse(Boolean.parseBoolean(configProperty.defaultValue().toString())); + .orElseGet(() -> Boolean.parseBoolean(configProperty.defaultValue().toString())); } public Long getLong(ConfigProperty configProperty) { @@ -166,8 +180,18 @@ public String getStringOrDefault(ConfigProperty configProperty, String de return rawValue.map(Object::toString).orElse(defaultVal); } - public Properties getProps() { - return props; + public TypedProperties getProps() { + return getProps(false); + } + + public TypedProperties getProps(boolean includeGlobalProps) { + if (includeGlobalProps) { + TypedProperties mergedProps = DFSPropertiesConfiguration.getGlobalProps(); + mergedProps.putAll(props); + return mergedProps; + } else { + return props; + } } public void setDefaultOnCondition(boolean condition, HoodieConfig config) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 75d014521ff4..51791c945d58 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -18,6 +18,9 @@ package org.apache.hudi.common.config; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.exception.HoodieNotSupportedException; + import javax.annotation.concurrent.Immutable; import java.io.File; @@ -115,20 +118,25 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Parallelism to use, when listing the table on lake storage."); - public static final ConfigProperty ENABLE_INLINE_READING = ConfigProperty - .key(METADATA_PREFIX + ".enable.inline.reading") - .defaultValue(true) - .sinceVersion("0.10.0") - .withDocumentation("Enable inline reading of Log files. By default log block contents are read as byte[] using regular input stream and records " - + "are deserialized from it. Enabling this will read each log block as an inline file and read records from the same. For instance, " - + "for HFileDataBlock, a inline file will be read using HFileReader."); - public static final ConfigProperty ENABLE_FULL_SCAN_LOG_FILES = ConfigProperty .key(METADATA_PREFIX + ".enable.full.scan.log.files") .defaultValue(true) .sinceVersion("0.10.0") .withDocumentation("Enable full scanning of log files while reading log records. If disabled, hudi does look up of only interested entries."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty + .key(METADATA_PREFIX + ".populate.meta.fields") + .defaultValue(true) + .sinceVersion("0.10.0") + .withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated."); + + public static final ConfigProperty IGNORE_SPURIOUS_DELETES = ConfigProperty + .key("_" + METADATA_PREFIX + ".ignore.spurious.deletes") + .defaultValue(true) + .sinceVersion("0.10.10") + .withDocumentation("There are cases when extra files are requested to be deleted from metadata table which was never added before. This config" + + "determines how to handle such spurious deletes"); + private HoodieMetadataConfig() { super(); } @@ -161,8 +169,17 @@ public boolean enableFullScan() { return getBoolean(ENABLE_FULL_SCAN_LOG_FILES); } + public boolean populateMetaFields() { + return getBooleanOrDefault(HoodieMetadataConfig.POPULATE_META_FIELDS); + } + + public boolean ignoreSpuriousDeletes() { + return getBoolean(IGNORE_SPURIOUS_DELETES); + } + public static class Builder { + private EngineType engineType = EngineType.SPARK; private final HoodieMetadataConfig metadataConfig = new HoodieMetadataConfig(); public Builder fromFile(File propertiesFile) throws IOException { @@ -202,6 +219,11 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } + public Builder withPopulateMetaFields(boolean populateMetaFields) { + metadataConfig.setValue(POPULATE_META_FIELDS, Boolean.toString(populateMetaFields)); + return this; + } + public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { metadataConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); metadataConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); @@ -233,10 +255,33 @@ public Builder enableFullScan(boolean enableFullScan) { return this; } + public Builder ignoreSpuriousDeletes(boolean validateMetadataPayloadConsistency) { + metadataConfig.setValue(IGNORE_SPURIOUS_DELETES, String.valueOf(validateMetadataPayloadConsistency)); + return this; + } + + public Builder withEngineType(EngineType engineType) { + this.engineType = engineType; + return this; + } + public HoodieMetadataConfig build() { + metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType)); metadataConfig.setDefaults(HoodieMetadataConfig.class.getName()); return metadataConfig; } + + private boolean getDefaultMetadataEnable(EngineType engineType) { + switch (engineType) { + case SPARK: + return ENABLE.defaultValue(); + case FLINK: + case JAVA: + return false; + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java index 41d8cd775f3a..0109f22097a3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java @@ -30,14 +30,14 @@ public class LockConfiguration implements Serializable { public static final String LOCK_PREFIX = "hoodie.write.lock."; public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "wait_time_ms_between_retry"; - public static final String DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L); + public static final String DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(1000L); public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "max_wait_time_ms_between_retry"; public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "client.wait_time_ms_between_retry"; public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "num_retries"; - public static final String DEFAULT_LOCK_ACQUIRE_NUM_RETRIES = String.valueOf(3); + public static final String DEFAULT_LOCK_ACQUIRE_NUM_RETRIES = String.valueOf(15); public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "client.num_retries"; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index dc4df23a4c3d..74b673dcda2c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -462,10 +462,19 @@ public static FileStatus[] getAllDataFilesInPartition(FileSystem fs, Path partit .map(HoodieFileFormat::getFileExtension).collect(Collectors.toCollection(HashSet::new)); final String logFileExtension = HoodieFileFormat.HOODIE_LOG.getFileExtension(); - return Arrays.stream(fs.listStatus(partitionPath, path -> { - String extension = FSUtils.getFileExtension(path.getName()); - return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension); - })).filter(FileStatus::isFile).toArray(FileStatus[]::new); + try { + return Arrays.stream(fs.listStatus(partitionPath, path -> { + String extension = FSUtils.getFileExtension(path.getName()); + return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension); + })).filter(FileStatus::isFile).toArray(FileStatus[]::new); + } catch (IOException e) { + // return empty FileStatus if partition does not exist already + if (!fs.exists(partitionPath)) { + return new FileStatus[0]; + } else { + throw e; + } + } } /** @@ -670,19 +679,30 @@ public static Map parallelizeSubPathProcess( .filter(subPathPredicate) .map(fileStatus -> fileStatus.getPath().toString()) .collect(Collectors.toList()); - if (subPaths.size() > 0) { - SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); - int actualParallelism = Math.min(subPaths.size(), parallelism); - result = hoodieEngineContext.mapToPair(subPaths, - subPath -> new ImmutablePair<>(subPath, pairFunction.apply(new ImmutablePair<>(subPath, conf))), - actualParallelism); - } + result = parallelizeFilesProcess(hoodieEngineContext, fs, parallelism, pairFunction, subPaths); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } return result; } + public static Map parallelizeFilesProcess( + HoodieEngineContext hoodieEngineContext, + FileSystem fs, + int parallelism, + SerializableFunction, T> pairFunction, + List subPaths) { + Map result = new HashMap<>(); + if (subPaths.size() > 0) { + SerializableConfiguration conf = new SerializableConfiguration(fs.getConf()); + int actualParallelism = Math.min(subPaths.size(), parallelism); + result = hoodieEngineContext.mapToPair(subPaths, + subPath -> new ImmutablePair<>(subPath, pairFunction.apply(new ImmutablePair<>(subPath, conf))), + actualParallelism); + } + return result; + } + /** * Deletes a sub-path. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 1cc00df4ebd7..5dcd0b156aa3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -54,6 +54,8 @@ public enum StorageSchemes { COSN("cosn", false), // Tencent Cloud HDFS CHDFS("ofs", true), + // Tencent Cloud CacheFileSystem + GOOSEFS("gfs", false), // Databricks file system DBFS("dbfs", false), // IBM Cloud Object Storage diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java index e4570f94227c..a2c60bc318e4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFSUtils.java @@ -19,6 +19,9 @@ package org.apache.hudi.common.fs.inline; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.util.ValidationUtils; + +import java.io.File; /** * Utils to parse InLineFileSystem paths. @@ -29,46 +32,58 @@ public class InLineFSUtils { private static final String START_OFFSET_STR = "start_offset"; private static final String LENGTH_STR = "length"; + private static final String PATH_SEPARATOR = "/"; + private static final String SCHEME_SEPARATOR = ":"; private static final String EQUALS_STR = "="; + private static final String LOCAL_FILESYSTEM_SCHEME = "file"; /** - * Fetch inline file path from outer path. - * Eg - * Input: - * Path = s3a://file1, origScheme: file, startOffset = 20, length = 40 - * Output: "inlinefs:/file1/s3a/?start_offset=20&length=40" + * Get the InlineFS Path for a given schema and its Path. + *

+ * Examples: + * Input Path: s3a://file1, origScheme: file, startOffset = 20, length = 40 + * Output: "inlinefs://file1/s3a/?start_offset=20&length=40" * - * @param outerPath - * @param origScheme - * @param inLineStartOffset - * @param inLineLength - * @return + * @param outerPath The outer file Path + * @param origScheme The file schema + * @param inLineStartOffset Start offset for the inline file + * @param inLineLength Length for the inline file + * @return InlineFS Path for the requested outer path and schema */ public static Path getInlineFilePath(Path outerPath, String origScheme, long inLineStartOffset, long inLineLength) { - String subPath = outerPath.toString().substring(outerPath.toString().indexOf(":") + 1); + final String subPath = new File(outerPath.toString().substring(outerPath.toString().indexOf(":") + 1)).getPath(); return new Path( - InLineFileSystem.SCHEME + "://" + subPath + "/" + origScheme - + "/" + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + InLineFileSystem.SCHEME + SCHEME_SEPARATOR + PATH_SEPARATOR + subPath + PATH_SEPARATOR + origScheme + + PATH_SEPARATOR + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + "&" + LENGTH_STR + EQUALS_STR + inLineLength ); } /** - * Inline file format - * "inlinefs:////?start_offset=start_offset>&length=" - * Outer File format - * "://" + * InlineFS Path format: + * "inlinefs://path/to/outer/file/outer_file_schema/?start_offset=start_offset>&length=" *

- * Eg input : "inlinefs://file1/sa3/?start_offset=20&length=40". - * Output : "sa3://file1" + * Outer File Path format: + * "outer_file_schema://path/to/outer/file" + *

+ * Example + * Input: "inlinefs://file1/s3a/?start_offset=20&length=40". + * Output: "s3a://file1" * - * @param inlinePath inline file system path - * @return + * @param inlineFSPath InLineFS Path to get the outer file Path + * @return Outer file Path from the InLineFS Path */ - public static Path getOuterfilePathFromInlinePath(Path inlinePath) { - String scheme = inlinePath.getParent().getName(); - Path basePath = inlinePath.getParent().getParent(); - return new Path(basePath.toString().replaceFirst(InLineFileSystem.SCHEME, scheme)); + public static Path getOuterFilePathFromInlinePath(Path inlineFSPath) { + final String scheme = inlineFSPath.getParent().getName(); + final Path basePath = inlineFSPath.getParent().getParent(); + ValidationUtils.checkArgument(basePath.toString().contains(SCHEME_SEPARATOR), + "Invalid InLineFSPath: " + inlineFSPath); + + final String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(SCHEME_SEPARATOR) + 1); + final String fullPath = scheme + SCHEME_SEPARATOR + + (scheme.equals(LOCAL_FILESYSTEM_SCHEME) ? PATH_SEPARATOR : "") + + pathExceptScheme; + return new Path(fullPath); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java index 4c693c5c5d0f..712b6c7ff4e3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/inline/InLineFileSystem.java @@ -63,7 +63,7 @@ public String getScheme() { @Override public FSDataInputStream open(Path inlinePath, int bufferSize) throws IOException { - Path outerPath = InLineFSUtils.getOuterfilePathFromInlinePath(inlinePath); + Path outerPath = InLineFSUtils.getOuterFilePathFromInlinePath(inlinePath); FileSystem outerFs = outerPath.getFileSystem(conf); FSDataInputStream outerStream = outerFs.open(outerPath, bufferSize); return new InLineFsDataInputStream(InLineFSUtils.startOffset(inlinePath), outerStream, InLineFSUtils.length(inlinePath)); @@ -80,7 +80,7 @@ public boolean exists(Path f) { @Override public FileStatus getFileStatus(Path inlinePath) throws IOException { - Path outerPath = InLineFSUtils.getOuterfilePathFromInlinePath(inlinePath); + Path outerPath = InLineFSUtils.getOuterFilePathFromInlinePath(inlinePath); FileSystem outerFs = outerPath.getFileSystem(conf); FileStatus status = outerFs.getFileStatus(outerPath); FileStatus toReturn = new FileStatus(InLineFSUtils.length(inlinePath), status.isDirectory(), status.getReplication(), status.getBlockSize(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java index 76474fde66ea..07ab5df9c7db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -113,7 +113,7 @@ protected boolean needUpdatingPersistedRecord(IndexedRecord currentValue, Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true); Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord, - properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), false); + properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true); return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java new file mode 100644 index 000000000000..9b1c695d491e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java @@ -0,0 +1,37 @@ +/* + * 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 org.apache.hudi.common.model; + +/** + * Hoodie TimelineZone. + */ +public enum HoodieTimelineTimeZone { + LOCAL("local"), + UTC("utc"); + + private final String timeZone; + + HoodieTimelineTimeZone(String timeZone) { + this.timeZone = timeZone; + } + + public String getTimeZone() { + return timeZone; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java new file mode 100644 index 000000000000..33f1d9f0025b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java @@ -0,0 +1,89 @@ +/* + * 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 org.apache.hudi.common.model.debezium; + +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; + +/** + * Base class that provides support for seamlessly applying changes captured via Debezium. + *

+ * Debezium change event types are determined for the op field in the payload + *

+ * - For inserts, op=i + * - For deletes, op=d + * - For updates, op=u + * - For snapshort inserts, op=r + *

+ * This payload implementation will issue matching insert, delete, updates against the hudi table + */ +public abstract class AbstractDebeziumAvroPayload extends OverwriteWithLatestAvroPayload { + + private static final Logger LOG = LogManager.getLogger(AbstractDebeziumAvroPayload.class); + + public AbstractDebeziumAvroPayload(GenericRecord record, Comparable orderingVal) { + super(record, orderingVal); + } + + public AbstractDebeziumAvroPayload(Option record) { + super(record); + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + IndexedRecord insertRecord = getInsertRecord(schema); + return handleDeleteOperation(insertRecord); + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + // Step 1: If the time occurrence of the current record in storage is higher than the time occurrence of the + // insert record (including a delete record), pick the current record. + if (shouldPickCurrentRecord(currentValue, getInsertRecord(schema), schema)) { + return Option.of(currentValue); + } + // Step 2: Pick the insert record (as a delete record if its a deleted event) + return getInsertValue(schema); + } + + protected abstract boolean shouldPickCurrentRecord(IndexedRecord currentRecord, IndexedRecord insertRecord, Schema schema) throws IOException; + + private Option handleDeleteOperation(IndexedRecord insertRecord) { + boolean delete = false; + if (insertRecord instanceof GenericRecord) { + GenericRecord record = (GenericRecord) insertRecord; + Object value = record.get(DebeziumConstants.FLATTENED_OP_COL_NAME); + delete = value != null && value.toString().equalsIgnoreCase(DebeziumConstants.DELETE_OP); + } + + return delete ? Option.empty() : Option.of(insertRecord); + } + + private IndexedRecord getInsertRecord(Schema schema) throws IOException { + return super.getInsertValue(schema).get(); + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/DebeziumConstants.java b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/DebeziumConstants.java new file mode 100644 index 000000000000..d3e115e3d9d8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/DebeziumConstants.java @@ -0,0 +1,81 @@ +/* + * 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 org.apache.hudi.common.model.debezium; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Constants used by {@link DebeziumSource} and {@link DebeziumAvroPayload}. + */ +public class DebeziumConstants { + + // INPUT COLUMNS + public static final String INCOMING_BEFORE_FIELD = "before"; + public static final String INCOMING_AFTER_FIELD = "after"; + public static final String INCOMING_SOURCE_FIELD = "source"; + public static final String INCOMING_OP_FIELD = "op"; + public static final String INCOMING_TS_MS_FIELD = "ts_ms"; + + public static final String INCOMING_SOURCE_NAME_FIELD = "source.name"; + public static final String INCOMING_SOURCE_TS_MS_FIELD = "source.ts_ms"; + public static final String INCOMING_SOURCE_TXID_FIELD = "source.txId"; + + // INPUT COLUMNS SPECIFIC TO MYSQL + public static final String INCOMING_SOURCE_FILE_FIELD = "source.file"; + public static final String INCOMING_SOURCE_POS_FIELD = "source.pos"; + public static final String INCOMING_SOURCE_ROW_FIELD = "source.row"; + + // INPUT COLUMNS SPECIFIC TO POSTGRES + public static final String INCOMING_SOURCE_LSN_FIELD = "source.lsn"; + public static final String INCOMING_SOURCE_XMIN_FIELD = "source.xmin"; + + // OUTPUT COLUMNS + public static final String FLATTENED_OP_COL_NAME = "_change_operation_type"; + public static final String UPSTREAM_PROCESSING_TS_COL_NAME = "_upstream_event_processed_ts_ms"; + public static final String FLATTENED_SHARD_NAME = "db_shard_source_partition"; + public static final String FLATTENED_TS_COL_NAME = "_event_origin_ts_ms"; + public static final String FLATTENED_TX_ID_COL_NAME = "_event_tx_id"; + + // OUTPUT COLUMNS SPECIFIC TO MYSQL + public static final String FLATTENED_FILE_COL_NAME = "_event_bin_file"; + public static final String FLATTENED_POS_COL_NAME = "_event_pos"; + public static final String FLATTENED_ROW_COL_NAME = "_event_row"; + public static final String ADDED_SEQ_COL_NAME = "_event_seq"; + + // OUTPUT COLUMNS SPECIFIC TO POSTGRES + public static final String FLATTENED_LSN_COL_NAME = "_event_lsn"; + public static final String FLATTENED_XMIN_COL_NAME = "_event_xmin"; + + // Other Constants + public static final String DELETE_OP = "d"; + + // List of meta data columns + public static List META_COLUMNS = Collections.unmodifiableList(Arrays.asList( + FLATTENED_OP_COL_NAME, + UPSTREAM_PROCESSING_TS_COL_NAME, + FLATTENED_TS_COL_NAME, + FLATTENED_TX_ID_COL_NAME, + FLATTENED_LSN_COL_NAME, + FLATTENED_XMIN_COL_NAME, + FLATTENED_SHARD_NAME + )); +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/MySqlDebeziumAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/MySqlDebeziumAvroPayload.java new file mode 100644 index 000000000000..ea6165d55d3b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/MySqlDebeziumAvroPayload.java @@ -0,0 +1,67 @@ +/* + * 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 org.apache.hudi.common.model.debezium; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; + +/** + * Provides support for seamlessly applying changes captured via Debezium for MysqlDB. + *

+ * Debezium change event types are determined for the op field in the payload + *

+ * - For inserts, op=i + * - For deletes, op=d + * - For updates, op=u + * - For snapshort inserts, op=r + *

+ * This payload implementation will issue matching insert, delete, updates against the hudi table + */ +public class MySqlDebeziumAvroPayload extends AbstractDebeziumAvroPayload { + + private static final Logger LOG = LogManager.getLogger(MySqlDebeziumAvroPayload.class); + + public MySqlDebeziumAvroPayload(GenericRecord record, Comparable orderingVal) { + super(record, orderingVal); + } + + public MySqlDebeziumAvroPayload(Option record) { + super(record); + } + + private String extractSeq(IndexedRecord record) { + return ((CharSequence) ((GenericRecord) record).get(DebeziumConstants.ADDED_SEQ_COL_NAME)).toString(); + } + + @Override + protected boolean shouldPickCurrentRecord(IndexedRecord currentRecord, IndexedRecord insertRecord, Schema schema) throws IOException { + String currentSourceSeq = extractSeq(currentRecord); + String insertSourceSeq = extractSeq(insertRecord); + // Pick the current value in storage only if its Seq (file+pos) is latest + // compared to the Seq (file+pos) of the insert value + return insertSourceSeq.compareTo(currentSourceSeq) < 0; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/PostgresDebeziumAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/PostgresDebeziumAvroPayload.java new file mode 100644 index 000000000000..448627d97cbf --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/PostgresDebeziumAvroPayload.java @@ -0,0 +1,130 @@ +/* + * 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 org.apache.hudi.common.model.debezium; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.List; + +/** + * Provides support for seamlessly applying changes captured via Debezium for PostgresDB. + *

+ * Debezium change event types are determined for the op field in the payload + *

+ * - For inserts, op=i + * - For deletes, op=d + * - For updates, op=u + * - For snapshort inserts, op=r + *

+ * This payload implementation will issue matching insert, delete, updates against the hudi table + */ +public class PostgresDebeziumAvroPayload extends AbstractDebeziumAvroPayload { + + private static final Logger LOG = LogManager.getLogger(PostgresDebeziumAvroPayload.class); + public static final String DEBEZIUM_TOASTED_VALUE = "__debezium_unavailable_value"; + + public PostgresDebeziumAvroPayload(GenericRecord record, Comparable orderingVal) { + super(record, orderingVal); + } + + public PostgresDebeziumAvroPayload(Option record) { + super(record); + } + + private Long extractLSN(IndexedRecord record) { + GenericRecord genericRecord = (GenericRecord) record; + return (Long) genericRecord.get(DebeziumConstants.FLATTENED_LSN_COL_NAME); + } + + @Override + protected boolean shouldPickCurrentRecord(IndexedRecord currentRecord, IndexedRecord insertRecord, Schema schema) throws IOException { + Long currentSourceLSN = extractLSN(currentRecord); + Long insertSourceLSN = extractLSN(insertRecord); + + // Pick the current value in storage only if its LSN is latest compared to the LSN of the insert value + return insertSourceLSN < currentSourceLSN; + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + // Specific to Postgres: If the updated record has TOASTED columns, + // we will need to keep the previous value for those columns + // see https://debezium.io/documentation/reference/connectors/postgresql.html#postgresql-toasted-values + Option insertOrDeleteRecord = super.combineAndGetUpdateValue(currentValue, schema); + + if (insertOrDeleteRecord.isPresent()) { + mergeToastedValuesIfPresent(insertOrDeleteRecord.get(), currentValue); + } + return insertOrDeleteRecord; + } + + private void mergeToastedValuesIfPresent(IndexedRecord incomingRecord, IndexedRecord currentRecord) { + List fields = incomingRecord.getSchema().getFields(); + + fields.forEach(field -> { + // There are only four avro data types that have unconstrained sizes, which are + // NON-NULLABLE STRING, NULLABLE STRING, NON-NULLABLE BYTES, NULLABLE BYTES + if (((GenericData.Record) incomingRecord).get(field.name()) != null + && (containsStringToastedValues(incomingRecord, field) || containsBytesToastedValues(incomingRecord, field))) { + ((GenericData.Record) incomingRecord).put(field.name(), ((GenericData.Record) currentRecord).get(field.name())); + } + }); + } + + /** + * Returns true if a column is either of type string or a union of one or more strings that contain a debezium toasted value. + * + * @param incomingRecord The incoming avro record + * @param field the column of interest + * @return + */ + private boolean containsStringToastedValues(IndexedRecord incomingRecord, Schema.Field field) { + return ((field.schema().getType() == Schema.Type.STRING + || (field.schema().getType() == Schema.Type.UNION && field.schema().getTypes().stream().anyMatch(s -> s.getType() == Schema.Type.STRING))) + // Check length first as an optimization + && ((CharSequence) ((GenericData.Record) incomingRecord).get(field.name())).length() == DEBEZIUM_TOASTED_VALUE.length() + && DEBEZIUM_TOASTED_VALUE.equals(((CharSequence) ((GenericData.Record) incomingRecord).get(field.name())).toString())); + } + + /** + * Returns true if a column is either of type bytes or a union of one or more bytes that contain a debezium toasted value. + * + * @param incomingRecord The incoming avro record + * @param field the column of interest + * @return + */ + private boolean containsBytesToastedValues(IndexedRecord incomingRecord, Schema.Field field) { + return ((field.schema().getType() == Schema.Type.BYTES + || (field.schema().getType() == Schema.Type.UNION && field.schema().getTypes().stream().anyMatch(s -> s.getType() == Schema.Type.BYTES))) + // Check length first as an optimization + && ((ByteBuffer) ((GenericData.Record) incomingRecord).get(field.name())).array().length == DEBEZIUM_TOASTED_VALUE.length() + && DEBEZIUM_TOASTED_VALUE.equals(new String(((ByteBuffer) ((GenericData.Record) incomingRecord).get(field.name())).array(), StandardCharsets.UTF_8))); + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index dc57fd1c6ff8..54724d5f37ab 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -25,9 +25,13 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieTimelineTimeZone; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; @@ -46,6 +50,8 @@ import java.util.Date; import java.util.Map; import java.util.Properties; +import java.util.Set; +import java.util.function.BiConsumer; import java.util.stream.Collectors; /** @@ -68,6 +74,7 @@ public class HoodieTableConfig extends HoodieConfig { private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class); public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; + public static final String HOODIE_PROPERTIES_FILE_BACKUP = "hoodie.properties.backup"; public static final ConfigProperty NAME = ConfigProperty .key("hoodie.table.name") @@ -162,6 +169,11 @@ public class HoodieTableConfig extends HoodieConfig { .noDefaultValue() .withDocumentation("Key Generator class property for the hoodie table"); + public static final ConfigProperty TIMELINE_TIMEZONE = ConfigProperty + .key("hoodie.table.timeline.timezone") + .defaultValue(HoodieTimelineTimeZone.LOCAL) + .withDocumentation("User can set hoodie commit timeline timezone, such as utc, local and so on. local is default"); + public static final ConfigProperty URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING; public static final ConfigProperty HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE; @@ -172,12 +184,11 @@ public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE); LOG.info("Loading table properties from " + propertyPath); try { - try (FSDataInputStream inputStream = fs.open(propertyPath)) { - props.load(inputStream); - } + fetchConfigs(fs, metaPath); if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null && !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) { setValue(PAYLOAD_CLASS_NAME, payloadClassName); + // FIXME(vc): wonder if this can be removed. Need to look into history. try (FSDataOutputStream outputStream = fs.create(propertyPath)) { props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } @@ -191,16 +202,103 @@ public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName /** * For serializing and de-serializing. - * */ public HoodieTableConfig() { super(); } + private void fetchConfigs(FileSystem fs, String metaPath) throws IOException { + Path cfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE); + try (FSDataInputStream is = fs.open(cfgPath)) { + props.load(is); + } catch (IOException ioe) { + if (!fs.exists(cfgPath)) { + LOG.warn("Run `table recover-configs` if config update/delete failed midway. Falling back to backed up configs."); + // try the backup. this way no query ever fails if update fails midway. + Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP); + try (FSDataInputStream is = fs.open(backupCfgPath)) { + props.load(is); + } + } else { + throw ioe; + } + } + } + + public static void recover(FileSystem fs, Path metadataFolder) throws IOException { + Path cfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); + Path backupCfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE_BACKUP); + recoverIfNeeded(fs, cfgPath, backupCfgPath); + } + + static void recoverIfNeeded(FileSystem fs, Path cfgPath, Path backupCfgPath) throws IOException { + if (!fs.exists(cfgPath)) { + // copy over from backup + try (FSDataInputStream in = fs.open(backupCfgPath); + FSDataOutputStream out = fs.create(cfgPath, false)) { + FileIOUtils.copy(in, out); + } + } + // regardless, we don't need the backup anymore. + fs.delete(backupCfgPath, false); + } + + private static void upsertProperties(Properties current, Properties updated) { + updated.forEach((k, v) -> current.setProperty(k.toString(), v.toString())); + } + + private static void deleteProperties(Properties current, Properties deleted) { + deleted.forEach((k, v) -> current.remove(k.toString())); + } + + private static void modify(FileSystem fs, Path metadataFolder, Properties modifyProps, BiConsumer modifyFn) { + Path cfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); + Path backupCfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE_BACKUP); + try { + // 0. do any recovery from prior attempts. + recoverIfNeeded(fs, cfgPath, backupCfgPath); + + // 1. backup the existing properties. + try (FSDataInputStream in = fs.open(cfgPath); + FSDataOutputStream out = fs.create(backupCfgPath, false)) { + FileIOUtils.copy(in, out); + } + /// 2. delete the properties file, reads will go to the backup, until we are done. + fs.delete(cfgPath, false); + // 3. read current props, upsert and save back. + try (FSDataInputStream in = fs.open(backupCfgPath); + FSDataOutputStream out = fs.create(cfgPath, true)) { + Properties props = new Properties(); + props.load(in); + modifyFn.accept(props, modifyProps); + props.store(out, "Updated at " + System.currentTimeMillis()); + } + // 4. verify and remove backup. + // FIXME(vc): generate a hash for verification. + fs.delete(backupCfgPath, false); + } catch (IOException e) { + throw new HoodieIOException("Error updating table configs.", e); + } + } + + /** + * Upserts the table config with the set of properties passed in. We implement a fail-safe backup protocol + * here for safely updating with recovery and also ensuring the table config continues to be readable. + */ + public static void update(FileSystem fs, Path metadataFolder, Properties updatedProps) { + modify(fs, metadataFolder, updatedProps, HoodieTableConfig::upsertProperties); + } + + public static void delete(FileSystem fs, Path metadataFolder, Set deletedProps) { + Properties props = new Properties(); + deletedProps.forEach(p -> props.setProperty(p, "")); + modify(fs, metadataFolder, props, HoodieTableConfig::deleteProperties); + } + /** * Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties). */ - public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Properties properties) + public static void create(FileSystem fs, Path metadataFolder, Properties properties) throws IOException { if (!fs.exists(metadataFolder)) { fs.mkdirs(metadataFolder); @@ -224,6 +322,9 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Pr // Use the default bootstrap index class. hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_NAME, getDefaultBootstrapIndexClass(properties)); } + if (hoodieConfig.contains(TIMELINE_TIMEZONE)) { + HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getString(TIMELINE_TIMEZONE))); + } hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } } @@ -364,7 +465,7 @@ public boolean populateMetaFields() { * @returns the record key field prop. */ public String getRecordKeyFieldProp() { - return getString(RECORDKEY_FIELDS); + return getStringOrDefault(RECORDKEY_FIELDS, HoodieRecord.RECORD_KEY_METADATA_FIELD); } public String getKeyGeneratorClassName() { @@ -375,7 +476,7 @@ public String getHiveStylePartitioningEnable() { return getString(HIVE_STYLE_PARTITIONING_ENABLE); } - public String getUrlEncodePartitoning() { + public String getUrlEncodePartitioning() { return getString(URL_ENCODE_PARTITIONING); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 450a3cc2eb3a..f44d28eca635 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.fs.NoOpConsistencyGuard; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieTimelineTimeZone; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; @@ -79,7 +80,7 @@ public class HoodieTableMetaClient implements Serializable { public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".aux"; public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + Path.SEPARATOR + ".bootstrap"; public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".heartbeat"; - public static final String ZINDEX_NAME = ".zindex"; + public static final String COLUMN_STATISTICS_INDEX_NAME = ".colstatsindex"; public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR + ".partitions"; public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR @@ -178,10 +179,10 @@ public String getMetaPath() { } /** - * @return z-index path + * @return Column Statistics index path */ - public String getZindexPath() { - return new Path(metaPath, ZINDEX_NAME).toString(); + public String getColumnStatsIndexPath() { + return new Path(metaPath, COLUMN_STATISTICS_INDEX_NAME).toString(); } /** @@ -377,7 +378,7 @@ public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hado } initializeBootstrapDirsIfNotExists(hadoopConf, basePath, fs); - HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); + HoodieTableConfig.create(fs, metaPathDir, props); // We should not use fs.getConf as this might be different from the original configuration // used to create the fs in unit tests HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); @@ -639,6 +640,7 @@ public static class PropertyBuilder { private String keyGeneratorClassProp; private Boolean hiveStylePartitioningEnable; private Boolean urlEncodePartitioning; + private HoodieTimelineTimeZone commitTimeZone; private PropertyBuilder() { @@ -737,6 +739,11 @@ public PropertyBuilder setUrlEncodePartitioning(Boolean urlEncodePartitioning) { return this; } + public PropertyBuilder setCommitTimezone(HoodieTimelineTimeZone timelineTimeZone) { + this.commitTimeZone = timelineTimeZone; + return this; + } + public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) @@ -873,6 +880,9 @@ public Properties build() { if (null != urlEncodePartitioning) { tableConfig.setValue(HoodieTableConfig.URL_ENCODE_PARTITIONING, Boolean.toString(urlEncodePartitioning)); } + if (null != commitTimeZone) { + tableConfig.setValue(HoodieTableConfig.TIMELINE_TIMEZONE, commitTimeZone.toString()); + } return tableConfig.getProps(); } @@ -886,5 +896,6 @@ public HoodieTableMetaClient initTable(Configuration configuration, String baseP throws IOException { return HoodieTableMetaClient.initTableAndGetMetaClient(configuration, basePath, build()); } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 51e3e273806c..a953ac3ab78d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.table; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.SchemaCompatibility; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; @@ -35,12 +40,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.InvalidTableException; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; -import org.apache.avro.SchemaCompatibility; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; @@ -85,7 +84,8 @@ private MessageType getTableParquetSchemaFromDataFile() throws Exception { // If this is COW, get the last commit and read the schema from a file written in the // last commit HoodieInstant lastCommit = - activeTimeline.getCommitsTimeline().filterCompletedInstants().lastInstant().orElseThrow(() -> new InvalidTableException(metaClient.getBasePath())); + activeTimeline.getCommitsTimeline().filterCompletedInstantsWithCommitMetadata() + .lastInstant().orElseThrow(() -> new InvalidTableException(metaClient.getBasePath())); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(activeTimeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class); String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny() @@ -97,8 +97,8 @@ private MessageType getTableParquetSchemaFromDataFile() throws Exception { // If this is MOR, depending on whether the latest commit is a delta commit or // compaction commit // Get a datafile written and get the schema from that file - Option lastCompactionCommit = - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant(); + Option lastCompactionCommit = metaClient.getActiveTimeline().getCommitTimeline() + .filterCompletedInstantsWithCommitMetadata().lastInstant(); LOG.info("Found the last compaction commit as " + lastCompactionCommit); Option lastDeltaCommit; @@ -198,8 +198,15 @@ public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception */ public MessageType getTableParquetSchema() throws Exception { Option schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(true); - return schemaFromCommitMetadata.isPresent() ? convertAvroSchemaToParquet(schemaFromCommitMetadata.get()) : - getTableParquetSchemaFromDataFile(); + if (schemaFromCommitMetadata.isPresent()) { + return convertAvroSchemaToParquet(schemaFromCommitMetadata.get()); + } + Option schemaFromTableConfig = metaClient.getTableConfig().getTableCreateSchema(); + if (schemaFromTableConfig.isPresent()) { + Schema schema = HoodieAvroUtils.addMetadataFields(schemaFromTableConfig.get(), withOperationField); + return convertAvroSchemaToParquet(schema); + } + return getTableParquetSchemaFromDataFile(); } /** @@ -414,6 +421,25 @@ public Schema getLatestSchema(Schema writeSchema, boolean convertTableSchemaToAd return latestSchema; } + + /** + * Get Last commit's Metadata. + */ + public Option getLatestCommitMetadata() { + try { + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + if (timeline.lastInstant().isPresent()) { + HoodieInstant instant = timeline.lastInstant().get(); + byte[] data = timeline.getInstantDetails(instant).get(); + return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class)); + } else { + return Option.empty(); + } + } catch (Exception e) { + throw new HoodieException("Failed to get commit metadata", e); + } + } + /** * Read the parquet schema from a parquet File. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index e2e76ad7d650..d495badeca4e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -120,28 +121,32 @@ public abstract class AbstractHoodieLogRecordReader { private int totalScannedLogFiles; // Progress private float progress = 0.0f; + // Partition name + private Option partitionName; + // Populate meta fields for the records + private boolean populateMetaFields = true; - protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, Option instantRange, boolean withOperationField) { - this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - true); + int bufferSize, Option instantRange, + boolean withOperationField) { + this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, + instantRange, withOperationField, true, Option.empty()); } - protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, Option instantRange, boolean withOperationField, - boolean enableFullScan) { + protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, Option instantRange, + boolean withOperationField, boolean enableFullScan, + Option partitionName) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build(); // load class from the payload fully qualified class name - this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); - this.preCombineField = this.hoodieTableMetaClient.getTableConfig().getPreCombineField(); HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); - if (!tableConfig.populateMetaFields()) { - this.simpleKeyGenFields = Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp())); - } + this.payloadClassFQN = tableConfig.getPayloadClass(); + this.preCombineField = tableConfig.getPreCombineField(); this.totalLogFiles.addAndGet(logFilePaths.size()); this.logFilePaths = logFilePaths; this.reverseReader = reverseReader; @@ -151,6 +156,22 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List> keys) { HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants(); HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights(); try { - // iterate over the paths + + // Get the key field based on populate meta fields config + // and the table type + final String keyField = getKeyField(); + + // Iterate over the paths logFormatReaderWrapper = new HoodieLogFormatReader(fs, logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()), - readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan); + readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField); Set scannedLogFiles = new HashSet<>(); while (logFormatReaderWrapper.hasNext()) { HoodieLogFile logFile = logFormatReaderWrapper.getLogFile(); @@ -339,15 +365,34 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option> ke } totalLogRecords.addAndGet(recs.size()); for (IndexedRecord rec : recs) { - processNextRecord(createHoodieRecord(rec)); + processNextRecord(createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, + this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); } } - protected HoodieRecord createHoodieRecord(IndexedRecord rec) { - if (!simpleKeyGenFields.isPresent()) { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN, this.preCombineField, this.withOperationField); + /** + * Create @{@link HoodieRecord} from the @{@link IndexedRecord}. + * + * @param rec - IndexedRecord to create the HoodieRecord from + * @param hoodieTableConfig - Table config + * @param payloadClassFQN - Payload class fully qualified name + * @param preCombineField - PreCombine field + * @param withOperationField - Whether operation field is enabled + * @param simpleKeyGenFields - Key generator fields when populate meta fields is tuened off + * @param partitionName - Partition name + * @return HoodieRecord created from the IndexedRecord + */ + protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + final String payloadClassFQN, final String preCombineField, + final boolean withOperationField, + final Option> simpleKeyGenFields, + final Option partitionName) { + if (this.populateMetaFields) { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, withOperationField); } else { - return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN, this.preCombineField, this.simpleKeyGenFields.get(), this.withOperationField); + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); } } @@ -418,6 +463,10 @@ protected String getPayloadClassFQN() { return payloadClassFQN; } + protected Option getPartitionName() { + return partitionName; + } + public long getTotalRollbacks() { return totalRollbacks.get(); } @@ -451,6 +500,10 @@ public abstract static class Builder { public abstract Builder withBufferSize(int bufferSize); + public Builder withPartition(String partitionName) { + throw new UnsupportedOperationException(); + } + public Builder withInstantRange(Option instantRange) { throw new UnsupportedOperationException(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java index f7a7acfa9022..a786e8305bc2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -23,33 +23,56 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import java.io.IOException; import java.util.Iterator; +import java.util.stream.StreamSupport; /** * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice. */ public class HoodieFileSliceReader implements Iterator> { - private Iterator> recordsIterator; + private final Iterator> recordsIterator; - public static HoodieFileSliceReader getFileSliceReader( - HoodieFileReader baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, - String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { - Iterator baseIterator = baseFileReader.getRecordIterator(schema); - while (baseIterator.hasNext()) { - GenericRecord record = (GenericRecord) baseIterator.next(); - HoodieRecord hoodieRecord = simpleKeyGenFieldsOpt.isPresent() - ? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField()) - : SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, preCombineField, scanner.isWithOperationField()); - scanner.processNextRecord(hoodieRecord); + public static HoodieFileSliceReader getFileSliceReader( + Option baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass, + String preCombineField, Option> simpleKeyGenFieldsOpt) throws IOException { + if (baseFileReader.isPresent()) { + Iterator baseIterator = baseFileReader.get().getRecordIterator(schema); + while (baseIterator.hasNext()) { + GenericRecord record = (GenericRecord) baseIterator.next(); + HoodieRecord hoodieRecord = transform( + record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); + scanner.processNextRecord(hoodieRecord); + } + return new HoodieFileSliceReader(scanner.iterator()); + } else { + Iterable> iterable = () -> scanner.iterator(); + return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false) + .map(e -> { + try { + GenericRecord record = (GenericRecord) e.getData().getInsertValue(schema).get(); + return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt); + } catch (IOException io) { + throw new HoodieIOException("Error while creating reader for file slice with no base file.", io); + } + }).iterator()); } - return new HoodieFileSliceReader(scanner.iterator()); + } + + private static HoodieRecord transform( + GenericRecord record, HoodieMergedLogRecordScanner scanner, String payloadClass, + String preCombineField, Option> simpleKeyGenFieldsOpt) { + return simpleKeyGenFieldsOpt.isPresent() + ? SpillableMapUtils.convertToHoodieRecordPayload(record, + payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField(), Option.empty()) + : SpillableMapUtils.convertToHoodieRecordPayload(record, + payloadClass, preCombineField, scanner.isWithOperationField(), scanner.getPartitionName()); } private HoodieFileSliceReader(Iterator> recordsItr) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index cdf3065587d1..e6ead54a48d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.SchemeAwareFSDataInputStream; import org.apache.hudi.common.fs.TimedFSDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieCorruptBlock; @@ -66,6 +67,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private final HoodieLogFile logFile; private final byte[] magicBuffer = new byte[6]; private final Schema readerSchema; + private final String keyField; private boolean readBlockLazily; private long reverseLogFilePosition; private long lastReverseLogFilePosition; @@ -74,13 +76,20 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private boolean closed = false; private transient Thread shutdownThread = null; + public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, + boolean readBlockLazily) throws IOException { + this(fs, logFile, readerSchema, bufferSize, readBlockLazily, false); + } + public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, boolean readBlockLazily, boolean reverseReader) throws IOException { - this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false); + this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false, + HoodieRecord.RECORD_KEY_METADATA_FIELD); } public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readBlockLazily, boolean reverseReader, boolean enableInlineReading) throws IOException { + boolean readBlockLazily, boolean reverseReader, boolean enableInlineReading, + String keyField) throws IOException { FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize); this.logFile = logFile; this.inputStream = getFSDataInputStream(fsDataInputStream, fs, bufferSize); @@ -88,17 +97,13 @@ public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSc this.readBlockLazily = readBlockLazily; this.reverseReader = reverseReader; this.enableInlineReading = enableInlineReading; + this.keyField = keyField; if (this.reverseReader) { - this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen(); + this.reverseLogFilePosition = this.lastReverseLogFilePosition = logFile.getFileSize(); } addShutDownHook(); } - public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean readBlockLazily, - boolean reverseReader) throws IOException { - this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader); - } - public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false); } @@ -251,11 +256,12 @@ private HoodieLogBlock readBlock() throws IOException { return HoodieAvroDataBlock.getBlock(content, readerSchema); } else { return new HoodieAvroDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer); + contentPosition, contentLength, blockEndPos, readerSchema, header, footer, keyField); } case HFILE_DATA_BLOCK: return new HoodieHFileDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer, enableInlineReading); + contentPosition, contentLength, blockEndPos, readerSchema, + header, footer, enableInlineReading, keyField); case DELETE_BLOCK: return HoodieDeleteBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily, contentPosition, contentLength, blockEndPos, header, footer); @@ -278,7 +284,7 @@ private HoodieLogBlock createCorruptBlock() throws IOException { long contentPosition = inputStream.getPos(); byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); return HoodieCorruptBlock.getBlock(logFile, inputStream, Option.ofNullable(corruptedBytes), readBlockLazily, - contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), new HashMap<>()); + contentPosition, corruptedBlockSize, nextBlockOffset, new HashMap<>(), new HashMap<>()); } private boolean isBlockCorrupt(int blocksize) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java index c566788fd166..569b4a23b683 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java @@ -274,7 +274,7 @@ static WriterBuilder newWriterBuilder() { static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { - return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false); + return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false); } static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java index 36fa187aa411..d4a173d069c7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java @@ -44,17 +44,15 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final Schema readerSchema; private final boolean readBlocksLazily; private final boolean reverseLogReader; + private final String recordKeyField; + private final boolean enableInlineReading; private int bufferSize; private static final Logger LOG = LogManager.getLogger(HoodieLogFormatReader.class); HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily, - boolean reverseLogReader, int bufferSize) throws IOException { - this(fs, logFiles, readerSchema, readBlocksLazily, reverseLogReader, bufferSize, false); - } - - HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily, - boolean reverseLogReader, int bufferSize, boolean enableInlineReading) throws IOException { + boolean reverseLogReader, int bufferSize, boolean enableInlineReading, + String recordKeyField) throws IOException { this.logFiles = logFiles; this.fs = fs; this.readerSchema = readerSchema; @@ -62,9 +60,12 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { this.reverseLogReader = reverseLogReader; this.bufferSize = bufferSize; this.prevReadersInOpenState = new ArrayList<>(); + this.recordKeyField = recordKeyField; + this.enableInlineReading = enableInlineReading; if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, enableInlineReading); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, + enableInlineReading, recordKeyField); } } @@ -103,8 +104,8 @@ public boolean hasNext() { } else { this.prevReadersInOpenState.add(currentReader); } - this.currentReader = - new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false); + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false, + enableInlineReading, recordKeyField); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java index d7e4f7ef108b..1c33b81246c5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java @@ -148,10 +148,11 @@ public AppendResult appendBlocks(List blocks) throws IOException HoodieLogFormat.LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.CURRENT_VERSION); - FSDataOutputStream outputStream = getOutputStream(); - long startPos = outputStream.getPos(); + FSDataOutputStream originalOutputStream = getOutputStream(); + long startPos = originalOutputStream.getPos(); long sizeWritten = 0; - + // HUDI-2655. here we wrap originalOutputStream to ensure huge blocks can be correctly written + FSDataOutputStream outputStream = new FSDataOutputStream(originalOutputStream, new FileSystem.Statistics(fs.getScheme()), startPos); for (HoodieLogBlock block: blocks) { long startSize = outputStream.size(); @@ -189,6 +190,11 @@ public AppendResult appendBlocks(List blocks) throws IOException outputStream.writeLong(outputStream.size() - startSize); // Fetch the size again, so it accounts also (9). + + // HUDI-2655. Check the size written to avoid log blocks whose size overflow. + if (outputStream.size() == Integer.MAX_VALUE) { + throw new HoodieIOException("Blocks appended may overflow. Please decrease log block size or log block amount"); + } sizeWritten += outputStream.size() - startSize; } // Flush all blocks to disk diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index a8d97ac1b5f1..2e47e695d314 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -76,10 +76,13 @@ protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List instantRange, boolean autoScan, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, - boolean withOperationField, boolean enableFullScan) { - super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, - enableFullScan); + ExternalSpillableMap.DiskMapType diskMapType, + boolean isBitCaskDiskMapCompressionEnabled, + boolean withOperationField, boolean enableFullScan, + Option partitionName) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, + instantRange, withOperationField, + enableFullScan, partitionName); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), @@ -187,6 +190,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { private boolean autoScan = true; // operation field default false private boolean withOperationField = false; + protected String partitionName; @Override public Builder withFileSystem(FileSystem fs) { @@ -272,12 +276,19 @@ public Builder withOperationField(boolean withOperationField) { return this; } + @Override + public Builder withPartition(String partitionName) { + this.partitionName = partitionName; + return this; + } + @Override public HoodieMergedLogRecordScanner build() { return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, bufferSize, spillableMapBasePath, instantRange, autoScan, - diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true); + diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true, + Option.ofNullable(partitionName)); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/LogReaderUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/LogReaderUtils.java index fe159df00778..c1b20cbb4c55 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/LogReaderUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/LogReaderUtils.java @@ -27,14 +27,15 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.collection.Pair; import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -42,9 +43,10 @@ */ public class LogReaderUtils { - private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, Path path) + private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, HoodieLogFile hoodieLogFile) throws IOException { - Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true, true); + // set length for the HoodieLogFile as it will be leveraged by HoodieLogFormat.Reader with reverseReading enabled + Reader reader = HoodieLogFormat.newReader(fs, hoodieLogFile, null, true, true); Schema writerSchema = null; HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); while (reader.hasPrev()) { @@ -62,17 +64,17 @@ private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActive return writerSchema; } - public static Schema readLatestSchemaFromLogFiles(String basePath, List deltaFilePaths, Configuration config) + public static Schema readLatestSchemaFromLogFiles(String basePath, List logFiles, Configuration config) throws IOException { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(config).setBasePath(basePath).build(); - List deltaPaths = deltaFilePaths.stream().map(s -> new HoodieLogFile(new Path(s))) - .sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString()) + List deltaPaths = logFiles.stream().sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString()) .collect(Collectors.toList()); if (deltaPaths.size() > 0) { + Map deltaFilePathToFileStatus = logFiles.stream().map(entry -> Pair.of(entry.getPath().toString(), entry)) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); for (String logPath : deltaPaths) { FileSystem fs = FSUtils.getFs(logPath, config); - Schema schemaFromLogFile = - readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), new Path(logPath)); + Schema schemaFromLogFile = readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), deltaFilePathToFileStatus.get(logPath)); if (schemaFromLogFile != null) { return schemaFromLogFile; } @@ -80,5 +82,4 @@ public static Schema readLatestSchemaFromLogFiles(String basePath, List } return null; } - } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index 31fc352acad0..1d3f5f3b01c5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.SizeAwareDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; @@ -58,22 +59,27 @@ public class HoodieAvroDataBlock extends HoodieDataBlock { private ThreadLocal decoderCache = new ThreadLocal<>(); public HoodieAvroDataBlock(@Nonnull Map logBlockHeader, - @Nonnull Map logBlockFooter, - @Nonnull Option blockContentLocation, @Nonnull Option content, - FSDataInputStream inputStream, boolean readBlockLazily) { + @Nonnull Map logBlockFooter, + @Nonnull Option blockContentLocation, @Nonnull Option content, + FSDataInputStream inputStream, boolean readBlockLazily) { super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); } public HoodieAvroDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option content, - boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, - Map header, Map footer) { + boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, + Map header, Map footer, String keyField) { super(content, inputStream, readBlockLazily, - Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, - footer); + Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, + footer, keyField); + } + + public HoodieAvroDataBlock(@Nonnull List records, @Nonnull Map header, String keyField) { + super(records, header, new HashMap<>(), keyField); } public HoodieAvroDataBlock(@Nonnull List records, @Nonnull Map header) { - super(records, header, new HashMap<>()); + super(records, header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java index 558053bc3f39..08909233a576 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java @@ -67,10 +67,10 @@ public byte[] getContentBytes() { } public static HoodieLogBlock getBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option content, - boolean readBlockLazily, long position, long blockSize, long blockEndpos, Map header, + boolean readBlockLazily, long position, long blockSize, long blockEndPos, Map header, Map footer) { return new HoodieCommandBlock(content, inputStream, readBlockLazily, - Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), header, footer); + Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index 2e4338ef785d..66c9571487df 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.log.block; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -29,7 +30,6 @@ import javax.annotation.Nonnull; import java.io.IOException; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,39 +46,62 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { protected List records; protected Schema schema; + protected String keyField; public HoodieDataBlock(@Nonnull Map logBlockHeader, @Nonnull Map logBlockFooter, @Nonnull Option blockContentLocation, @Nonnull Option content, FSDataInputStream inputStream, boolean readBlockLazily) { super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); + this.keyField = HoodieRecord.RECORD_KEY_METADATA_FIELD; } public HoodieDataBlock(@Nonnull List records, @Nonnull Map header, - @Nonnull Map footer) { - super(header, footer, Option.empty(), Option.empty(), null, false); + @Nonnull Map footer, String keyField) { + this(header, footer, Option.empty(), Option.empty(), null, false); this.records = records; this.schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - } - - public HoodieDataBlock(@Nonnull List records, @Nonnull Map header) { - this(records, header, new HashMap<>()); + this.keyField = keyField; } protected HoodieDataBlock(Option content, @Nonnull FSDataInputStream inputStream, boolean readBlockLazily, - Option blockContentLocation, Schema readerSchema, - @Nonnull Map headers, @Nonnull Map footer) { - super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); + Option blockContentLocation, Schema readerSchema, + @Nonnull Map headers, @Nonnull Map footer, String keyField) { + this(headers, footer, blockContentLocation, content, inputStream, readBlockLazily); this.schema = readerSchema; + this.keyField = keyField; } + /** + * Util method to get a data block for the requested type. + * + * @param logDataBlockFormat - Data block type + * @param recordList - List of records that goes in the data block + * @param header - data block header + * @return Data block of the requested type. + */ public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List recordList, Map header) { + return getBlock(logDataBlockFormat, recordList, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); + } + + /** + * Util method to get a data block for the requested type. + * + * @param logDataBlockFormat - Data block type + * @param recordList - List of records that goes in the data block + * @param header - data block header + * @param keyField - FieldId to get the key from the records + * @return Data block of the requested type. + */ + public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List recordList, + Map header, String keyField) { switch (logDataBlockFormat) { case AVRO_DATA_BLOCK: - return new HoodieAvroDataBlock(recordList, header); + return new HoodieAvroDataBlock(recordList, header, keyField); case HFILE_DATA_BLOCK: - return new HoodieHFileDataBlock(recordList, header); + return new HoodieHFileDataBlock(recordList, header, keyField); default: throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index a1e0c129803f..7f1fa2aa1d64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -26,18 +26,16 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieHBaseKVComparator; import org.apache.hudi.io.storage.HoodieHFileReader; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -45,6 +43,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.util.Pair; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.Nonnull; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -56,8 +58,6 @@ import java.util.TreeMap; import java.util.stream.Collectors; -import javax.annotation.Nonnull; - /** * HoodieHFileDataBlock contains a list of records stored inside an HFile format. It is used with the HFile * base file format. @@ -68,24 +68,23 @@ public class HoodieHFileDataBlock extends HoodieDataBlock { private static int blockSize = 1 * 1024 * 1024; private boolean enableInlineReading = false; - public HoodieHFileDataBlock(@Nonnull Map logBlockHeader, - @Nonnull Map logBlockFooter, - @Nonnull Option blockContentLocation, @Nonnull Option content, - FSDataInputStream inputStream, boolean readBlockLazily) { - super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily); - } - public HoodieHFileDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option content, - boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema, - Map header, Map footer, boolean enableInlineReading) { + boolean readBlockLazily, long position, long blockSize, long blockEndpos, + Schema readerSchema, Map header, + Map footer, boolean enableInlineReading, String keyField) { super(content, inputStream, readBlockLazily, - Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header, - footer); + Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), + readerSchema, header, footer, keyField); this.enableInlineReading = enableInlineReading; } + public HoodieHFileDataBlock(@Nonnull List records, @Nonnull Map header, + String keyField) { + super(records, header, new HashMap<>(), keyField); + } + public HoodieHFileDataBlock(@Nonnull List records, @Nonnull Map header) { - super(records, header, new HashMap<>()); + this(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); } @Override @@ -103,7 +102,7 @@ protected byte[] serializeRecords() throws IOException { FSDataOutputStream ostream = new FSDataOutputStream(baos, null); HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig) - .withOutputStream(ostream).withFileContext(context).create(); + .withOutputStream(ostream).withFileContext(context).withComparator(new HoodieHBaseKVComparator()).create(); // Serialize records into bytes Map sortedRecordsMap = new TreeMap<>(); @@ -111,7 +110,7 @@ protected byte[] serializeRecords() throws IOException { boolean useIntegerKey = false; int key = 0; int keySize = 0; - Field keyField = records.get(0).getSchema().getField(HoodieRecord.RECORD_KEY_METADATA_FIELD); + Field keyField = records.get(0).getSchema().getField(this.keyField); if (keyField == null) { // Missing key metadata field so we should use an integer sequence key useIntegerKey = true; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 37631b0253c0..0c7d97bc3988 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -36,7 +36,6 @@ import java.io.IOException; import java.io.Serializable; import java.text.ParseException; -import java.time.Instant; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -72,24 +71,17 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { protected HoodieTableMetaClient metaClient; /** - * Parse the timestamp of an Instant and return a {@code SimpleDateFormat}. + * Parse the timestamp of an Instant and return a {@code Date}. */ - public static Date parseInstantTime(String timestamp) throws ParseException { - return HoodieInstantTimeGenerator.parseInstantTime(timestamp); - } - - /** - * Format the java.time.Instant to a String representing the timestamp of a Hoodie Instant. - */ - public static String formatInstantTime(Instant timestamp) { - return HoodieInstantTimeGenerator.formatInstantTime(timestamp); + public static Date parseDateFromInstantTime(String timestamp) throws ParseException { + return HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp); } /** * Format the Date to a String representing the timestamp of a Hoodie Instant. */ - public static String formatInstantTime(Date timestamp) { - return HoodieInstantTimeGenerator.formatInstantTime(timestamp); + public static String formatDate(Date timestamp) { + return HoodieInstantTimeGenerator.formatDate(timestamp); } /** @@ -100,6 +92,7 @@ public static String createNewInstantTime() { return HoodieInstantTimeGenerator.createNewInstantTime(0); } + /** * Returns next instant time that adds N milliseconds to current time. * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity @@ -197,9 +190,8 @@ public static void deleteInstantFile(FileSystem fs, String metaPath, HoodieInsta } } - public void deletePendingIfExists(HoodieInstant.State state, String action, String instantStr) { - HoodieInstant instant = new HoodieInstant(state, action, instantStr); - ValidationUtils.checkArgument(!instant.isCompleted()); + public void deleteEmptyInstantIfExists(HoodieInstant instant) { + ValidationUtils.checkArgument(isEmpty(instant)); deleteInstantFileIfExists(instant); } @@ -389,14 +381,13 @@ public HoodieInstant transitionRollbackInflightToComplete(HoodieInstant inflight * Transition Rollback State from requested to inflight. * * @param requestedInstant requested instant - * @param data Optional data to be stored * @return commit instant */ - public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant, Option data) { + public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant) { ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); ValidationUtils.checkArgument(requestedInstant.isRequested()); HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, ROLLBACK_ACTION, requestedInstant.getTimestamp()); - transitionState(requestedInstant, inflight, data); + transitionState(requestedInstant, inflight, Option.empty()); return inflight; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 4926b2a55ffa..e82819e73e1d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -36,6 +36,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.Nonnull; + import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; @@ -51,7 +53,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; /** * Represents the Archived Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the @@ -66,7 +67,7 @@ */ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { private static final Pattern ARCHIVE_FILE_PATTERN = - Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$"); + Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$"); private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits"; private static final String ACTION_TYPE_KEY = "actionType"; @@ -113,6 +114,11 @@ public void loadInstantDetailsInMemory(String startTs, String endTs) { loadInstants(startTs, endTs); } + public void loadCompletedInstantDetailsInMemory() { + loadInstants(null, true, + record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString())); + } + public void loadCompactionDetailsInMemory(String compactionInstantTime) { loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime); } @@ -144,14 +150,17 @@ public HoodieArchivedTimeline reload() { } private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) { - final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString(); + final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString(); final String action = record.get(ACTION_TYPE_KEY).toString(); if (loadDetails) { - Option.ofNullable(record.get(getMetadataKey(action))).map(actionData -> { - if (action.equals(HoodieTimeline.COMPACTION_ACTION)) { - this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord)actionData)); - } else { - this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8)); + getMetadataKey(action).map(key -> { + Object actionData = record.get(key); + if (actionData != null) { + if (action.equals(HoodieTimeline.COMPACTION_ACTION)) { + this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData)); + } else { + this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8)); + } } return null; }); @@ -159,22 +168,25 @@ private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) { return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, instantTime); } - private String getMetadataKey(String action) { + @Nonnull + private Option getMetadataKey(String action) { switch (action) { case HoodieTimeline.CLEAN_ACTION: - return "hoodieCleanMetadata"; + return Option.of("hoodieCleanMetadata"); case HoodieTimeline.COMMIT_ACTION: - return "hoodieCommitMetadata"; case HoodieTimeline.DELTA_COMMIT_ACTION: - return "hoodieCommitMetadata"; + return Option.of("hoodieCommitMetadata"); case HoodieTimeline.ROLLBACK_ACTION: - return "hoodieRollbackMetadata"; + return Option.of("hoodieRollbackMetadata"); case HoodieTimeline.SAVEPOINT_ACTION: - return "hoodieSavePointMetadata"; + return Option.of("hoodieSavePointMetadata"); case HoodieTimeline.COMPACTION_ACTION: - return "hoodieCompactionPlan"; + return Option.of("hoodieCompactionPlan"); + case HoodieTimeline.REPLACE_COMMIT_ACTION: + return Option.of("hoodieReplaceCommitMetadata"); default: - throw new HoodieIOException("Unknown action in metadata " + action); + LOG.error(String.format("Unknown action in metadata (%s)", action)); + return Option.empty(); } } @@ -199,35 +211,33 @@ private List loadInstants(TimeRangeFilter filter, boolean loadIns private List loadInstants(TimeRangeFilter filter, boolean loadInstantDetails, Function commitsFilter) { try { - // list all files + // List all files FileStatus[] fsStatuses = metaClient.getFs().globStatus( new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - // sort files by version suffix in reverse (implies reverse chronological order) + // Sort files by version suffix in reverse (implies reverse chronological order) Arrays.sort(fsStatuses, new ArchiveFileVersionComparator()); List instantsInRange = new ArrayList<>(); for (FileStatus fs : fsStatuses) { - //read the archived file + // Read the archived file try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(), new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) { int instantsInPreviousFile = instantsInRange.size(); - //read the avro blocks + // Read the avro blocks while (reader.hasNext()) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); // TODO If we can store additional metadata in datablock, we can skip parsing records // (such as startTime, endTime of records in the block) List records = blk.getRecords(); - // filter blocks in desired time window - Stream instantsInBlkStream = records.stream() - .filter(r -> commitsFilter.apply((GenericRecord) r)) - .map(r -> readCommit((GenericRecord) r, loadInstantDetails)); - - if (filter != null) { - instantsInBlkStream = instantsInBlkStream.filter(filter::isInRange); - } - - instantsInRange.addAll(instantsInBlkStream.collect(Collectors.toList())); + // Filter blocks in desired time window + instantsInRange.addAll( + records.stream() + .filter(r -> commitsFilter.apply((GenericRecord) r)) + .map(r -> readCommit((GenericRecord) r, loadInstantDetails)) + .filter(c -> filter == null || filter.isInRange(c)) + .collect(Collectors.toList()) + ); } if (filter != null) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 552adfa8f349..15691f14f7af 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.table.timeline; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; @@ -100,6 +102,12 @@ public HoodieTimeline filterCompletedInstants() { return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details); } + @Override + public HoodieTimeline filterCompletedInstantsWithCommitMetadata() { + return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted) + .filter(i -> !isDeletePartitionType(i)), details); + } + @Override public HoodieTimeline filterCompletedAndCompactionInstants() { return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() @@ -115,7 +123,7 @@ public HoodieDefaultTimeline getWriteTimeline() { @Override public HoodieTimeline getCompletedReplaceTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(s -> s.isCompleted()), details); + instants.stream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(HoodieInstant::isCompleted), details); } @Override @@ -124,6 +132,12 @@ public HoodieTimeline filterPendingReplaceTimeline() { s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()), details); } + @Override + public HoodieTimeline filterPendingRollbackTimeline() { + return new HoodieDefaultTimeline(instants.stream().filter( + s -> s.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && !s.isCompleted()), details); + } + @Override public HoodieTimeline filterPendingCompactionTimeline() { return new HoodieDefaultTimeline( @@ -274,6 +288,12 @@ public Option firstInstant() { return Option.fromJavaOptional(instants.stream().findFirst()); } + @Override + public Option firstInstant(String action, State state) { + return Option.fromJavaOptional(instants.stream() + .filter(s -> action.equals(s.getAction()) && state.equals(s.getState())).findFirst()); + } + @Override public Option nthInstant(int n) { if (empty() || n >= countInstants()) { @@ -339,6 +359,26 @@ public Option getInstantDetails(HoodieInstant instant) { return details.apply(instant); } + @Override + public boolean isDeletePartitionType(HoodieInstant instant) { + Option operationType; + + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(getInstantDetails(instant).get(), HoodieCommitMetadata.class); + operationType = Option.of(commitMetadata.getOperationType()); + } catch (Exception e) { + operationType = Option.empty(); + } + + return operationType.isPresent() && WriteOperationType.DELETE_PARTITION.equals(operationType.get()); + } + + @Override + public boolean isEmpty(HoodieInstant instant) { + return getInstantDetails(instant).get().length == 0; + } + @Override public String toString() { return this.getClass().getName() + ": " + instants.stream().map(Object::toString).collect(Collectors.joining(",")); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index 817b39254ef0..9c0e33b9527b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -18,11 +18,15 @@ package org.apache.hudi.common.table.timeline; -import java.time.Instant; +import org.apache.hudi.common.model.HoodieTimelineTimeZone; +import java.text.ParseException; import java.time.LocalDateTime; import java.time.ZoneId; +import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; import java.time.format.DateTimeParseException; +import java.time.temporal.ChronoField; import java.time.temporal.TemporalAccessor; import java.util.Date; import java.util.concurrent.atomic.AtomicReference; @@ -32,13 +36,29 @@ */ public class HoodieInstantTimeGenerator { // Format of the timestamp used for an Instant - private static final String INSTANT_TIMESTAMP_FORMAT = "yyyyMMddHHmmss"; + public static final String SECS_INSTANT_TIMESTAMP_FORMAT = "yyyyMMddHHmmss"; + public static final int SECS_INSTANT_ID_LENGTH = SECS_INSTANT_TIMESTAMP_FORMAT.length(); + public static final String MILLIS_INSTANT_TIMESTAMP_FORMAT = "yyyyMMddHHmmssSSS"; + public static final int MILLIS_INSTANT_ID_LENGTH = MILLIS_INSTANT_TIMESTAMP_FORMAT.length(); + public static final int MILLIS_INSTANT_TIMESTAMP_FORMAT_LENGTH = MILLIS_INSTANT_TIMESTAMP_FORMAT.length(); // Formatter to generate Instant timestamps - private static DateTimeFormatter INSTANT_TIME_FORMATTER = DateTimeFormatter.ofPattern(INSTANT_TIMESTAMP_FORMAT); + // Unfortunately millisecond format is not parsable as is https://bugs.openjdk.java.net/browse/JDK-8031085. hence have to do appendValue() + private static DateTimeFormatter MILLIS_INSTANT_TIME_FORMATTER = new DateTimeFormatterBuilder().appendPattern(SECS_INSTANT_TIMESTAMP_FORMAT) + .appendValue(ChronoField.MILLI_OF_SECOND, 3).toFormatter(); + private static final String MILLIS_GRANULARITY_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS"; + private static DateTimeFormatter MILLIS_GRANULARITY_DATE_FORMATTER = DateTimeFormatter.ofPattern(MILLIS_GRANULARITY_DATE_FORMAT); + // The last Instant timestamp generated private static AtomicReference lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE)); private static final String ALL_ZERO_TIMESTAMP = "00000000000000"; + // The default number of milliseconds that we add if they are not present + // We prefer the max timestamp as it mimics the current behavior with second granularity + // when performing comparisons such as LESS_THAN_OR_EQUAL_TO + private static final String DEFAULT_MILLIS_EXT = "999"; + + private static HoodieTimelineTimeZone commitTimeZone = HoodieTimelineTimeZone.LOCAL; + /** * Returns next instant time that adds N milliseconds to the current time. * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity @@ -49,36 +69,74 @@ public static String createNewInstantTime(long milliseconds) { return lastInstantTime.updateAndGet((oldVal) -> { String newCommitTime; do { - Date d = new Date(System.currentTimeMillis() + milliseconds); - newCommitTime = INSTANT_TIME_FORMATTER.format(convertDateToTemporalAccessor(d)); + if (commitTimeZone.equals(HoodieTimelineTimeZone.UTC.toString())) { + LocalDateTime now = LocalDateTime.now(ZoneOffset.UTC); + newCommitTime = now.format(MILLIS_INSTANT_TIME_FORMATTER); + } else { + Date d = new Date(System.currentTimeMillis() + milliseconds); + newCommitTime = MILLIS_INSTANT_TIME_FORMATTER.format(convertDateToTemporalAccessor(d)); + } } while (HoodieTimeline.compareTimestamps(newCommitTime, HoodieActiveTimeline.LESSER_THAN_OR_EQUALS, oldVal)); return newCommitTime; }); } - public static Date parseInstantTime(String timestamp) { + public static Date parseDateFromInstantTime(String timestamp) throws ParseException { try { - LocalDateTime dt = LocalDateTime.parse(timestamp, INSTANT_TIME_FORMATTER); + // Enables backwards compatibility with non-millisecond granularity instants + String timestampInMillis = timestamp; + if (isSecondGranularity(timestamp)) { + // Add milliseconds to the instant in order to parse successfully + timestampInMillis = timestamp + DEFAULT_MILLIS_EXT; + } else if (timestamp.length() > MILLIS_INSTANT_TIMESTAMP_FORMAT_LENGTH) { + // compaction and cleaning in metadata has special format. handling it by trimming extra chars and treating it with ms granularity + timestampInMillis = timestamp.substring(0, MILLIS_INSTANT_TIMESTAMP_FORMAT_LENGTH); + } + + LocalDateTime dt = LocalDateTime.parse(timestampInMillis, MILLIS_INSTANT_TIME_FORMATTER); return Date.from(dt.atZone(ZoneId.systemDefault()).toInstant()); } catch (DateTimeParseException e) { // Special handling for all zero timestamp which is not parsable by DateTimeFormatter if (timestamp.equals(ALL_ZERO_TIMESTAMP)) { return new Date(0); } - throw e; } } - public static String formatInstantTime(Instant timestamp) { - return INSTANT_TIME_FORMATTER.format(timestamp); + private static boolean isSecondGranularity(String instant) { + return instant.length() == SECS_INSTANT_ID_LENGTH; + } + + public static String formatDate(Date timestamp) { + return getInstantFromTemporalAccessor(convertDateToTemporalAccessor(timestamp)); } - public static String formatInstantTime(Date timestamp) { - return INSTANT_TIME_FORMATTER.format(convertDateToTemporalAccessor(timestamp)); + public static String getInstantFromTemporalAccessor(TemporalAccessor temporalAccessor) { + return MILLIS_INSTANT_TIME_FORMATTER.format(temporalAccessor); + } + + /** + * Creates an instant string given a valid date-time string. + * @param dateString A date-time string in the format yyyy-MM-dd HH:mm:ss[:SSS] + * @return A timeline instant + * @throws ParseException If we cannot parse the date string + */ + public static String getInstantForDateString(String dateString) { + try { + return getInstantFromTemporalAccessor(LocalDateTime.parse(dateString, MILLIS_GRANULARITY_DATE_FORMATTER)); + } catch (Exception e) { + // Attempt to add the milliseconds in order to complete parsing + return getInstantFromTemporalAccessor(LocalDateTime.parse( + String.format("%s:%s", dateString, DEFAULT_MILLIS_EXT), MILLIS_GRANULARITY_DATE_FORMATTER)); + } } private static TemporalAccessor convertDateToTemporalAccessor(Date d) { return d.toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime(); } + + public static void setCommitTimeZone(HoodieTimelineTimeZone commitTimeZone) { + HoodieInstantTimeGenerator.commitTimeZone = commitTimeZone; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index b473c7b1fb4d..3b2779ca3141 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -131,6 +131,14 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterCompletedAndCompactionInstants(); + /** + * Filter this timeline to include the completed and exclude operation type is delete partition instants. + * + * @return New instance of HoodieTimeline with include the completed and + * exclude operation type is delete partition instants + */ + HoodieTimeline filterCompletedInstantsWithCommitMetadata(); + /** * Timeline to just include commits (commit/deltacommit), compaction and replace actions. * @@ -157,6 +165,11 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterPendingReplaceTimeline(); + /** + * Filter this timeline to include pending rollbacks. + */ + HoodieTimeline filterPendingRollbackTimeline(); + /** * Create a new Timeline with all the instants after startTs. */ @@ -209,6 +222,13 @@ public interface HoodieTimeline extends Serializable { */ Option firstInstant(); + /** + * @param action Instant action String. + * @param state Instant State. + * @return first instant of a specific action and state if available + */ + Option firstInstant(String action, State state); + /** * @return nth completed instant from the first completed instant */ @@ -269,6 +289,13 @@ public interface HoodieTimeline extends Serializable { */ Option getInstantDetails(HoodieInstant instant); + boolean isEmpty(HoodieInstant instant); + + /** + * Check WriteOperationType is DeletePartition. + */ + boolean isDeletePartitionType(HoodieInstant instant); + /** * Helper methods to compare instants. **/ @@ -322,6 +349,10 @@ static HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { return new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, timestamp); } + static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { + return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant); + } + /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names * between inflight and completed instants (compaction <=> commit). diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index eca3718f18a0..92e6171b6832 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -125,7 +125,7 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { /** * Adds the provided statuses into the file system view, and also caches it inside this object. */ - protected List addFilesToView(FileStatus[] statuses) { + public List addFilesToView(FileStatus[] statuses) { HoodieTimer timer = new HoodieTimer().startTimer(); List fileGroups = buildFileGroups(statuses, visibleCommitsAndCompactionTimeline, true); long fgBuildTimeTakenMs = timer.endTimer(); @@ -243,19 +243,24 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { + replacedFileGroups.size() + " replaced file groups"); } + @Override + public void close() { + try { + writeLock.lock(); + clear(); + } finally { + writeLock.unlock(); + } + } + /** * Clears the partition Map and reset view states. */ @Override - public final void reset() { + public void reset() { try { writeLock.lock(); - - addedPartitions.clear(); - resetViewState(); - - bootstrapIndex = null; - + clear(); // Initialize with new Hoodie timeline. init(metaClient, getTimeline()); } finally { @@ -263,6 +268,15 @@ public final void reset() { } } + /** + * Clear the resource. + */ + private void clear() { + addedPartitions.clear(); + resetViewState(); + bootstrapIndex = null; + } + /** * Allows all view metadata in file system view storage to be reset by subclasses. */ @@ -315,12 +329,17 @@ private void ensurePartitionLoadedCorrectly(String partition) { * @throws IOException */ protected FileStatus[] listPartition(Path partitionPath) throws IOException { - // Create the path if it does not exist already - if (!metaClient.getFs().exists(partitionPath)) { - metaClient.getFs().mkdirs(partitionPath); - return new FileStatus[0]; - } else { + try { return metaClient.getFs().listStatus(partitionPath); + } catch (IOException e) { + // Create the path if it does not exist already + if (!metaClient.getFs().exists(partitionPath)) { + metaClient.getFs().mkdirs(partitionPath); + return new FileStatus[0]; + } else { + // in case the partition path was created by another caller + return metaClient.getFs().listStatus(partitionPath); + } } } @@ -925,7 +944,7 @@ Stream fetchAllFileSlices(String partitionPath) { /** * Default implementation for fetching latest base-files for the partition-path. */ - Stream fetchLatestBaseFiles(final String partitionPath) { + public Stream fetchLatestBaseFiles(final String partitionPath) { return fetchAllStoredFileGroups(partitionPath) .map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg))) .filter(p -> p.getValue().isPresent()) @@ -1116,8 +1135,7 @@ public void sync() { */ protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { refreshTimeline(newTimeline); - addedPartitions.clear(); - resetViewState(); + clear(); // Initialize with new Hoodie timeline. init(metaClient, newTimeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 0bbbf3e9e530..b8f7f313a158 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -349,12 +349,12 @@ protected Option getReplaceInstant(final HoodieFileGroupId fileGr @Override public void close() { - closed = true; - super.reset(); + super.close(); partitionToFileGroupsMap = null; fgIdToPendingCompaction = null; fgIdToBootstrapBaseFile = null; fgIdToReplaceInstants = null; + closed = true; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index 3783d00b65d2..ff44c7cef017 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -32,6 +32,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.http.HttpStatus; +import org.apache.http.client.HttpResponseException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -65,7 +67,7 @@ private R execute(Function0 preferredFunction, Function0 secondaryFunc try { return preferredFunction.apply(); } catch (RuntimeException re) { - LOG.error("Got error running preferred function. Trying secondary", re); + handleRuntimeException(re); errorOnPreferredView = true; return secondaryFunction.apply(); } @@ -80,7 +82,7 @@ private R execute(T1 val, Function1 preferredFunction, Function1< try { return preferredFunction.apply(val); } catch (RuntimeException re) { - LOG.error("Got error running preferred function. Trying secondary", re); + handleRuntimeException(re); errorOnPreferredView = true; return secondaryFunction.apply(val); } @@ -96,7 +98,7 @@ private R execute(T1 val, T2 val2, Function2 preferredFun try { return preferredFunction.apply(val, val2); } catch (RuntimeException re) { - LOG.error("Got error running preferred function. Trying secondary", re); + handleRuntimeException(re); errorOnPreferredView = true; return secondaryFunction.apply(val, val2); } @@ -112,13 +114,21 @@ private R execute(T1 val, T2 val2, T3 val3, Function3 getLatestBaseFiles(String partitionPath) { return execute(partitionPath, preferredView::getLatestBaseFiles, secondaryView::getLatestBaseFiles); @@ -229,6 +239,7 @@ public void close() { public void reset() { preferredView.reset(); secondaryView.reset(); + errorOnPreferredView = false; } @Override @@ -243,8 +254,9 @@ public HoodieTimeline getTimeline() { @Override public void sync() { - preferredView.reset(); - secondaryView.reset(); + preferredView.sync(); + secondaryView.sync(); + errorOnPreferredView = false; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index 4c2153010207..099b79cbba0a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -126,7 +126,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, private final int serverPort; private final String basePath; private final HoodieTableMetaClient metaClient; - private final HoodieTimeline timeline; + private HoodieTimeline timeline; private final ObjectMapper mapper; private final int timeoutSecs; @@ -413,6 +413,8 @@ public Stream getAllReplacedFileGroups(String partitionPath) { public boolean refresh() { Map paramsMap = getParams(); try { + // refresh the local timeline first. + this.timeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); return executeRequest(REFRESH_TABLE, paramsMap, new TypeReference() {}, RequestMethod.POST); } catch (IOException e) { throw new HoodieRemoteException(e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/ZOrderingUtil.java b/hudi-common/src/main/java/org/apache/hudi/common/util/BinaryUtil.java similarity index 95% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/ZOrderingUtil.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/BinaryUtil.java index 3aa808075d33..0c7e89895767 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/ZOrderingUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/BinaryUtil.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.hudi.optimize; +package org.apache.hudi.common.util; import java.nio.charset.Charset; -public class ZOrderingUtil { +public class BinaryUtil { /** * Lexicographically compare two arrays. @@ -176,9 +176,14 @@ public static byte[] utf8To8Byte(String a) { public static Long convertStringToLong(String a) { byte[] bytes = utf8To8Byte(a); + return convertBytesToLong(bytes); + } + + public static long convertBytesToLong(byte[] bytes) { + byte[] paddedBytes = paddingTo8Byte(bytes); long temp = 0L; for (int i = 7; i >= 0; i--) { - temp = temp | (((long)bytes[i] & 0xff) << (7 - i) * 8); + temp = temp | (((long) paddedBytes[i] & 0xff) << (7 - i) * 8); } return temp; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index 0d790be84261..6687e583afca 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -124,7 +125,16 @@ public static Map getAllFileGroupsInPendingClu // get all filegroups in the plan getFileGroupEntriesInClusteringPlan(clusteringPlan.getLeft(), clusteringPlan.getRight())); - Map resultMap = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Map resultMap; + try { + resultMap = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } catch (Exception e) { + if (e instanceof IllegalStateException && e.getMessage().contains("Duplicate key")) { + throw new HoodieException("Found duplicate file groups pending clustering. If you're running deltastreamer in continuous mode, consider adding delay using --min-sync-interval-seconds. " + + "Or consider setting write concurrency mode to optimistic_concurrency_control.", e); + } + throw new HoodieException("Error getting all file groups in pending clustering", e); + } LOG.info("Found " + resultMap.size() + " files in pending clustering operations"); return resultMap; } @@ -166,22 +176,20 @@ public static HoodieClusteringPlan createClusteringPlan(String strategyClassName .setStrategyClassName(strategyClassName).setStrategyParams(strategyParams) .build(); - HoodieClusteringPlan plan = HoodieClusteringPlan.newBuilder() + return HoodieClusteringPlan.newBuilder() .setInputGroups(clusteringGroups) .setExtraMetadata(extraMetadata) .setStrategy(strategy) .build(); - - return plan; } private static List getFileSliceInfo(List slices) { - return slices.stream().map(slice -> new HoodieSliceInfo().newBuilder() + return slices.stream().map(slice -> HoodieSliceInfo.newBuilder() .setPartitionPath(slice.getPartitionPath()) .setFileId(slice.getFileId()) .setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(null)) .setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().getName()).collect(Collectors.toList())) - .setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(null)).orElse(null)) + .setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(BaseFile::getPath).orElse(null)).orElse(null)) .build()).collect(Collectors.toList()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index 265a601ebc68..cbcdbc4040a4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -18,15 +18,17 @@ package org.apache.hudi.common.util; -import java.util.Properties; import org.apache.hudi.common.util.collection.Pair; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -35,6 +37,36 @@ public class CollectionUtils { public static final Properties EMPTY_PROPERTIES = new Properties(); + /** + * Combines provided {@link List}s into one + */ + public static List combine(List one, List another) { + ArrayList combined = new ArrayList<>(one); + combined.addAll(another); + return combined; + } + + /** + * Returns difference b/w {@code one} {@link Set} of elements and {@code another} + */ + public static Set diff(Set one, Set another) { + Set diff = new HashSet<>(one); + diff.removeAll(another); + return diff; + } + + /** + * Returns difference b/w {@code one} {@link List} of elements and {@code another} + * + * NOTE: This is less optimal counterpart to {@link #diff(Set, Set)}, accepting {@link List} + * as a holding collection to support duplicate elements use-cases + */ + public static List diff(List one, List another) { + List diff = new ArrayList<>(one); + diff.removeAll(another); + return diff; + } + /** * Determines whether two iterators contain equal elements in the same order. More specifically, * this method returns {@code true} if {@code iterator1} and {@code iterator2} contain the same diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java index 1c18a77d6087..e52e5660957d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java @@ -19,14 +19,26 @@ package org.apache.hudi.common.util; +import java.time.Duration; import java.time.Instant; import java.time.format.DateTimeParseException; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; public class DateTimeUtils { + private static final Map LABEL_TO_UNIT_MAP = + Collections.unmodifiableMap(initMap()); /** * Parse input String to a {@link java.time.Instant}. + * * @param s Input String should be Epoch time in millisecond or ISO-8601 format. */ public static Instant parseDateTime(String s) throws DateTimeParseException { @@ -37,4 +49,141 @@ public static Instant parseDateTime(String s) throws DateTimeParseException { return Instant.parse(s); } } + + /** + * Parse the given string to a java {@link Duration}. The string is in format "{length + * value}{time unit label}", e.g. "123ms", "321 s". If no time unit label is specified, it will + * be considered as milliseconds. + * + *

Supported time unit labels are: + * + *

    + *
  • DAYS: "d", "day" + *
  • HOURS: "h", "hour" + *
  • MINUTES: "min", "minute" + *
  • SECONDS: "s", "sec", "second" + *
  • MILLISECONDS: "ms", "milli", "millisecond" + *
  • MICROSECONDS: "µs", "micro", "microsecond" + *
  • NANOSECONDS: "ns", "nano", "nanosecond" + *
+ * + * @param text string to parse. + */ + public static Duration parseDuration(String text) { + ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(text)); + + final String trimmed = text.trim(); + ValidationUtils.checkArgument(!trimmed.isEmpty(), "argument is an empty- or whitespace-only string"); + + final int len = trimmed.length(); + int pos = 0; + + char current; + while (pos < len && (current = trimmed.charAt(pos)) >= '0' && current <= '9') { + pos++; + } + + final String number = trimmed.substring(0, pos); + final String unitLabel = trimmed.substring(pos).trim().toLowerCase(Locale.US); + + if (number.isEmpty()) { + throw new NumberFormatException("text does not start with a number"); + } + + final long value; + try { + value = Long.parseLong(number); // this throws a NumberFormatException on overflow + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "The value '" + + number + + "' cannot be re represented as 64bit number (numeric overflow)."); + } + + if (unitLabel.isEmpty()) { + return Duration.of(value, ChronoUnit.MILLIS); + } + + ChronoUnit unit = LABEL_TO_UNIT_MAP.get(unitLabel); + if (unit != null) { + return Duration.of(value, unit); + } else { + throw new IllegalArgumentException( + "Time interval unit label '" + + unitLabel + + "' does not match any of the recognized units: " + + TimeUnit.getAllUnits()); + } + } + + private static Map initMap() { + Map labelToUnit = new HashMap<>(); + for (TimeUnit timeUnit : TimeUnit.values()) { + for (String label : timeUnit.getLabels()) { + labelToUnit.put(label, timeUnit.getUnit()); + } + } + return labelToUnit; + } + + /** + * Enum which defines time unit, mostly used to parse value from configuration file. + */ + private enum TimeUnit { + DAYS(ChronoUnit.DAYS, singular("d"), plural("day")), + HOURS(ChronoUnit.HOURS, singular("h"), plural("hour")), + MINUTES(ChronoUnit.MINUTES, singular("min"), plural("minute")), + SECONDS(ChronoUnit.SECONDS, singular("s"), plural("sec"), plural("second")), + MILLISECONDS(ChronoUnit.MILLIS, singular("ms"), plural("milli"), plural("millisecond")), + MICROSECONDS(ChronoUnit.MICROS, singular("µs"), plural("micro"), plural("microsecond")), + NANOSECONDS(ChronoUnit.NANOS, singular("ns"), plural("nano"), plural("nanosecond")); + + private static final String PLURAL_SUFFIX = "s"; + + private final List labels; + + private final ChronoUnit unit; + + TimeUnit(ChronoUnit unit, String[]... labels) { + this.unit = unit; + this.labels = + Arrays.stream(labels) + .flatMap(Arrays::stream) + .collect(Collectors.toList()); + } + + /** + * @param label the original label + * @return the singular format of the original label + */ + private static String[] singular(String label) { + return new String[] {label}; + } + + /** + * @param label the original label + * @return both the singular format and plural format of the original label + */ + private static String[] plural(String label) { + return new String[] {label, label + PLURAL_SUFFIX}; + } + + public List getLabels() { + return labels; + } + + public ChronoUnit getUnit() { + return unit; + } + + public static String getAllUnits() { + return Arrays.stream(TimeUnit.values()) + .map(TimeUnit::createTimeUnitString) + .collect(Collectors.joining(", ")); + } + + private static String createTimeUnitString(TimeUnit timeUnit) { + return timeUnit.name() + ": (" + String.join(" | ", timeUnit.getLabels()) + ")"; + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java index cb3f103a9c87..8fc43ef1cfa2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.util; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -98,6 +103,31 @@ public static void copy(InputStream inputStream, OutputStream outputStream) thro } } + /** + * Copies the file content from source path to destination path. + * + * @param fileSystem {@link FileSystem} instance. + * @param sourceFilePath Source file path. + * @param destFilePath Destination file path. + */ + public static void copy( + FileSystem fileSystem, org.apache.hadoop.fs.Path sourceFilePath, + org.apache.hadoop.fs.Path destFilePath) { + FSDataInputStream fsDataInputStream = null; + FSDataOutputStream fsDataOutputStream = null; + try { + fsDataInputStream = fileSystem.open(sourceFilePath); + fsDataOutputStream = fileSystem.create(destFilePath, false); + copy(fsDataInputStream, fsDataOutputStream); + } catch (IOException e) { + throw new HoodieIOException(String.format("Cannot copy from %s to %s", + sourceFilePath.toString(), destFilePath.toString()), e); + } finally { + closeQuietly(fsDataInputStream); + closeQuietly(fsDataOutputStream); + } + } + public static byte[] readAsByteArray(InputStream input) throws IOException { return readAsByteArray(input, 128); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FutureUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FutureUtils.java new file mode 100644 index 000000000000..b0029917eebd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FutureUtils.java @@ -0,0 +1,46 @@ +/* + * 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 org.apache.hudi.common.util; + +import javax.annotation.Nonnull; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * A utility class for future operation. + */ +public class FutureUtils { + + /** + * Parallel CompletableFutures + * + * @param futures CompletableFuture list + * @return a new CompletableFuture which will completed when all of the given CompletableFutures complete. + */ + public static CompletableFuture> allOf(@Nonnull List> futures) { + return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])) + .thenApply(aVoid -> + futures.stream() + // NOTE: This join wouldn't block, since all the + // futures are completed at this point. + .map(CompletableFuture::join) + .collect(Collectors.toList())); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java index d6a56fe39ca9..1e320a552f02 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java @@ -21,7 +21,8 @@ import org.apache.hudi.exception.HoodieException; import java.io.IOException; -import java.net.ServerSocket; +import java.net.DatagramSocket; +import java.net.InetAddress; /** * A utility class for network. @@ -29,20 +30,13 @@ public class NetworkUtils { public static synchronized String getHostname() { - ServerSocket s = null; - try { - s = new ServerSocket(0); - return s.getInetAddress().getHostAddress(); + try (DatagramSocket s = new DatagramSocket()) { + // see https://stackoverflow.com/questions/9481865/getting-the-ip-address-of-the-current-machine-using-java + // for details. + s.connect(InetAddress.getByName("8.8.8.8"), 10002); + return s.getLocalAddress().getHostAddress(); } catch (IOException e) { throw new HoodieException("Unable to find server port", e); - } finally { - if (null != s) { - try { - s.close(); - } catch (IOException e) { - throw new HoodieException("Unable to close server port", e); - } - } } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index c142e8a9608b..136206150cbb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -18,6 +18,10 @@ package org.apache.hudi.common.util; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; @@ -26,11 +30,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.MetadataNotFoundException; import org.apache.hudi.keygen.BaseKeyGenerator; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.avro.AvroSchemaConverter; @@ -38,11 +37,15 @@ import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.DecimalMetadata; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import javax.annotation.Nonnull; import java.io.IOException; +import java.math.BigDecimal; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -50,6 +53,7 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Utility functions involving with parquet. @@ -284,55 +288,96 @@ public Boolean apply(String recordKey) { /** * Parse min/max statistics stored in parquet footers for all columns. */ - public Collection> readRangeFromParquetMetadata(Configuration conf, Path parquetFilePath, List cols) { + public List> readRangeFromParquetMetadata( + @Nonnull Configuration conf, + @Nonnull Path parquetFilePath, + @Nonnull List cols + ) { ParquetMetadata metadata = readMetadata(conf, parquetFilePath); - // collect stats from all parquet blocks - Map>> columnToStatsListMap = metadata.getBlocks().stream().flatMap(blockMetaData -> { - return blockMetaData.getColumns().stream().filter(f -> cols.contains(f.getPath().toDotString())).map(columnChunkMetaData -> - new HoodieColumnRangeMetadata<>(parquetFilePath.getName(), columnChunkMetaData.getPath().toDotString(), - columnChunkMetaData.getStatistics().genericGetMin(), - columnChunkMetaData.getStatistics().genericGetMax(), - columnChunkMetaData.getStatistics().getNumNulls(), - columnChunkMetaData.getPrimitiveType().stringifier())); - }).collect(Collectors.groupingBy(e -> e.getColumnName())); - - // we only intend to keep file level statistics. - return new ArrayList<>(columnToStatsListMap.values().stream() - .map(blocks -> getColumnRangeInFile(blocks)) - .collect(Collectors.toList())); + // Collect stats from all individual Parquet blocks + Map>> columnToStatsListMap = metadata.getBlocks().stream().sequential() + .flatMap(blockMetaData -> blockMetaData.getColumns().stream() + .filter(f -> cols.contains(f.getPath().toDotString())) + .map(columnChunkMetaData -> + new HoodieColumnRangeMetadata( + parquetFilePath.getName(), + columnChunkMetaData.getPath().toDotString(), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + columnChunkMetaData.getStatistics().genericGetMin()), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + columnChunkMetaData.getStatistics().genericGetMax()), + columnChunkMetaData.getStatistics().getNumNulls(), + columnChunkMetaData.getPrimitiveType().stringifier())) + ).collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName)); + + // Combine those into file-level statistics + // NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer + // expression type correctly) + Stream> stream = columnToStatsListMap.values() + .stream() + .map(this::getColumnRangeInFile); + + return stream.collect(Collectors.toList()); } - private HoodieColumnRangeMetadata getColumnRangeInFile(final List> blockRanges) { + private > HoodieColumnRangeMetadata getColumnRangeInFile( + @Nonnull List> blockRanges + ) { if (blockRanges.size() == 1) { // only one block in parquet file. we can just return that range. return blockRanges.get(0); - } else { - // there are multiple blocks. Compute min(block_mins) and max(block_maxs) - return blockRanges.stream().reduce((b1, b2) -> combineRanges(b1, b2)).get(); } + + // there are multiple blocks. Compute min(block_mins) and max(block_maxs) + return blockRanges.stream() + .sequential() + .reduce(this::combineRanges).get(); } - private HoodieColumnRangeMetadata combineRanges(HoodieColumnRangeMetadata range1, - HoodieColumnRangeMetadata range2) { - final Comparable minValue; - final Comparable maxValue; - if (range1.getMinValue() != null && range2.getMinValue() != null) { - minValue = range1.getMinValue().compareTo(range2.getMinValue()) < 0 ? range1.getMinValue() : range2.getMinValue(); - } else if (range1.getMinValue() == null) { - minValue = range2.getMinValue(); + private > HoodieColumnRangeMetadata combineRanges( + HoodieColumnRangeMetadata one, + HoodieColumnRangeMetadata another + ) { + final T minValue; + final T maxValue; + if (one.getMinValue() != null && another.getMinValue() != null) { + minValue = one.getMinValue().compareTo(another.getMinValue()) < 0 ? one.getMinValue() : another.getMinValue(); + } else if (one.getMinValue() == null) { + minValue = another.getMinValue(); } else { - minValue = range1.getMinValue(); + minValue = one.getMinValue(); } - if (range1.getMaxValue() != null && range2.getMaxValue() != null) { - maxValue = range1.getMaxValue().compareTo(range2.getMaxValue()) < 0 ? range2.getMaxValue() : range1.getMaxValue(); - } else if (range1.getMaxValue() == null) { - maxValue = range2.getMaxValue(); + if (one.getMaxValue() != null && another.getMaxValue() != null) { + maxValue = one.getMaxValue().compareTo(another.getMaxValue()) < 0 ? another.getMaxValue() : one.getMaxValue(); + } else if (one.getMaxValue() == null) { + maxValue = another.getMaxValue(); } else { - maxValue = range1.getMaxValue(); + maxValue = one.getMaxValue(); + } + + return new HoodieColumnRangeMetadata( + one.getFilePath(), + one.getColumnName(), minValue, maxValue, one.getNumNulls() + another.getNumNulls(), one.getStringifier()); + } + + private static Comparable convertToNativeJavaType(PrimitiveType primitiveType, Comparable val) { + if (primitiveType.getOriginalType() == OriginalType.DECIMAL) { + DecimalMetadata decimalMetadata = primitiveType.getDecimalMetadata(); + return BigDecimal.valueOf((Integer) val, decimalMetadata.getScale()); + } else if (primitiveType.getOriginalType() == OriginalType.DATE) { + // NOTE: This is a workaround to address race-condition in using + // {@code SimpleDataFormat} concurrently (w/in {@code DateStringifier}) + // TODO cleanup after Parquet upgrade to 1.12 + synchronized (primitiveType.stringifier()) { + return java.sql.Date.valueOf( + primitiveType.stringifier().stringify((Integer) val) + ); + } } - return new HoodieColumnRangeMetadata<>(range1.getFilePath(), - range1.getColumnName(), minValue, maxValue, range1.getNumNulls() + range2.getNumNulls(), range1.getStringifier()); + return val; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java index 5dd0c5a8bc9c..934b5b5f616c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java @@ -115,22 +115,38 @@ public static long computePayloadSize(R value, SizeEstimator valueSizeEst * Utility method to convert bytes to HoodieRecord using schema and payload class. */ public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) { - return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), withOperationField); + return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, Option.empty()); + } + + public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + String preCombineField, + boolean withOperationField, + Option partitionName) { + return convertToHoodieRecordPayload(record, payloadClazz, preCombineField, + Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD), + withOperationField, partitionName); } /** * Utility method to convert bytes to HoodieRecord using schema and payload class. */ - public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, - String preCombineField, Pair recordKeyPartitionPathPair, - boolean withOperationField) { - String recKey = rec.get(recordKeyPartitionPathPair.getLeft()).toString(); - String partitionPath = rec.get(recordKeyPartitionPathPair.getRight()).toString(); - Object preCombineVal = getPreCombineVal(rec, preCombineField); + public static R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz, + String preCombineField, + Pair recordKeyPartitionPathFieldPair, + boolean withOperationField, + Option partitionName) { + final String recKey = record.get(recordKeyPartitionPathFieldPair.getKey()).toString(); + final String partitionPath = (partitionName.isPresent() ? partitionName.get() : + record.get(recordKeyPartitionPathFieldPair.getRight()).toString()); + + Object preCombineVal = getPreCombineVal(record, preCombineField); HoodieOperation operation = withOperationField - ? HoodieOperation.fromName(getNullableValAsString(rec, HoodieRecord.OPERATION_METADATA_FIELD)) : null; + ? HoodieOperation.fromName(getNullableValAsString(record, HoodieRecord.OPERATION_METADATA_FIELD)) : null; HoodieRecord hoodieRecord = new HoodieRecord<>(new HoodieKey(recKey, partitionPath), - ReflectionUtils.loadPayload(payloadClazz, new Object[] {rec, preCombineVal}, GenericRecord.class, Comparable.class), operation); + ReflectionUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class, + Comparable.class), operation); return (R) hoodieRecord; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/TypeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/TypeUtils.java new file mode 100644 index 000000000000..d713b183a654 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/TypeUtils.java @@ -0,0 +1,42 @@ +/* + * 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 org.apache.hudi.common.util; + +import javax.annotation.Nonnull; +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public final class TypeUtils { + + private TypeUtils() {} + + /** + * Maps values from the provided Enum's {@link Class} into corresponding values, + * extracted by provided {@code valueMapper} + */ + public static > Map getValueToEnumMap( + @Nonnull Class klass, + @Nonnull Function valueMapper + ) { + return Arrays.stream(klass.getEnumConstants()) + .collect(Collectors.toMap(valueMapper, Function.identity())); + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java index d31b0aaa6c30..a6e8d5cfb35c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java @@ -208,7 +208,8 @@ public R put(T key, R value) { // Note, the converter may over estimate the size of a record in the JVM this.estimatedPayloadSize = keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); LOG.info("Estimated Payload size => " + estimatedPayloadSize); - } else if (shouldEstimatePayloadSize && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + } else if (shouldEstimatePayloadSize && !inMemoryMap.isEmpty() + && (inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0)) { // Re-estimate the size of a record by calculating the size of the entire map containing // N entries and then dividing by the number of entries present (N). This helps to get a // correct estimation of the size of each record in the JVM. diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/ExceptionUtil.java b/hudi-common/src/main/java/org/apache/hudi/exception/ExceptionUtil.java new file mode 100644 index 000000000000..a0550ba9eaf6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/ExceptionUtil.java @@ -0,0 +1,42 @@ +/* + * 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 org.apache.hudi.exception; + +import javax.annotation.Nonnull; + +/** + * Class collecting common utilities helping in handling {@link Exception}s + */ +public final class ExceptionUtil { + private ExceptionUtil() {} + + /** + * Fetches inner-most cause of the provided {@link Throwable} + */ + @Nonnull + public static Throwable getRootCause(@Nonnull Throwable t) { + Throwable cause = t; + while (cause.getCause() != null) { + cause = cause.getCause(); + } + + return cause; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java new file mode 100644 index 000000000000..2d4d96959e15 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java @@ -0,0 +1,29 @@ +/* + * 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 org.apache.hudi.io.storage; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * This class is explicitly used as Key Comparator to work around the hard coded + * legacy format class names inside HBase. Otherwise, we will face issues with shading. + */ +public class HoodieHBaseKVComparator extends KeyValue.KVComparator { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 7b80d1a58597..e3e38eca86ca 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -246,7 +246,7 @@ public Option getRecordByKey(String key, Schema readerSchema) throws IOException synchronized (this) { if (keyScanner == null) { - keyScanner = reader.getScanner(false, true); + keyScanner = reader.getScanner(false, false); } if (keyScanner.seekTo(kv) == 0) { diff --git a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java index 3e369a54b970..d8535a21450c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java +++ b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java @@ -52,7 +52,7 @@ public class KeyGeneratorOptions extends HoodieConfig { public static final ConfigProperty PARTITIONPATH_FIELD_NAME = ConfigProperty .key("hoodie.datasource.write.partitionpath.field") - .defaultValue("partitionpath") + .noDefaultValue() .withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. " + "Actual value ontained by invoking .toString()"); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index b560b7694132..ccd421e67765 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -156,11 +156,7 @@ protected List fetchAllPartitionPaths() throws IOException { List partitions = Collections.emptyList(); if (hoodieRecord.isPresent()) { - if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { - throw new HoodieMetadataException("Metadata partition list record is inconsistent: " - + hoodieRecord.get().getData()); - } - + mayBeHandleSpuriousDeletes(hoodieRecord, "\"all partitions\""); partitions = hoodieRecord.get().getData().getFilenames(); // Partition-less tables have a single empty partition if (partitions.contains(NON_PARTITIONED_NAME)) { @@ -190,10 +186,7 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { FileStatus[] statuses = {}; if (hoodieRecord.isPresent()) { - if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { - throw new HoodieMetadataException("Metadata record for partition " + partitionName + " is inconsistent: " - + hoodieRecord.get().getData()); - } + mayBeHandleSpuriousDeletes(hoodieRecord, partitionName); statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath); } @@ -228,10 +221,7 @@ Map fetchAllFilesInPartitionPaths(List partitionPath for (Pair>> entry: partitionsFileStatus) { if (entry.getValue().isPresent()) { - if (!entry.getValue().get().getData().getDeletions().isEmpty()) { - throw new HoodieMetadataException("Metadata record for partition " + entry.getKey() + " is inconsistent: " - + entry.getValue().get().getData()); - } + mayBeHandleSpuriousDeletes(entry.getValue(), entry.getKey()); result.put(partitionInfo.get(entry.getKey()).toString(), entry.getValue().get().getData().getFileStatuses(hadoopConf.get(), partitionInfo.get(entry.getKey()))); } } @@ -240,6 +230,23 @@ Map fetchAllFilesInPartitionPaths(List partitionPath return result; } + /** + * May be handle spurious deletes. Depending on config, throw an exception or log a warn msg. + * @param hoodieRecord instance of {@link HoodieRecord} of interest. + * @param partitionName partition name of interest. + */ + private void mayBeHandleSpuriousDeletes(Option> hoodieRecord, String partitionName) { + if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { + if (!metadataConfig.ignoreSpuriousDeletes()) { + throw new HoodieMetadataException("Metadata record for " + partitionName + " is inconsistent: " + + hoodieRecord.get().getData()); + } else { + LOG.warn("Metadata record for " + partitionName + " encountered some files to be deleted which was not added before. " + + "Ignoring the spurious deletes as the `" + HoodieMetadataConfig.IGNORE_SPURIOUS_DELETES.key() + "` config is set to false"); + } + } + } + protected abstract Option> getRecordByKey(String key, String partitionName); protected abstract List>>> getRecordsByKeys(List key, String partitionName); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index f5e14ba1dd34..759e0f1a3e43 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -139,4 +139,9 @@ public Option getLatestCompactionTime() { public void close() throws Exception { // no-op } + + @Override + public void reset() { + // no-op + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 1ee8a78e3e2c..c9e538f72eaa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -121,7 +121,8 @@ private void initIfNeeded() { @Override protected Option> getRecordByKey(String key, String partitionName) { - return getRecordsByKeys(Collections.singletonList(key), partitionName).get(0).getValue(); + List>>> recordsByKeys = getRecordsByKeys(Collections.singletonList(key), partitionName); + return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue(); } protected List>>> getRecordsByKeys(List keys, String partitionName) { @@ -131,10 +132,14 @@ protected List>>> getRec HoodieFileReader baseFileReader = readers.getKey(); HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight(); + if (baseFileReader == null && logRecordScanner == null) { + return Collections.emptyList(); + } + // local map to assist in merging with base file records Map>> logRecords = readLogRecords(logRecordScanner, keys, timings); - List>>> result = readFromBaseAndMergeWithLogRecords(baseFileReader, - keys, logRecords, timings); + List>>> result = readFromBaseAndMergeWithLogRecords( + baseFileReader, keys, logRecords, timings, partitionName); LOG.info(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", keys.size(), timings)); return result; } catch (IOException ioe) { @@ -175,8 +180,8 @@ private Map>> readLogRecords( } private List>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader, - List keys, Map>> logRecords, - List timings) throws IOException { + List keys, Map>> logRecords, List timings, String partitionName) throws IOException { List>>> result = new ArrayList<>(); // merge with base records HoodieTimer timer = new HoodieTimer().startTimer(); @@ -184,14 +189,12 @@ private List>>> readFrom HoodieRecord hoodieRecord = null; // Retrieve record from base file if (baseFileReader != null) { - HoodieTimer readTimer = new HoodieTimer().startTimer(); + HoodieTimer readTimer = new HoodieTimer(); for (String key : keys) { + readTimer.startTimer(); Option baseRecord = baseFileReader.getRecordByKey(key); if (baseRecord.isPresent()) { - hoodieRecord = metadataTableConfig.populateMetaFields() - ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false) - : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), - Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()), false); + hoodieRecord = getRecord(baseRecord, partitionName); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); // merge base file record w/ log record if present if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) { @@ -217,6 +220,18 @@ private List>>> readFrom return result; } + private HoodieRecord getRecord(Option baseRecord, String partitionName) { + ValidationUtils.checkState(baseRecord.isPresent()); + if (metadataTableConfig.populateMetaFields()) { + return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false); + } + return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), + Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()), + false, Option.of(partitionName)); + } + /** * Returns a new pair of readers to the base and log files. */ @@ -230,7 +245,11 @@ private Pair openReadersI // Metadata is in sync till the latest completed instant on the dataset HoodieTimer timer = new HoodieTimer().startTimer(); - List latestFileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName); + List latestFileSlices = HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName); + if (latestFileSlices.size() == 0) { + // empty partition + return Pair.of(null, null); + } ValidationUtils.checkArgument(latestFileSlices.size() == 1, String.format("Invalid number of file slices: found=%d, required=%d", latestFileSlices.size(), 1)); final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, latestFileSlices.size())); @@ -240,7 +259,8 @@ private Pair openReadersI baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); // Open the log record scanner using the log files from the latest file slice - Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice); + Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice, + partitionName); logRecordScanner = logRecordScannerOpenTimePair.getKey(); logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); @@ -292,7 +312,7 @@ private Set getValidInstantTimestamps() { return validInstantTimestamps; } - private Pair getLogRecordScanner(FileSlice slice) { + private Pair getLogRecordScanner(FileSlice slice, String partitionName) { HoodieTimer timer = new HoodieTimer().startTimer(); List logFilePaths = slice.getLogFiles() .sorted(HoodieLogFile.getLogFileComparator()) @@ -322,6 +342,7 @@ private Pair getLogRecordScanner(File .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) .withLogBlockTimestamps(validInstantTimestamps) .enableFullScan(metadataConfig.enableFullScan()) + .withPartition(partitionName) .build(); Long logScannerOpenMs = timer.endTimer(); @@ -420,4 +441,10 @@ public Option getLatestCompactionTime() { } return Option.empty(); } + + @Override + public void reset() { + initIfNeeded(); + dataMetaClient.reloadActiveTimeline(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java index 453ec8f15ff0..d3b569ceb623 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java @@ -65,6 +65,18 @@ protected FileStatus[] listPartition(Path partitionPath) throws IOException { return tableMetadata.getAllFilesInPartition(partitionPath); } + @Override + public void reset() { + super.reset(); + tableMetadata.reset(); + } + + @Override + public void sync() { + super.sync(); + tableMetadata.reset(); + } + @Override public void close() { try { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java index 131ca3b91762..01c8d05e9b22 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java @@ -25,7 +25,13 @@ import java.util.Set; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -49,13 +55,17 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc // Set of all record keys that are to be read in memory private Set mergeKeyFilter; - private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, + private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, String partitionName, + List logFilePaths, + Schema readerSchema, String latestInstantTime, + Long maxMemorySizeInBytes, int bufferSize, String spillableMapBasePath, Set mergeKeyFilter, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, + ExternalSpillableMap.DiskMapType diskMapType, + boolean isBitCaskDiskMapCompressionEnabled, Option instantRange, boolean enableFullScan) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, - spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false, enableFullScan); + spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false, + enableFullScan, Option.of(partitionName)); this.mergeKeyFilter = mergeKeyFilter; if (enableFullScan) { performScan(); @@ -76,6 +86,23 @@ protected void processNextDeletedKey(HoodieKey hoodieKey) { } } + @Override + protected HoodieRecord createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig, + final String payloadClassFQN, final String preCombineField, + final boolean withOperationField, + final Option> simpleKeyGenFields, + final Option partitionName) { + if (hoodieTableConfig.populateMetaFields()) { + return super.createHoodieRecord(rec, hoodieTableConfig, payloadClassFQN, preCombineField, withOperationField, + simpleKeyGenFields, partitionName); + } + + // When meta fields are not available, create the record using the + // preset key field and the known partition name + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN, + preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName); + } + /** * Returns the builder for {@code HoodieMetadataMergedLogRecordScanner}. */ @@ -93,7 +120,10 @@ public List>>> getRecord return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key)))); } - public List>>> getRecordsByKeys(List keys) { + public synchronized List>>> getRecordsByKeys(List keys) { + // Following operations have to be atomic, otherwise concurrent + // readers would race with each other and could crash when + // processing log block records as part of scan. records.clear(); scan(Option.of(keys)); List>>> metadataRecords = new ArrayList<>(); @@ -107,12 +137,17 @@ public List>>> getRecord return metadataRecords; } + @Override + protected String getKeyField() { + return HoodieMetadataPayload.SCHEMA_FIELD_ID_KEY; + } + /** * Builder used to build {@code HoodieMetadataMergedLogRecordScanner}. */ public static class Builder extends HoodieMergedLogRecordScanner.Builder { private Set mergeKeyFilter = Collections.emptySet(); - private boolean enableFullScan; + private boolean enableFullScan = HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.defaultValue(); private boolean enableInlineReading; @Override @@ -161,6 +196,12 @@ public Builder withBufferSize(int bufferSize) { return this; } + @Override + public Builder withPartition(String partitionName) { + this.partitionName = partitionName; + return this; + } + @Override public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) { this.maxMemorySizeInBytes = maxMemorySizeInBytes; @@ -202,7 +243,7 @@ public Builder enableFullScan(boolean enableFullScan) { @Override public HoodieMetadataMergedLogRecordReader build() { - return new HoodieMetadataMergedLogRecordReader(fs, basePath, logFilePaths, readerSchema, + return new HoodieMetadataMergedLogRecordReader(fs, basePath, partitionName, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter, diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, enableFullScan); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 16eef8a5fd85..0b0d144a6e7e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -61,6 +61,12 @@ * HoodieMetadataRecord for ease of operations. */ public class HoodieMetadataPayload implements HoodieRecordPayload { + + // HoodieMetadata schema field ids + public static final String SCHEMA_FIELD_ID_KEY = "key"; + public static final String SCHEMA_FIELD_ID_TYPE = "type"; + public static final String SCHEMA_FIELD_ID_METADATA = "filesystemMetadata"; + // Type of the record // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810 private static final int PARTITION_LIST = 1; @@ -78,13 +84,13 @@ public HoodieMetadataPayload(Option record) { if (record.isPresent()) { // This can be simplified using SpecificData.deepcopy once this bug is fixed // https://issues.apache.org/jira/browse/AVRO-1811 - key = record.get().get("key").toString(); - type = (int) record.get().get("type"); - if (record.get().get("filesystemMetadata") != null) { + key = record.get().get(SCHEMA_FIELD_ID_KEY).toString(); + type = (int) record.get().get(SCHEMA_FIELD_ID_TYPE); + if (record.get().get(SCHEMA_FIELD_ID_METADATA) != null) { filesystemMetadata = (Map) record.get().get("filesystemMetadata"); filesystemMetadata.keySet().forEach(k -> { GenericRecord v = filesystemMetadata.get(k); - filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted"))); + filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); }); } } @@ -231,8 +237,8 @@ private Map combineFilesystemMetadata(HoodieMeta @Override public String toString() { final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); - sb.append("key=").append(key).append(", "); - sb.append("type=").append(type).append(", "); + sb.append(SCHEMA_FIELD_ID_KEY + "=").append(key).append(", "); + sb.append(SCHEMA_FIELD_ID_TYPE + "=").append(type).append(", "); sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); sb.append('}'); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index f5c176261039..d981b7085195 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -48,6 +48,7 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable { String RECORDKEY_PARTITION_LIST = "__all_partitions__"; // The partition name used for non-partitioned tables String NON_PARTITIONED_NAME = "."; + String EMPTY_PARTITION_NAME = ""; // Base path of the Metadata Table relative to the dataset (.hoodie/metadata) static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata"; @@ -112,4 +113,9 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad * Returns the timestamp of the latest compaction. */ Option getLatestCompactionTime(); + + /** + * Clear the states of the table metadata. + */ + void reset(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index b028056bb70d..58d63a194e81 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -50,7 +50,9 @@ import java.util.Map; import java.util.function.BiFunction; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; /** @@ -88,7 +90,7 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c List records = new LinkedList<>(); List allPartitions = new LinkedList<>(); commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { - final String partition = partitionStatName.equals("") ? NON_PARTITIONED_NAME : partitionStatName; + final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; allPartitions.add(partition); Map newFiles = new HashMap<>(writeStats.size()); @@ -102,8 +104,10 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : partition.length() + 1; String filename = pathWithPartition.substring(offset); - ValidationUtils.checkState(!newFiles.containsKey(filename), "Duplicate files in HoodieCommitMetadata"); - newFiles.put(filename, hoodieWriteStat.getTotalWriteBytes()); + long totalWriteBytes = newFiles.containsKey(filename) + ? newFiles.get(filename) + hoodieWriteStat.getTotalWriteBytes() + : hoodieWriteStat.getTotalWriteBytes(); + newFiles.put(filename, totalWriteBytes); }); // New files added to a partition HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( @@ -130,7 +134,8 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c public static List convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) { List records = new LinkedList<>(); int[] fileDeleteCount = {0}; - cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { + cleanMetadata.getPartitionMetadata().forEach((partitionName, partitionMetadata) -> { + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; // Files deleted from a partition List deletedFiles = partitionMetadata.getDeletePathPatterns(); HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), @@ -154,7 +159,7 @@ public static List convertMetadataToRecords(HoodieCleanMetadata cl * @return a list of metadata table records */ public static List convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline, - HoodieRestoreMetadata restoreMetadata, String instantTime, Option lastSyncTs) { + HoodieRestoreMetadata restoreMetadata, String instantTime, Option lastSyncTs) { Map> partitionToAppendedFiles = new HashMap<>(); Map> partitionToDeletedFiles = new HashMap<>(); restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { @@ -279,12 +284,13 @@ private static List convertFilesToRecords(Map List records = new LinkedList<>(); int[] fileChangeCount = {0, 0}; // deletes, appends - partitionToDeletedFiles.forEach((partition, deletedFiles) -> { + partitionToDeletedFiles.forEach((partitionName, deletedFiles) -> { fileChangeCount[0] += deletedFiles.size(); + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; Option> filesAdded = Option.empty(); - if (partitionToAppendedFiles.containsKey(partition)) { - filesAdded = Option.of(partitionToAppendedFiles.remove(partition)); + if (partitionToAppendedFiles.containsKey(partitionName)) { + filesAdded = Option.of(partitionToAppendedFiles.remove(partitionName)); } HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, filesAdded, @@ -292,7 +298,8 @@ private static List convertFilesToRecords(Map records.add(record); }); - partitionToAppendedFiles.forEach((partition, appendedFileMap) -> { + partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> { + final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; fileChangeCount[1] += appendedFileMap.size(); // Validate that no appended file has been deleted @@ -331,26 +338,65 @@ public static int mapRecordKeyToFileGroupIndex(String recordKey, int numFileGrou } /** - * Loads the list of file groups for a partition of the Metadata Table with latest file slices. + * Get the latest file slices for a Metadata Table partition. If the file slice is + * because of pending compaction instant, then merge the file slice with the one + * just before the compaction instant time. The list of file slices returned is + * sorted in the correct order of file group name. * - * The list of file slices returned is sorted in the correct order of file group name. - * @param metaClient instance of {@link HoodieTableMetaClient}. - * @param partition The name of the partition whose file groups are to be loaded. + * @param metaClient - Instance of {@link HoodieTableMetaClient}. + * @param partition - The name of the partition whose file groups are to be loaded. * @return List of latest file slices for all file groups in a given partition. */ - public static List loadPartitionFileGroupsWithLatestFileSlices(HoodieTableMetaClient metaClient, String partition) { - LOG.info("Loading file groups for metadata table partition " + partition); + public static List getPartitionLatestMergedFileSlices(HoodieTableMetaClient metaClient, String partition) { + LOG.info("Loading latest merged file slices for metadata table partition " + partition); + return getPartitionFileSlices(metaClient, partition, true); + } + + /** + * Get the latest file slices for a Metadata Table partition. The list of file slices + * returned is sorted in the correct order of file group name. + * + * @param metaClient - Instance of {@link HoodieTableMetaClient}. + * @param partition - The name of the partition whose file groups are to be loaded. + * @return List of latest file slices for all file groups in a given partition. + */ + public static List getPartitionLatestFileSlices(HoodieTableMetaClient metaClient, String partition) { + LOG.info("Loading latest file slices for metadata table partition " + partition); + return getPartitionFileSlices(metaClient, partition, false); + } - // If there are no commits on the metadata table then the table's default FileSystemView will not return any file - // slices even though we may have initialized them. + /** + * Get the latest file slices for a given partition. + * + * @param metaClient - Instance of {@link HoodieTableMetaClient}. + * @param partition - The name of the partition whose file groups are to be loaded. + * @param mergeFileSlices - When enabled, will merge the latest file slices with the last known + * completed instant. This is useful for readers when there are pending + * compactions. MergeFileSlices when disabled, will return the latest file + * slices without any merging, and this is needed for the writers. + * @return List of latest file slices for all file groups in a given partition. + */ + private static List getPartitionFileSlices(HoodieTableMetaClient metaClient, String partition, + boolean mergeFileSlices) { + // If there are no commits on the metadata table then the table's + // default FileSystemView will not return any file slices even + // though we may have initialized them. HoodieTimeline timeline = metaClient.getActiveTimeline(); if (timeline.empty()) { - final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime()); + final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, + HoodieActiveTimeline.createNewInstantTime()); timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); } HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline); - return fsView.getLatestFileSlices(partition).sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())) - .collect(Collectors.toList()); + Stream fileSliceStream; + if (mergeFileSlices) { + fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn( + partition, timeline.filterCompletedInstants().lastInstant().get().getTimestamp()); + } else { + fileSliceStream = fsView.getLatestFileSlices(partition); + } + return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList()); } + } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java index aba11220fa90..5b7147111a3a 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java @@ -94,7 +94,7 @@ public void testNoOpBootstrapIndex() throws IOException { props.put(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE.key(), "false"); Properties properties = new Properties(); properties.putAll(props); - HoodieTableConfig.createHoodieProperties(metaClient.getFs(), new Path(metaClient.getMetaPath()), properties); + HoodieTableConfig.create(metaClient.getFs(), new Path(metaClient.getMetaPath()), properties); metaClient = HoodieTableMetaClient.builder().setConf(metaClient.getHadoopConf()).setBasePath(basePath).build(); BootstrapIndex bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index 65c729e7aaed..0a2c5b4ea3db 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -79,14 +79,14 @@ public void setUp() throws IOException { @Test public void testMakeDataFileName() { - String instantTime = HoodieActiveTimeline.formatInstantTime(new Date()); + String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION); } @Test public void testMaskFileName() { - String instantTime = HoodieActiveTimeline.formatInstantTime(new Date()); + String instantTime = HoodieActiveTimeline.formatDate(new Date()); int taskPartitionId = 2; assertEquals(FSUtils.maskWithoutFileId(instantTime, taskPartitionId), "*_" + taskPartitionId + "_" + instantTime + BASE_FILE_EXTENSION); } @@ -154,7 +154,7 @@ public void testProcessFiles() throws Exception { @Test public void testGetCommitTime() { - String instantTime = HoodieActiveTimeline.formatInstantTime(new Date()); + String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(instantTime, FSUtils.getCommitTime(fullFileName)); @@ -165,7 +165,7 @@ public void testGetCommitTime() { @Test public void testGetFileNameWithoutMeta() { - String instantTime = HoodieActiveTimeline.formatInstantTime(new Date()); + String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(fileName, FSUtils.getFileId(fullFileName)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java index 4553aa5a923a..92f83aad7fd7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java @@ -296,6 +296,64 @@ public void testsetWorkingDirectory() throws IOException { }, "Should have thrown exception"); } + static class TestFSPath { + final Path inputPath; + final Path expectedInLineFSPath; + final Path transformedInputPath; + + TestFSPath(final Path inputPath, final Path expectedInLineFSPath, final Path transformedInputPath) { + this.inputPath = inputPath; + this.expectedInLineFSPath = expectedInLineFSPath; + this.transformedInputPath = transformedInputPath; + } + } + + @Test + public void testInLineFSPathConversions() { + final List expectedInLinePaths = Arrays.asList( + new TestFSPath( + new Path("/zero/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://zero/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/zero/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("file:/one/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://one/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/one/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("file://two/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://two/524bae7e-f01d-47ae-b7cd-910400a81336/file/?start_offset=10&length=10"), + new Path("file:/two/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("hdfs://three/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://three/524bae7e-f01d-47ae-b7cd-910400a81336/hdfs/?start_offset=10&length=10"), + new Path("hdfs://three/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("s3://four/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://four/524bae7e-f01d-47ae-b7cd-910400a81336/s3/?start_offset=10&length=10"), + new Path("s3://four/524bae7e-f01d-47ae-b7cd-910400a81336")), + new TestFSPath( + new Path("s3a://five/524bae7e-f01d-47ae-b7cd-910400a81336"), + new Path("inlinefs://five/524bae7e-f01d-47ae-b7cd-910400a81336/s3a/?start_offset=10&length=10"), + new Path("s3a://five/524bae7e-f01d-47ae-b7cd-910400a81336")) + ); + + for (TestFSPath entry : expectedInLinePaths) { + final Path inputPath = entry.inputPath; + final Path expectedInLineFSPath = entry.expectedInLineFSPath; + final Path expectedTransformedInputPath = entry.transformedInputPath; + + String scheme = "file"; + if (inputPath.toString().contains(":")) { + scheme = inputPath.toString().split(":")[0]; + } + final Path actualInLineFSPath = InLineFSUtils.getInlineFilePath(inputPath, scheme, 10, 10); + assertEquals(expectedInLineFSPath, actualInLineFSPath); + + final Path actualOuterFilePath = InLineFSUtils.getOuterFilePathFromInlinePath(actualInLineFSPath); + assertEquals(expectedTransformedInputPath, actualOuterFilePath); + } + } + @Test public void testExists() throws IOException { Path inlinePath = getRandomInlinePath(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index 4122d500b435..cc59b4602479 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.fs.inline; import org.apache.hudi.common.testutils.FileSystemTestUtils; +import org.apache.hudi.io.storage.HoodieHBaseKVComparator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -92,7 +93,7 @@ public void testSimpleInlineFileSystem() throws IOException { HFile.Writer writer = HFile.getWriterFactory(inMemoryConf, cacheConf) .withOutputStream(fout) .withFileContext(meta) - .withComparator(new KeyValue.KVComparator()) + .withComparator(new HoodieHBaseKVComparator()) .create(); writeRecords(writer); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index 3368c17c7bd1..bbfd8cf4ad39 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -51,9 +51,11 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.exception.CorruptedLogFileException; +import org.apache.hudi.exception.HoodieIOException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -385,6 +387,66 @@ public void testBasicWriteAndScan() throws IOException, URISyntaxException, Inte reader.close(); } + @Test + public void testHugeLogFileWrite() throws IOException, URISyntaxException, InterruptedException { + Writer writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(3L * 1024 * 1024 * 1024) + .build(); + Schema schema = getSimpleSchema(); + List records = SchemaTestUtil.generateTestRecords(0, 1000); + List copyOfRecords = records.stream() + .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + byte[] dataBlockContentBytes = getDataBlock(records, header).getContentBytes(); + HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, null, + Option.ofNullable(dataBlockContentBytes), false, 0, dataBlockContentBytes.length, + 0, getSimpleSchema(), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD); + long writtenSize = 0; + int logBlockWrittenNum = 0; + while (writtenSize < Integer.MAX_VALUE) { + AppendResult appendResult = writer.appendBlock(reusableDataBlock); + assertTrue(appendResult.size() > 0); + writtenSize += appendResult.size(); + logBlockWrittenNum++; + } + writer.close(); + + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + true, true); + assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it"); + HoodieLogBlock nextBlock = reader.next(); + assertEquals(dataBlockType, nextBlock.getBlockType(), "The next block should be a data block"); + HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock; + assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(), + "Read records size should be equal to the written records size"); + assertEquals(copyOfRecords, dataBlockRead.getRecords(), + "Both records lists should be the same. (ordering guaranteed)"); + int logBlockReadNum = 1; + while (reader.hasNext()) { + reader.next(); + logBlockReadNum++; + } + assertEquals(logBlockWrittenNum, logBlockReadNum, "All written log should be correctly found"); + reader.close(); + + // test writing oversize data block which should be rejected + Writer oversizeWriter = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withSizeThreshold(3L * 1024 * 1024 * 1024).withFs(fs) + .build(); + List dataBlocks = new ArrayList<>(logBlockWrittenNum + 1); + for (int i = 0; i < logBlockWrittenNum + 1; i++) { + dataBlocks.add(reusableDataBlock); + } + assertThrows(HoodieIOException.class, () -> { + oversizeWriter.appendBlocks(dataBlocks); + }, "Blocks appended may overflow. Please decrease log block size or log block amount"); + oversizeWriter.close(); + } + @Test public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException { Writer writer = @@ -603,6 +665,63 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep reader.close(); } + @Test + public void testValidateCorruptBlockEndPosition() throws IOException, URISyntaxException, InterruptedException { + Writer writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); + List records = SchemaTestUtil.generateTestRecords(0, 100); + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + HoodieDataBlock dataBlock = getDataBlock(records, header); + writer.appendBlock(dataBlock); + writer.close(); + + // Append some arbit byte[] to the end of the log (mimics a partially written commit) + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); + FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); + // create a block with + outputStream.write(HoodieLogFormat.MAGIC); + // Write out a length that does not confirm with the content + outputStream.writeLong(474); + outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal()); + outputStream.writeInt(HoodieLogFormat.CURRENT_VERSION); + // Write out a length that does not confirm with the content + outputStream.writeLong(400); + // Write out incomplete content + outputStream.write("something-random".getBytes()); + // get corrupt block end position + long corruptBlockEndPos = outputStream.getPos(); + outputStream.flush(); + outputStream.close(); + + // Append a proper block again + writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); + records = SchemaTestUtil.generateTestRecords(0, 10); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + dataBlock = getDataBlock(records, header); + writer.appendBlock(dataBlock); + writer.close(); + + // Read data and corrupt block + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + assertTrue(reader.hasNext(), "First block should be available"); + reader.next(); + assertTrue(reader.hasNext(), "We should have corrupted block next"); + HoodieLogBlock block = reader.next(); + assertEquals(HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType(), "The read block should be a corrupt block"); + // validate the corrupt block end position correctly. + assertEquals(corruptBlockEndPos, block.getBlockContentLocation().get().getBlockEndPos()); + assertTrue(reader.hasNext(), "Third block should be available"); + reader.next(); + assertFalse(reader.hasNext(), "There should be no more block left"); + + reader.close(); + } + @ParameterizedTest @MethodSource("testArguments") public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMapType, @@ -1482,7 +1601,8 @@ public void testBasicAppendAndReadInReverse(boolean readBlocksLazily) FileCreateUtils.createDeltaCommit(basePath, "100", fs); - HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + HoodieLogFileReader reader = new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(), + fs.getFileStatus(writer.getLogFile().getPath()).getLen()), SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true); assertTrue(reader.hasPrev(), "Last block should be available"); @@ -1560,7 +1680,8 @@ public void testAppendAndReadOnCorruptedLogInReverse(boolean readBlocksLazily) // First round of reads - we should be able to read the first block and then EOF HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), schema, bufferSize, readBlocksLazily, true); + new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(), + fs.getFileStatus(writer.getLogFile().getPath()).getLen()), schema, bufferSize, readBlocksLazily, true); assertTrue(reader.hasPrev(), "Last block should be available"); HoodieLogBlock block = reader.prev(); @@ -1610,7 +1731,8 @@ public void testBasicAppendAndTraverseInReverse(boolean readBlocksLazily) FileCreateUtils.createDeltaCommit(basePath, "100", fs); - HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + HoodieLogFileReader reader = new HoodieLogFileReader(fs, new HoodieLogFile(writer.getLogFile().getPath(), + fs.getFileStatus(writer.getLogFile().getPath()).getLen()), SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true); assertTrue(reader.hasPrev(), "Third block should be available"); @@ -1672,9 +1794,9 @@ private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List header) { switch (dataBlockType) { case AVRO_DATA_BLOCK: - return new HoodieAvroDataBlock(records, header); + return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); case HFILE_DATA_BLOCK: - return new HoodieHFileDataBlock(records, header); + return new HoodieHFileDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); default: throw new RuntimeException("Unknown data block type " + dataBlockType); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index 8fb9dddaa2e8..f8995ab4c07a 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -36,7 +36,7 @@ public class TestHoodieWriteStat { @Test public void testSetPaths() { - String instantTime = HoodieActiveTimeline.formatInstantTime(new Date()); + String instantTime = HoodieActiveTimeline.formatDate(new Date()); String basePathString = "/data/tables/some-hoodie-table"; String partitionPathString = "2017/12/31"; String fileName = UUID.randomUUID().toString(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java new file mode 100644 index 000000000000..6163c0ac468f --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or mo 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 org.apache.hudi.common.model.debezium; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestMySqlDebeziumAvroPayload { + + private static final String KEY_FIELD_NAME = "Key"; + + private Schema avroSchema; + + @BeforeEach + void setUp() { + this.avroSchema = Schema.createRecord(Arrays.asList( + new Schema.Field(KEY_FIELD_NAME, Schema.create(Schema.Type.INT), "", 0), + new Schema.Field(DebeziumConstants.FLATTENED_OP_COL_NAME, Schema.create(Schema.Type.STRING), "", null), + new Schema.Field(DebeziumConstants.ADDED_SEQ_COL_NAME, Schema.create(Schema.Type.STRING), "", null) + )); + } + + @Test + public void testInsert() throws IOException { + GenericRecord insertRecord = createRecord(0, Operation.INSERT, "00001.111"); + MySqlDebeziumAvroPayload payload = new MySqlDebeziumAvroPayload(insertRecord, "00001.111"); + validateRecord(payload.getInsertValue(avroSchema), 0, Operation.INSERT, "00001.111"); + } + + @Test + public void testPreCombine() { + GenericRecord insertRecord = createRecord(0, Operation.INSERT, "00002.111"); + MySqlDebeziumAvroPayload insertPayload = new MySqlDebeziumAvroPayload(insertRecord, "00002.111"); + + GenericRecord updateRecord = createRecord(0, Operation.UPDATE, "00001.111"); + MySqlDebeziumAvroPayload updatePayload = new MySqlDebeziumAvroPayload(updateRecord, "00001.111"); + + GenericRecord deleteRecord = createRecord(0, Operation.DELETE, "00002.11"); + MySqlDebeziumAvroPayload deletePayload = new MySqlDebeziumAvroPayload(deleteRecord, "00002.11"); + + assertEquals(insertPayload, insertPayload.preCombine(updatePayload)); + assertEquals(deletePayload, deletePayload.preCombine(updatePayload)); + assertEquals(insertPayload, deletePayload.preCombine(insertPayload)); + } + + @Test + public void testMergeWithUpdate() throws IOException { + GenericRecord updateRecord = createRecord(1, Operation.UPDATE, "00002.11"); + MySqlDebeziumAvroPayload payload = new MySqlDebeziumAvroPayload(updateRecord, "00002.11"); + + GenericRecord existingRecord = createRecord(1, Operation.INSERT, "00001.111"); + Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 1, Operation.UPDATE, "00002.11"); + + GenericRecord lateRecord = createRecord(1, Operation.UPDATE, "00000.222"); + payload = new MySqlDebeziumAvroPayload(lateRecord, "00000.222"); + mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 1, Operation.INSERT, "00001.111"); + } + + @Test + public void testMergeWithDelete() throws IOException { + GenericRecord deleteRecord = createRecord(2, Operation.DELETE, "00002.11"); + MySqlDebeziumAvroPayload payload = new MySqlDebeziumAvroPayload(deleteRecord, "00002.11"); + + GenericRecord existingRecord = createRecord(2, Operation.UPDATE, "00001.111"); + Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + // expect nothing to be committed to table + assertFalse(mergedRecord.isPresent()); + + GenericRecord lateRecord = createRecord(2, Operation.DELETE, "00000.222"); + payload = new MySqlDebeziumAvroPayload(lateRecord, "00000.222"); + mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 2, Operation.UPDATE, "00001.111"); + } + + private GenericRecord createRecord(int primaryKeyValue, Operation op, String seqValue) { + GenericRecord record = new GenericData.Record(avroSchema); + record.put(KEY_FIELD_NAME, primaryKeyValue); + record.put(DebeziumConstants.FLATTENED_OP_COL_NAME, op.op); + record.put(DebeziumConstants.ADDED_SEQ_COL_NAME, seqValue); + return record; + } + + private void validateRecord(Option iRecord, int primaryKeyValue, Operation op, String seqValue) { + IndexedRecord record = iRecord.get(); + assertEquals(primaryKeyValue, (int) record.get(0)); + assertEquals(op.op, record.get(1).toString()); + assertEquals(seqValue, record.get(2).toString()); + } + + private enum Operation { + INSERT("c"), + UPDATE("u"), + DELETE("d"); + + public final String op; + + Operation(String op) { + this.op = op; + } + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java new file mode 100644 index 000000000000..07512b1c36ea --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or mo 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 org.apache.hudi.common.model.debezium; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.util.Utf8; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class TestPostgresDebeziumAvroPayload { + + private static final String KEY_FIELD_NAME = "Key"; + private Schema avroSchema; + + @BeforeEach + void setUp() { + this.avroSchema = Schema.createRecord(Arrays.asList( + new Schema.Field(KEY_FIELD_NAME, Schema.create(Schema.Type.INT), "", 0), + new Schema.Field(DebeziumConstants.FLATTENED_OP_COL_NAME, Schema.create(Schema.Type.STRING), "", null), + new Schema.Field(DebeziumConstants.FLATTENED_LSN_COL_NAME, Schema.create(Schema.Type.LONG), "", null) + )); + } + + @Test + public void testInsert() throws IOException { + GenericRecord insertRecord = createRecord(0, Operation.INSERT, 100L); + PostgresDebeziumAvroPayload payload = new PostgresDebeziumAvroPayload(insertRecord, 100L); + validateRecord(payload.getInsertValue(avroSchema), 0, Operation.INSERT, 100L); + } + + @Test + public void testPreCombine() { + GenericRecord insertRecord = createRecord(0, Operation.INSERT, 120L); + PostgresDebeziumAvroPayload insertPayload = new PostgresDebeziumAvroPayload(insertRecord, 120L); + + GenericRecord updateRecord = createRecord(0, Operation.UPDATE, 99L); + PostgresDebeziumAvroPayload updatePayload = new PostgresDebeziumAvroPayload(updateRecord, 99L); + + GenericRecord deleteRecord = createRecord(0, Operation.DELETE, 111L); + PostgresDebeziumAvroPayload deletePayload = new PostgresDebeziumAvroPayload(deleteRecord, 111L); + + assertEquals(insertPayload, insertPayload.preCombine(updatePayload)); + assertEquals(deletePayload, deletePayload.preCombine(updatePayload)); + assertEquals(insertPayload, deletePayload.preCombine(insertPayload)); + } + + @Test + public void testMergeWithUpdate() throws IOException { + GenericRecord updateRecord = createRecord(1, Operation.UPDATE, 100L); + PostgresDebeziumAvroPayload payload = new PostgresDebeziumAvroPayload(updateRecord, 100L); + + GenericRecord existingRecord = createRecord(1, Operation.INSERT, 99L); + Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 1, Operation.UPDATE, 100L); + + GenericRecord lateRecord = createRecord(1, Operation.UPDATE, 98L); + payload = new PostgresDebeziumAvroPayload(lateRecord, 98L); + mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 1, Operation.INSERT, 99L); + } + + @Test + public void testMergeWithDelete() throws IOException { + GenericRecord deleteRecord = createRecord(2, Operation.DELETE, 100L); + PostgresDebeziumAvroPayload payload = new PostgresDebeziumAvroPayload(deleteRecord, 100L); + + GenericRecord existingRecord = createRecord(2, Operation.UPDATE, 99L); + Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + // expect nothing to be committed to table + assertFalse(mergedRecord.isPresent()); + + GenericRecord lateRecord = createRecord(2, Operation.DELETE, 98L); + payload = new PostgresDebeziumAvroPayload(lateRecord, 98L); + mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); + validateRecord(mergedRecord, 2, Operation.UPDATE, 99L); + } + + @Test + public void testMergeWithToastedValues() throws IOException { + Schema avroSchema = SchemaBuilder.builder() + .record("test_schema") + .namespace("test_namespace") + .fields() + .name(DebeziumConstants.FLATTENED_LSN_COL_NAME).type().longType().noDefault() + .name("string_col").type().stringType().noDefault() + .name("byte_col").type().bytesType().noDefault() + .name("string_null_col_1").type().nullable().stringType().noDefault() + .name("byte_null_col_1").type().nullable().bytesType().noDefault() + .name("string_null_col_2").type().nullable().stringType().noDefault() + .name("byte_null_col_2").type().nullable().bytesType().noDefault() + .endRecord(); + + GenericRecord oldVal = new GenericData.Record(avroSchema); + oldVal.put(DebeziumConstants.FLATTENED_LSN_COL_NAME, 100L); + oldVal.put("string_col", "valid string value"); + oldVal.put("byte_col", ByteBuffer.wrap("valid byte value".getBytes())); + oldVal.put("string_null_col_1", "valid string value"); + oldVal.put("byte_null_col_1", ByteBuffer.wrap("valid byte value".getBytes())); + oldVal.put("string_null_col_2", null); + oldVal.put("byte_null_col_2", null); + + GenericRecord newVal = new GenericData.Record(avroSchema); + newVal.put(DebeziumConstants.FLATTENED_LSN_COL_NAME, 105L); + newVal.put("string_col", PostgresDebeziumAvroPayload.DEBEZIUM_TOASTED_VALUE); + newVal.put("byte_col", ByteBuffer.wrap(PostgresDebeziumAvroPayload.DEBEZIUM_TOASTED_VALUE.getBytes())); + newVal.put("string_null_col_1", null); + newVal.put("byte_null_col_1", null); + newVal.put("string_null_col_2", "valid string value"); + newVal.put("byte_null_col_2", ByteBuffer.wrap("valid byte value".getBytes())); + + PostgresDebeziumAvroPayload payload = new PostgresDebeziumAvroPayload(Option.of(newVal)); + + GenericRecord outputRecord = (GenericRecord) payload + .combineAndGetUpdateValue(oldVal, avroSchema).get(); + + assertEquals("valid string value", outputRecord.get("string_col")); + assertEquals("valid byte value", new String(((ByteBuffer) outputRecord.get("byte_col")).array(), StandardCharsets.UTF_8)); + assertNull(outputRecord.get("string_null_col_1")); + assertNull(outputRecord.get("byte_null_col_1")); + assertEquals("valid string value", ((Utf8) outputRecord.get("string_null_col_2")).toString()); + assertEquals("valid byte value", new String(((ByteBuffer) outputRecord.get("byte_null_col_2")).array(), StandardCharsets.UTF_8)); + } + + private GenericRecord createRecord(int primaryKeyValue, Operation op, long lsnValue) { + GenericRecord record = new GenericData.Record(avroSchema); + record.put(KEY_FIELD_NAME, primaryKeyValue); + record.put(DebeziumConstants.FLATTENED_OP_COL_NAME, op.op); + record.put(DebeziumConstants.FLATTENED_LSN_COL_NAME, lsnValue); + return record; + } + + private void validateRecord(Option iRecord, int primaryKeyValue, Operation op, long lsnValue) { + IndexedRecord record = iRecord.get(); + assertEquals(primaryKeyValue, (int) record.get(0)); + assertEquals(op.op, record.get(1).toString()); + assertEquals(lsnValue, (long) record.get(2)); + } + + private enum Operation { + INSERT("c"), + UPDATE("u"), + DELETE("d"); + + public final String op; + + Operation(String op) { + this.op = op; + } + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java new file mode 100644 index 000000000000..73d101cf2c71 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java @@ -0,0 +1,137 @@ +/* + * 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 org.apache.hudi.common.table; + +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.util.Properties; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieTableConfig extends HoodieCommonTestHarness { + + private FileSystem fs; + private Path metaPath; + private Path cfgPath; + private Path backupCfgPath; + + @BeforeEach + public void setUp() throws Exception { + initPath(); + fs = new Path(basePath).getFileSystem(new Configuration()); + metaPath = new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + Properties props = new Properties(); + props.setProperty(HoodieTableConfig.NAME.key(), "test-table"); + HoodieTableConfig.create(fs, metaPath, props); + cfgPath = new Path(metaPath, HoodieTableConfig.HOODIE_PROPERTIES_FILE); + backupCfgPath = new Path(metaPath, HoodieTableConfig.HOODIE_PROPERTIES_FILE_BACKUP); + } + + @Test + public void testCreate() throws IOException { + assertTrue(fs.exists(new Path(metaPath, HoodieTableConfig.HOODIE_PROPERTIES_FILE))); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + assertEquals(4, config.getProps().size()); + } + + @Test + public void testUpdate() throws IOException { + Properties updatedProps = new Properties(); + updatedProps.setProperty(HoodieTableConfig.NAME.key(), "test-table2"); + updatedProps.setProperty(HoodieTableConfig.PRECOMBINE_FIELD.key(), "new_field"); + HoodieTableConfig.update(fs, metaPath, updatedProps); + + assertTrue(fs.exists(cfgPath)); + assertFalse(fs.exists(backupCfgPath)); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + assertEquals(5, config.getProps().size()); + assertEquals("test-table2", config.getTableName()); + assertEquals("new_field", config.getPreCombineField()); + } + + @Test + public void testDelete() throws IOException { + Set deletedProps = CollectionUtils.createSet(HoodieTableConfig.ARCHIVELOG_FOLDER.key(), "hoodie.invalid.config"); + HoodieTableConfig.delete(fs, metaPath, deletedProps); + + assertTrue(fs.exists(cfgPath)); + assertFalse(fs.exists(backupCfgPath)); + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + assertEquals(3, config.getProps().size()); + assertNull(config.getProps().getProperty("hoodie.invalid.config")); + assertFalse(config.getProps().contains(HoodieTableConfig.ARCHIVELOG_FOLDER.key())); + } + + @Test + public void testReadsWhenPropsFileDoesNotExist() throws IOException { + fs.delete(cfgPath, false); + assertThrows(HoodieIOException.class, () -> { + new HoodieTableConfig(fs, metaPath.toString(), null); + }); + } + + @Test + public void testReadsWithUpdateFailures() throws IOException { + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + fs.delete(cfgPath, false); + try (FSDataOutputStream out = fs.create(backupCfgPath)) { + config.getProps().store(out, ""); + } + + assertFalse(fs.exists(cfgPath)); + assertTrue(fs.exists(backupCfgPath)); + config = new HoodieTableConfig(fs, metaPath.toString(), null); + assertEquals(4, config.getProps().size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testUpdateRecovery(boolean shouldPropsFileExist) throws IOException { + HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null); + if (!shouldPropsFileExist) { + fs.delete(cfgPath, false); + } + try (FSDataOutputStream out = fs.create(backupCfgPath)) { + config.getProps().store(out, ""); + } + + HoodieTableConfig.recoverIfNeeded(fs, cfgPath, backupCfgPath); + assertTrue(fs.exists(cfgPath)); + assertFalse(fs.exists(backupCfgPath)); + config = new HoodieTableConfig(fs, metaPath.toString(), null); + assertEquals(4, config.getProps().size()); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 5f2d6928cbaa..9d89c2a6b5fe 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -26,11 +26,13 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; + import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.io.IOException; +import java.text.ParseException; import java.util.ArrayList; import java.util.Collections; import java.util.Date; @@ -175,6 +177,15 @@ public void testTimelineOperations() { assertFalse(timeline.empty()); assertFalse(timeline.getCommitTimeline().filterPendingExcludingCompaction().empty()); assertEquals(12, timeline.countInstants()); + assertEquals("01", timeline.firstInstant( + HoodieTimeline.COMMIT_ACTION, State.COMPLETED).get().getTimestamp()); + assertEquals("21", timeline.firstInstant( + HoodieTimeline.COMMIT_ACTION, State.INFLIGHT).get().getTimestamp()); + assertFalse(timeline.firstInstant( + HoodieTimeline.COMMIT_ACTION, State.REQUESTED).isPresent()); + assertFalse(timeline.firstInstant( + HoodieTimeline.REPLACE_COMMIT_ACTION, State.COMPLETED).isPresent()); + HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants(); assertEquals(10, activeCommitTimeline.countInstants()); @@ -443,7 +454,7 @@ public void testCreateNewInstantTime() throws Exception { } // All zero timestamp can be parsed - HoodieActiveTimeline.parseInstantTime("00000000000000"); + HoodieActiveTimeline.parseDateFromInstantTime("00000000000000"); // Multiple thread test final int numChecks = 100000; @@ -454,9 +465,9 @@ public void testCreateNewInstantTime() throws Exception { for (int idx = 0; idx < numThreads; ++idx) { futures.add(executorService.submit(() -> { Date date = new Date(System.currentTimeMillis() + (int)(Math.random() * numThreads) * milliSecondsInYear); - final String expectedFormat = HoodieActiveTimeline.formatInstantTime(date); + final String expectedFormat = HoodieActiveTimeline.formatDate(date); for (int tidx = 0; tidx < numChecks; ++tidx) { - final String curFormat = HoodieActiveTimeline.formatInstantTime(date); + final String curFormat = HoodieActiveTimeline.formatDate(date); if (!curFormat.equals(expectedFormat)) { throw new HoodieException("Format error: expected=" + expectedFormat + ", curFormat=" + curFormat); } @@ -472,6 +483,40 @@ public void testCreateNewInstantTime() throws Exception { } } + @Test + public void testMetadataCompactionInstantDateParsing() throws ParseException { + // default second granularity instant ID + String secondGranularityInstant = "20210101120101123"; + Date defaultSecsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant); + // metadata table compaction/cleaning : ms granularity instant ID + String compactionInstant = secondGranularityInstant + "001"; + Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(compactionInstant); + assertEquals(0, defaultMsGranularityDate.getTime() - defaultSecsGranularityDate.getTime(), "Expected the ms part to be 0"); + assertTrue(HoodieTimeline.compareTimestamps(secondGranularityInstant, HoodieTimeline.LESSER_THAN, compactionInstant)); + assertTrue(HoodieTimeline.compareTimestamps(compactionInstant, HoodieTimeline.GREATER_THAN, secondGranularityInstant)); + } + + @Test + public void testMillisGranularityInstantDateParsing() throws ParseException { + // Old second granularity instant ID + String secondGranularityInstant = "20210101120101"; + Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant); + // New ms granularity instant ID + String specificMsGranularityInstant = secondGranularityInstant + "009"; + Date msGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(specificMsGranularityInstant); + assertEquals(999, defaultMsGranularityDate.getTime() % 1000, "Expected the ms part to be 999"); + assertEquals(9, msGranularityDate.getTime() % 1000, "Expected the ms part to be 9"); + + // Ensure that any date math which expects second granularity still works + String laterDateInstant = "20210101120111"; // + 10 seconds from original instant + assertEquals( + 10, + HoodieActiveTimeline.parseDateFromInstantTime(laterDateInstant).getTime() / 1000 + - HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant).getTime() / 1000, + "Expected the difference between later instant and previous instant to be 10 seconds" + ); + } + /** * Returns an exhaustive list of all possible HoodieInstant. * @return list of HoodieInstant diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java index e0f1f42a85b2..62cc23a99c94 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java @@ -30,6 +30,11 @@ import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; +import org.apache.http.client.HttpResponseException; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.log4j.spi.LoggingEvent; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -37,12 +42,14 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -102,6 +109,30 @@ public void testGetLatestBaseFiles() { }); } + @Test + public void testBadRequestExceptionWithPrimary() { + final TestLogAppender appender = new TestLogAppender(); + final Logger logger = Logger.getRootLogger(); + try { + logger.addAppender(appender); + Stream actual; + Stream expected = testBaseFileStream; + + resetMocks(); + when(primary.getLatestBaseFiles()).thenThrow(new RuntimeException(new HttpResponseException(400, "Bad Request"))); + when(secondary.getLatestBaseFiles()).thenReturn(testBaseFileStream); + actual = fsView.getLatestBaseFiles(); + assertEquals(expected, actual); + final List logs = appender.getLog(); + final LoggingEvent firstLogEntry = logs.get(0); + assertEquals(firstLogEntry.getLevel(), Level.WARN); + assertTrue(((String)firstLogEntry.getMessage()).contains("Got error running preferred function. Likely due to another " + + "concurrent writer in progress. Trying secondary")); + } finally { + logger.removeAppender(appender); + } + } + @Test public void testGetLatestBaseFilesWithPartitionPath() { Stream actual; @@ -591,8 +622,8 @@ public void testGetTimeline() { @Test public void testSync() { fsView.sync(); - verify(primary, times(1)).reset(); - verify(secondary, times(1)).reset(); + verify(primary, times(1)).sync(); + verify(secondary, times(1)).sync(); } @Test @@ -633,4 +664,26 @@ public void testGetPreferredView() { public void testGetSecondaryView() { assertEquals(secondary, fsView.getSecondaryView()); } + + class TestLogAppender extends AppenderSkeleton { + private final List log = new ArrayList(); + + @Override + public boolean requiresLayout() { + return false; + } + + @Override + protected void append(final LoggingEvent loggingEvent) { + log.add(loggingEvent); + } + + @Override + public void close() { + } + + public List getLog() { + return new ArrayList(log); + } + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 491ad32f90df..1968ef422560 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -19,12 +19,14 @@ package org.apache.hudi.common.testutils; +import org.apache.directory.api.util.Strings; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; @@ -63,6 +65,7 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRestoreMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackPlan; public class FileCreateUtils { @@ -217,20 +220,36 @@ public static void createCleanFile(String basePath, String instantTime, HoodieCl createMetaFile(basePath, instantTime, HoodieTimeline.CLEAN_EXTENSION, serializeCleanMetadata(metadata).get()); } + public static void createCleanFile(String basePath, String instantTime, HoodieCleanMetadata metadata, boolean isEmpty) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.CLEAN_EXTENSION, isEmpty ? Strings.EMPTY_BYTES : serializeCleanMetadata(metadata).get()); + } + public static void createRequestedCleanFile(String basePath, String instantTime, HoodieCleanerPlan cleanerPlan) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_CLEAN_EXTENSION, serializeCleanerPlan(cleanerPlan).get()); } + public static void createRequestedCleanFile(String basePath, String instantTime, HoodieCleanerPlan cleanerPlan, boolean isEmpty) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_CLEAN_EXTENSION, isEmpty ? Strings.EMPTY_BYTES : serializeCleanerPlan(cleanerPlan).get()); + } + public static void createInflightCleanFile(String basePath, String instantTime, HoodieCleanerPlan cleanerPlan) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION, serializeCleanerPlan(cleanerPlan).get()); } + public static void createInflightCleanFile(String basePath, String instantTime, HoodieCleanerPlan cleanerPlan, boolean isEmpty) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION, isEmpty ? Strings.EMPTY_BYTES : serializeCleanerPlan(cleanerPlan).get()); + } + + public static void createRequestedRollbackFile(String basePath, String instantTime, HoodieRollbackPlan plan) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_ROLLBACK_EXTENSION, serializeRollbackPlan(plan).get()); + } + public static void createInflightRollbackFile(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); } - public static void createRollbackFile(String basePath, String instantTime, HoodieRollbackMetadata hoodieRollbackMetadata) throws IOException { - createMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION, serializeRollbackMetadata(hoodieRollbackMetadata).get()); + public static void createRollbackFile(String basePath, String instantTime, HoodieRollbackMetadata hoodieRollbackMetadata, boolean isEmpty) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION, isEmpty ? Strings.EMPTY_BYTES : serializeRollbackMetadata(hoodieRollbackMetadata).get()); } public static void createRestoreFile(String basePath, String instantTime, HoodieRestoreMetadata hoodieRestoreMetadata) throws IOException { @@ -340,6 +359,20 @@ public static void deleteReplaceCommit(String basePath, String instantTime) thro removeMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION); } + public static void deleteRollbackCommit(String basePath, String instantTime) throws IOException { + removeMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION); + } + + public static java.nio.file.Path renameFileToTemp(java.nio.file.Path sourcePath, String instantTime) throws IOException { + java.nio.file.Path dummyFilePath = sourcePath.getParent().resolve(instantTime + ".temp"); + Files.move(sourcePath, dummyFilePath); + return dummyFilePath; + } + + public static void renameTempToMetaFile(java.nio.file.Path tempFilePath, java.nio.file.Path destPath) throws IOException { + Files.move(tempFilePath, destPath); + } + public static long getTotalMarkerFileCount(String basePath, String partitionPath, String instantTime, IOType ioType) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime, partitionPath); if (Files.notExists(parentPath)) { @@ -353,7 +386,9 @@ public static List getPartitionPaths(Path basePath) throws IOException { if (Files.notExists(basePath)) { return Collections.emptyList(); } - return Files.list(basePath).filter(entry -> !entry.getFileName().toString().equals(HoodieTableMetaClient.METAFOLDER_NAME)).collect(Collectors.toList()); + return Files.list(basePath).filter(entry -> (!entry.getFileName().toString().equals(HoodieTableMetaClient.METAFOLDER_NAME) + && !entry.getFileName().toString().contains("parquet") && !entry.getFileName().toString().contains("log")) + && !entry.getFileName().toString().endsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)).collect(Collectors.toList()); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index e988c9df618c..21816a56c2db 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -76,7 +76,7 @@ *

* Test data uses a toy Uber trips, data model. */ -public class HoodieTestDataGenerator { +public class HoodieTestDataGenerator implements AutoCloseable { // based on examination of sample file, the schema produces the following per record size public static final int BYTES_PER_RECORD = (int) (1.2 * 1024); @@ -388,6 +388,17 @@ public static void createReplaceFile(String basePath, String instantTime, Config .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); } + public static void createPendingReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) { + Arrays.asList(HoodieTimeline.makeInflightReplaceFileName(instantTime), + HoodieTimeline.makeRequestedReplaceFileName(instantTime)) + .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); + } + + public static void createPendingReplaceFile(String basePath, String instantTime, Configuration configuration) { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + createPendingReplaceFile(basePath, instantTime, configuration, commitMetadata); + } + public static void createEmptyCleanRequestedFile(String basePath, String instantTime, Configuration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" @@ -849,6 +860,7 @@ public static class KeyPartition implements Serializable { public String partitionPath; } + @Override public void close() { existingKeysBySchema.clear(); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 95d0657cb208..7b8148a612a8 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -28,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; +import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.avro.model.HoodieSavepointPartitionMetadata; import org.apache.hudi.common.HoodieCleanStat; @@ -104,6 +105,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createRestoreFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; @@ -131,6 +133,7 @@ protected HoodieTestTable(String basePath, FileSystem fs, HoodieTableMetaClient this.basePath = basePath; this.fs = fs; this.metaClient = metaClient; + testTableState = HoodieTestTableState.of(); } public static HoodieTestTable of(HoodieTableMetaClient metaClient) { @@ -147,7 +150,7 @@ public static String makeNewCommitTime() { } public static String makeNewCommitTime(Instant dateTime) { - return HoodieActiveTimeline.formatInstantTime(Date.from(dateTime)); + return HoodieActiveTimeline.formatDate(Date.from(dateTime)); } public static List makeIncrementalCommitTimes(int num) { @@ -276,9 +279,13 @@ public HoodieTestTable addInflightClean(String instantTime, HoodieCleanerPlan cl } public HoodieTestTable addClean(String instantTime, HoodieCleanerPlan cleanerPlan, HoodieCleanMetadata metadata) throws IOException { - createRequestedCleanFile(basePath, instantTime, cleanerPlan); - createInflightCleanFile(basePath, instantTime, cleanerPlan); - createCleanFile(basePath, instantTime, metadata); + return addClean(instantTime, cleanerPlan, metadata, false); + } + + public HoodieTestTable addClean(String instantTime, HoodieCleanerPlan cleanerPlan, HoodieCleanMetadata metadata, boolean isEmpty) throws IOException { + createRequestedCleanFile(basePath, instantTime, cleanerPlan, isEmpty); + createInflightCleanFile(basePath, instantTime, cleanerPlan, isEmpty); + createCleanFile(basePath, instantTime, metadata, isEmpty); currentInstantTime = instantTime; return this; } @@ -308,6 +315,12 @@ public Pair getHoodieCleanMetadata(Strin return Pair.of(cleanerPlan, convertCleanMetadata(commitTime, Option.of(0L), cleanStats)); } + public HoodieTestTable addRequestedRollback(String instantTime, HoodieRollbackPlan plan) throws IOException { + createRequestedRollbackFile(basePath, instantTime, plan); + currentInstantTime = instantTime; + return this; + } + public HoodieTestTable addInflightRollback(String instantTime) throws IOException { createInflightRollbackFile(basePath, instantTime); currentInstantTime = instantTime; @@ -315,8 +328,12 @@ public HoodieTestTable addInflightRollback(String instantTime) throws IOExceptio } public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + return addRollback(instantTime, rollbackMetadata, false); + } + + public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata rollbackMetadata, boolean isEmpty) throws IOException { createInflightRollbackFile(basePath, instantTime); - createRollbackFile(basePath, instantTime, rollbackMetadata); + createRollbackFile(basePath, instantTime, rollbackMetadata, isEmpty); currentInstantTime = instantTime; return this; } @@ -601,7 +618,7 @@ public Path getPartitionPath(String partition) { } public List getAllPartitionPaths() throws IOException { - java.nio.file.Path basePathPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME).getParent().getParent(); + java.nio.file.Path basePathPath = Paths.get(basePath); return FileCreateUtils.getPartitionPaths(basePathPath); } @@ -659,8 +676,10 @@ public FileStatus[] listAllFilesInPartition(String partitionPath) throws IOExcep return FileSystemTestUtils.listRecursive(fs, new Path(Paths.get(basePath, partitionPath).toString())).stream() .filter(entry -> { boolean toReturn = true; + String filePath = entry.getPath().toString(); String fileName = entry.getPath().getName(); - if (fileName.equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) { + if (fileName.equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE) || (!fileName.contains("log") && !fileName.contains("parquet")) + || filePath.contains("metadata")) { toReturn = false; } else { for (String inflight : inflightCommits) { @@ -703,6 +722,25 @@ public HoodieTestTable doRollback(String commitTimeToRollback, String commitTime return addRollback(commitTime, rollbackMetadata); } + public HoodieTestTable doRollbackWithExtraFiles(String commitTimeToRollback, String commitTime, Map> extraFiles) throws Exception { + metaClient = HoodieTableMetaClient.reload(metaClient); + Option commitMetadata = getMetadataForInstant(commitTimeToRollback); + if (!commitMetadata.isPresent()) { + throw new IllegalArgumentException("Instant to rollback not present in timeline: " + commitTimeToRollback); + } + Map> partitionFiles = getPartitionFiles(commitMetadata.get()); + for (Map.Entry> entry : partitionFiles.entrySet()) { + deleteFilesInPartition(entry.getKey(), entry.getValue()); + } + for (Map.Entry> entry: extraFiles.entrySet()) { + if (partitionFiles.containsKey(entry.getKey())) { + partitionFiles.get(entry.getKey()).addAll(entry.getValue()); + } + } + HoodieRollbackMetadata rollbackMetadata = getRollbackMetadata(commitTimeToRollback, partitionFiles); + return addRollback(commitTime, rollbackMetadata); + } + public HoodieTestTable doRestore(String commitToRestoreTo, String restoreTime) throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); List commitsToRollback = metaClient.getActiveTimeline().getCommitsTimeline() @@ -985,7 +1023,7 @@ private static HoodieTestTableState getTestTableStateWithPartitionFileInfo(Write * @param tableType - Hudi table type * @param commitTime - Write commit time * @param partitionToFilesNameLengthMap - Map of partition names to its list of files and their lengths - * @return Test tabke state for the requested partitions and files + * @return Test table state for the requested partitions and files */ private static HoodieTestTableState getTestTableStateWithPartitionFileInfo(WriteOperationType operationType, HoodieTableType tableType, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestZOrderingUtil.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestBinaryUtil.java similarity index 72% rename from hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestZOrderingUtil.java rename to hudi-common/src/test/java/org/apache/hudi/common/util/TestBinaryUtil.java index 7dab6c2057c7..1efe5a06865d 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/optimize/TestZOrderingUtil.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestBinaryUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.hudi.optimize; +package org.apache.hudi.common.util; import org.junit.jupiter.api.Test; @@ -27,7 +27,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; -public class TestZOrderingUtil { +public class TestBinaryUtil { @Test public void testIntConvert() { @@ -37,12 +37,12 @@ public void testIntConvert() { List> convertResultWrappers = new ArrayList<>(); for (int i = 0; i < testInt.length; i++) { valueWrappers.add(new OrginValueWrapper<>(i, testInt[i])); - convertResultWrappers.add(new ConvertResultWrapper<>(i, ZOrderingUtil.intTo8Byte(testInt[i]))); + convertResultWrappers.add(new ConvertResultWrapper<>(i, BinaryUtil.intTo8Byte(testInt[i]))); } Collections.sort(valueWrappers, ((o1, o2) -> o1.originValue.compareTo(o2.originValue))); - Collections.sort(convertResultWrappers, ((o1, o2) -> ZOrderingUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); + Collections.sort(convertResultWrappers, ((o1, o2) -> BinaryUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); for (int i = 0; i < testInt.length; i++) { assertEquals(valueWrappers.get(i).index, convertResultWrappers.get(i).index); @@ -57,12 +57,12 @@ public void testLongConvert() { List> convertResultWrappers = new ArrayList<>(); for (int i = 0; i < testLong.length; i++) { valueWrappers.add(new OrginValueWrapper<>((long)i, testLong[i])); - convertResultWrappers.add(new ConvertResultWrapper<>((long)i, ZOrderingUtil.longTo8Byte(testLong[i]))); + convertResultWrappers.add(new ConvertResultWrapper<>((long)i, BinaryUtil.longTo8Byte(testLong[i]))); } Collections.sort(valueWrappers, ((o1, o2) -> o1.originValue.compareTo(o2.originValue))); - Collections.sort(convertResultWrappers, ((o1, o2) -> ZOrderingUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); + Collections.sort(convertResultWrappers, ((o1, o2) -> BinaryUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); for (int i = 0; i < testLong.length; i++) { assertEquals(valueWrappers.get(i).index, convertResultWrappers.get(i).index); @@ -77,12 +77,12 @@ public void testDoubleConvert() { List> convertResultWrappers = new ArrayList<>(); for (int i = 0; i < testDouble.length; i++) { valueWrappers.add(new OrginValueWrapper<>((Double)(i * 1.0), testDouble[i])); - convertResultWrappers.add(new ConvertResultWrapper<>((Double)(i * 1.0), ZOrderingUtil.doubleTo8Byte(testDouble[i]))); + convertResultWrappers.add(new ConvertResultWrapper<>((Double)(i * 1.0), BinaryUtil.doubleTo8Byte(testDouble[i]))); } Collections.sort(valueWrappers, ((o1, o2) -> o1.originValue.compareTo(o2.originValue))); - Collections.sort(convertResultWrappers, ((o1, o2) -> ZOrderingUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); + Collections.sort(convertResultWrappers, ((o1, o2) -> BinaryUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); for (int i = 0; i < testDouble.length; i++) { assertEquals(valueWrappers.get(i).index, convertResultWrappers.get(i).index); @@ -97,12 +97,12 @@ public void testFloatConvert() { List> convertResultWrappers = new ArrayList<>(); for (int i = 0; i < testDouble.length; i++) { valueWrappers.add(new OrginValueWrapper<>((float)(i * 1.0), testDouble[i])); - convertResultWrappers.add(new ConvertResultWrapper<>((float)(i * 1.0), ZOrderingUtil.doubleTo8Byte((double) testDouble[i]))); + convertResultWrappers.add(new ConvertResultWrapper<>((float)(i * 1.0), BinaryUtil.doubleTo8Byte((double) testDouble[i]))); } Collections.sort(valueWrappers, ((o1, o2) -> o1.originValue.compareTo(o2.originValue))); - Collections.sort(convertResultWrappers, ((o1, o2) -> ZOrderingUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); + Collections.sort(convertResultWrappers, ((o1, o2) -> BinaryUtil.compareTo(o1.result, 0, o1.result.length, o2.result, 0, o2.result.length))); for (int i = 0; i < testDouble.length; i++) { assertEquals(valueWrappers.get(i).index, convertResultWrappers.get(i).index); @@ -126,4 +126,29 @@ public OrginValueWrapper(T index, T originValue) { this.originValue = originValue; } } + + @Test + public void testConvertBytesToLong() { + long[] tests = new long[] {Long.MIN_VALUE, -1L, 0, 1L, Long.MAX_VALUE}; + for (int i = 0; i < tests.length; i++) { + assertEquals(BinaryUtil.convertBytesToLong(convertLongToBytes(tests[i])), tests[i]); + } + } + + @Test + public void testConvertBytesToLongWithPadding() { + byte[] bytes = new byte[2]; + bytes[0] = 2; + bytes[1] = 127; + assertEquals(BinaryUtil.convertBytesToLong(bytes), 2 * 256 + 127); + } + + private byte[] convertLongToBytes(long num) { + byte[] byteNum = new byte[8]; + for (int i = 0; i < 8; i++) { + int offset = 64 - (i + 1) * 8; + byteNum[i] = (byte) ((num >> offset) & 0xff); + } + return byteNum; + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java index ddce3216b0d6..465739340dc8 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java @@ -25,10 +25,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.Rule; +import org.junit.contrib.java.lang.system.EnvironmentVariables; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import java.io.File; import java.io.IOException; import java.io.PrintStream; @@ -47,6 +51,10 @@ public class TestDFSPropertiesConfiguration { private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; + @Rule + public static final EnvironmentVariables ENVIRONMENT_VARIABLES + = new EnvironmentVariables(); + @BeforeAll public static void initClass() throws Exception { hdfsTestService = new HdfsTestService(); @@ -73,12 +81,17 @@ public static void initClass() throws Exception { } @AfterAll - public static void cleanupClass() throws Exception { + public static void cleanupClass() { if (hdfsTestService != null) { hdfsTestService.stop(); } } + @AfterEach + public void cleanupGlobalConfig() { + DFSPropertiesConfiguration.clearGlobalProps(); + } + private static void writePropertiesFile(Path path, String[] lines) throws IOException { PrintStream out = new PrintStream(dfs.create(path, true)); for (String line : lines) { @@ -90,8 +103,8 @@ private static void writePropertiesFile(Path path, String[] lines) throws IOExce @Test public void testParsing() { - DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props")); - TypedProperties props = cfg.getConfig(); + DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t1.props")); + TypedProperties props = cfg.getProps(); assertEquals(5, props.size()); assertThrows(IllegalArgumentException.class, () -> { props.getString("invalid.key"); @@ -118,8 +131,8 @@ public void testParsing() { @Test public void testIncludes() { - DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t3.props")); - TypedProperties props = cfg.getConfig(); + DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t3.props")); + TypedProperties props = cfg.getProps(); assertEquals(123, props.getInteger("int.prop")); assertEquals(243.4, props.getDouble("double.prop"), 0.001); @@ -127,7 +140,54 @@ public void testIncludes() { assertEquals("t3.value", props.getString("string.prop")); assertEquals(1354354354, props.getLong("long.prop")); assertThrows(IllegalStateException.class, () -> { - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t4.props")); + cfg.addPropsFromFile(new Path(dfsBasePath + "/t4.props")); }, "Should error out on a self-included file."); } + + @Test + public void testLocalFileSystemLoading() throws IOException { + DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t1.props")); + + cfg.addPropsFromFile( + new Path( + String.format( + "file:%s", + getClass().getClassLoader() + .getResource("props/test.properties") + .getPath() + ) + )); + + TypedProperties props = cfg.getProps(); + + assertEquals(123, props.getInteger("int.prop")); + assertEquals(113.4, props.getDouble("double.prop"), 0.001); + assertTrue(props.getBoolean("boolean.prop")); + assertEquals("str", props.getString("string.prop")); + assertEquals(1354354354, props.getLong("long.prop")); + assertEquals(123, props.getInteger("some.random.prop")); + } + + @Test + public void testNoGlobalConfFileConfigured() { + ENVIRONMENT_VARIABLES.clear(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME); + // Should not throw any exception when no external configuration file configured + DFSPropertiesConfiguration.refreshGlobalProps(); + assertEquals(0, DFSPropertiesConfiguration.getGlobalProps().size()); + } + + @Test + public void testLoadGlobalConfFile() { + // set HUDI_CONF_DIR + String testPropsFilePath = new File("src/test/resources/external-config").getAbsolutePath(); + ENVIRONMENT_VARIABLES.set(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, testPropsFilePath); + + DFSPropertiesConfiguration.refreshGlobalProps(); + assertEquals(5, DFSPropertiesConfiguration.getGlobalProps().size()); + assertEquals("jdbc:hive2://localhost:10000", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.jdbcurl")); + assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.use_jdbc")); + assertEquals("false", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.support_timestamp")); + assertEquals("BLOOM", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.index.type")); + assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.metadata.enable")); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java index 4fed5a80eb1f..f7b45e9d839b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java @@ -51,6 +51,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -83,7 +84,7 @@ public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType, boole List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); assert (recordKeys.size() == 100); - + // Test iterator Iterator> itr = records.iterator(); int cntSize = 0; @@ -93,7 +94,7 @@ public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType, boole assert recordKeys.contains(rec.getRecordKey()); } assertEquals(recordKeys.size(), cntSize); - + // Test value stream List> values = records.valueStream().collect(Collectors.toList()); cntSize = 0; @@ -221,7 +222,9 @@ failureOutputPath, new DefaultSizeEstimator(), @ParameterizedTest @MethodSource("testArguments") - public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException { + public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled) throws IOException, + URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); @@ -274,7 +277,9 @@ record = records.get(key); @ParameterizedTest @MethodSource("testArguments") - public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws IOException, URISyntaxException { + public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled) throws IOException, + URISyntaxException { Schema schema = SchemaTestUtil.getSimpleSchema(); @@ -337,9 +342,34 @@ record = records.get(key); assertEquals(gRecord.get(fieldName).toString(), newValue); } - // TODO : come up with a performance eval test for spillableMap @Test - public void testLargeInsertUpsert() {} + public void testEstimationWithEmptyMap() throws IOException, URISyntaxException { + final ExternalSpillableMap.DiskMapType diskMapType = ExternalSpillableMap.DiskMapType.BITCASK; + final boolean isCompressionEnabled = false; + final Schema schema = SchemaTestUtil.getSimpleSchema(); + + ExternalSpillableMap> records = + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType, isCompressionEnabled); + + List recordKeys = new ArrayList<>(); + + // Put a single record. Payload size estimation happens as part of this initial put. + HoodieRecord seedRecord = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1).get(0); + records.put(seedRecord.getRecordKey(), seedRecord); + + // Remove the key immediately to make the map empty again. + records.remove(seedRecord.getRecordKey()); + + // Verify payload size re-estimation does not throw exception + List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 250); + hoodieRecords.stream().forEach(hoodieRecord -> { + assertDoesNotThrow(() -> { + records.put(hoodieRecord.getRecordKey(), hoodieRecord); + }, "ExternalSpillableMap put() should not throw exception!"); + recordKeys.add(hoodieRecord.getRecordKey()); + }); + } private static Stream testArguments() { // Arguments : 1. Disk Map Type 2. isCompressionEnabled for BitCaskMap diff --git a/hudi-common/src/test/resources/external-config/hudi-defaults.conf b/hudi-common/src/test/resources/external-config/hudi-defaults.conf new file mode 100644 index 000000000000..1133adb4d773 --- /dev/null +++ b/hudi-common/src/test/resources/external-config/hudi-defaults.conf @@ -0,0 +1,26 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 +hoodie.datasource.hive_sync.use_jdbc true +hoodie.datasource.hive_sync.support_timestamp false +hoodie.index.type BLOOM +hoodie.metadata.enable true diff --git a/hudi-common/src/test/resources/props/test.properties b/hudi-common/src/test/resources/props/test.properties new file mode 100644 index 000000000000..8e848aff79d3 --- /dev/null +++ b/hudi-common/src/test/resources/props/test.properties @@ -0,0 +1,18 @@ + +# 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. + +some.random.prop=123 \ No newline at end of file diff --git a/hudi-examples/README.md b/hudi-examples/README.md index dfaf5d788ed1..ee077c3f7865 100644 --- a/hudi-examples/README.md +++ b/hudi-examples/README.md @@ -36,7 +36,7 @@ To run the demo: 5.3 Run `bin/kafka-delta-streamer-example.sh` - 5.4 continuously write source data to the Kafka topic your configured with `hoodie.deltastreamer.source.kafka.topic` in `kafka-source.properties` + 5.4 Continuously write source data to the Kafka topic your configured with `hoodie.deltastreamer.source.kafka.topic` in `kafka-source.properties` 6. Some notes delta streamer demo: diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index 8e58acdda0af..2ea284f20320 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java index e385e476dd26..d11d2eddfa51 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java @@ -40,7 +40,7 @@ public class HoodieSparkBootstrapExample { public static void main(String[] args) throws Exception { if (args.length < 5) { - System.err.println("Usage: HoodieWriteClientExample "); + System.err.println("Usage: HoodieSparkBootstrapExample "); System.exit(1); } String recordKey = args[0]; diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml index 8885007748f5..c8fac38be5b1 100644 --- a/hudi-flink/pom.xml +++ b/hudi-flink/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 @@ -144,12 +144,6 @@ flink-hadoop-compatibility_${scala.binary.version} ${flink.version} - - org.apache.flink - flink-avro - ${flink.version} - provided - org.apache.flink flink-parquet_${scala.binary.version} diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 65a95ed7c2b0..77c3f15e54c4 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -196,6 +196,17 @@ private FlinkOptions() { .defaultValue(60)// default 1 minute .withDescription("Check interval for streaming read of SECOND, default 1 minute"); + // this option is experimental + public static final ConfigOption READ_STREAMING_SKIP_COMPACT = ConfigOptions + .key("read.streaming.skip_compaction") + .booleanType() + .defaultValue(false)// default read as batch + .withDescription("Whether to skip compaction instants for streaming read,\n" + + "there are two cases that this option can be used to avoid reading duplicates:\n" + + "1) you are definitely sure that the consumer reads faster than any compaction instants, " + + "usually with delta time compaction strategy that is long enough, for e.g, one week;\n" + + "2) changelog mode is enabled, this option is a solution to keep data integrity"); + public static final String START_COMMIT_EARLIEST = "earliest"; public static final ConfigOption READ_START_COMMIT = ConfigOptions .key("read.start-commit") @@ -406,7 +417,7 @@ private FlinkOptions() { + "For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance."); public static final ConfigOption WRITE_PARQUET_PAGE_SIZE = ConfigOptions - .key("hoodie.parquet.page.size") + .key("write.parquet.page.size") .intType() .defaultValue(1) .withDescription("Parquet page size. Page is the unit of read within a parquet file. " diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index 376b36e3dcd8..acb4af61110f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.table.format.FilePathUtils; import org.apache.flink.configuration.Configuration; @@ -92,4 +93,21 @@ public static boolean isDeltaTimeCompaction(Configuration conf) { final String strategy = conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toLowerCase(Locale.ROOT); return FlinkOptions.TIME_ELAPSED.equals(strategy) || FlinkOptions.NUM_OR_TIME.equals(strategy); } + + /** + * Returns whether the table is partitioned. + */ + public static boolean isPartitionedTable(Configuration conf) { + return FilePathUtils.extractPartitionKeys(conf).length > 0; + } + + /** + * Returns whether the source should emit changelog. + * + * @return true if the source is read as streaming with changelog mode enabled + */ + public static boolean emitChangelog(Configuration conf) { + return conf.getBoolean(FlinkOptions.READ_AS_STREAMING) + && conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED); + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java index 26ac9f3adf6f..195e430d0b1b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java @@ -60,7 +60,9 @@ public CleanFunction(Configuration conf) { public void open(Configuration parameters) throws Exception { super.open(parameters); if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) { - this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + // do not use the remote filesystem view because the async cleaning service + // local timeline is very probably to fall behind with the remote one. + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext(), false); this.executor = NonThrownExecutor.builder(LOG).build(); } } @@ -96,4 +98,11 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception { // no operation } + + @Override + public void close() throws Exception { + if (this.writeClient != null) { + this.writeClient.close(); + } + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 08a04e3f84be..d72eb12066b2 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -333,7 +333,6 @@ private void startInstant() { this.instant = instant; this.writeClient.startCommitWithTime(instant, tableState.commitAction); this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant); - this.writeClient.upgradeDowngrade(this.instant); LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant, this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE)); } @@ -360,6 +359,8 @@ private void initInstant(String instant) { } // starts a new instant startInstant(); + // upgrade downgrade + this.writeClient.upgradeDowngrade(this.instant); }, "initialize instant %s", instant); } @@ -376,6 +377,9 @@ private void handleEndInputEvent(WriteMetadataEvent event) { if (allEventsReceived()) { // start to commit the instant. commitInstant(this.instant); + // The executor thread inherits the classloader of the #handleEventFromOperator + // caller, which is a AppClassLoader. + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); // sync Hive if is enabled in batch mode. syncHiveIfEnabled(); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java index 090ed29b8e33..a72b885a22c7 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java @@ -30,7 +30,10 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.List; /** @@ -43,6 +46,7 @@ * @see StreamWriteOperatorCoordinator */ public class AppendWriteFunction extends AbstractStreamWriteFunction { + private static final Logger LOG = LoggerFactory.getLogger(AppendWriteFunction.class); private static final long serialVersionUID = 1L; @@ -113,14 +117,19 @@ this.currentInstant, this.taskID, getRuntimeContext().getNumberOfParallelSubtask } private void flushData(boolean endInput) { - if (this.writerHelper == null) { - // does not process any inputs, returns early. - return; + final List writeStatus; + final String instant; + if (this.writerHelper != null) { + writeStatus = this.writerHelper.getWriteStatuses(this.taskID); + instant = this.writerHelper.getInstantTime(); + } else { + LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); + writeStatus = Collections.emptyList(); + instant = instantToWrite(false); } - final List writeStatus = this.writerHelper.getWriteStatuses(this.taskID); final WriteMetadataEvent event = WriteMetadataEvent.builder() .taskID(taskID) - .instantTime(this.writerHelper.getInstantTime()) + .instantTime(instant) .writeStatus(writeStatus) .lastBatch(true) .endInput(endInput) diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index f6055ba11d2f..4832f18bf7f0 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -179,9 +179,6 @@ public void processElement(StreamRecord element) throws Exception { protected void loadRecords(String partitionPath) throws Exception { long start = System.currentTimeMillis(); - BaseFileUtils fileUtils = BaseFileUtils.getInstance(this.hoodieTable.getBaseFileFormat()); - Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema(); - final int parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); final int maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks(); final int taskID = getRuntimeContext().getIndexOfThisSubtask(); @@ -193,6 +190,9 @@ protected void loadRecords(String partitionPath) throws Exception { Option latestCommitTime = commitsTimeline.filterCompletedInstants().lastInstant(); if (latestCommitTime.isPresent()) { + BaseFileUtils fileUtils = BaseFileUtils.getInstance(this.hoodieTable.getBaseFileFormat()); + Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema(); + List fileSlices = this.hoodieTable.getSliceView() .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp(), true) .collect(toList()); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java index 0309278f483e..53127359cb98 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.CleanFunction; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.util.CompactionUtil; @@ -67,6 +68,12 @@ public class CompactionCommitSink extends CleanFunction { */ private transient Map> commitBuffer; + /** + * Cache to store compaction plan for each instant. + * Stores the mapping of instant_time -> compactionPlan. + */ + private transient Map compactionPlanCache; + /** * The hoodie table. */ @@ -84,6 +91,7 @@ public void open(Configuration parameters) throws Exception { this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); } this.commitBuffer = new HashMap<>(); + this.compactionPlanCache = new HashMap<>(); this.table = this.writeClient.getHoodieTable(); } @@ -108,8 +116,15 @@ public void invoke(CompactionCommitEvent event, Context context) throws Exceptio * @param events Commit events ever received for the instant */ private void commitIfNecessary(String instant, Collection events) throws IOException { - HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( - this.writeClient.getHoodieTable().getMetaClient(), instant); + HoodieCompactionPlan compactionPlan = compactionPlanCache.computeIfAbsent(instant, k -> { + try { + return CompactionUtils.getCompactionPlan( + this.writeClient.getHoodieTable().getMetaClient(), instant); + } catch (IOException e) { + throw new HoodieException(e); + } + }); + boolean isReady = compactionPlan.getOperations().size() == events.size(); if (!isReady) { return; @@ -143,5 +158,6 @@ private void doCommit(String instant, Collection events) private void reset(String instant) { this.commitBuffer.remove(instant); + this.compactionPlanCache.remove(instant); } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java index f6dd241ec069..6df11fe2242f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java @@ -98,7 +98,7 @@ public void notifyCheckpointComplete(long checkpointId) { } private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) throws IOException { - // the last instant takes the highest priority. + // the first instant takes the highest priority. Option firstRequested = table.getActiveTimeline().filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).firstInstant(); if (!firstRequested.isPresent()) { diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java index d8f96dbe3cf2..4f3faadb92f0 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java @@ -102,6 +102,13 @@ public class FlinkCompactionConfig extends Configuration { + "2). LIFO: execute the latest plan first, by default LIFO", required = false) public String compactionSeq = SEQ_LIFO; + @Parameter(names = {"--service"}, description = "Flink Compaction runs in service mode, disable by default") + public Boolean serviceMode = false; + + @Parameter(names = {"--min-compaction-interval-seconds"}, + description = "Min compaction interval of async compaction service, default 10 minutes") + public Integer minCompactionIntervalSeconds = 600; + /** * Transforms a {@code HoodieFlinkCompaction.config} into {@code Configuration}. * The latter is more suitable for the table APIs. It reads all the properties diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index a22bea9f31b6..a6161f2c88cf 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.compact; +import org.apache.hudi.async.HoodieAsyncService; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -25,12 +26,15 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.util.CompactionUtil; import org.apache.hudi.util.StreamerUtil; import com.beust.jcommander.JCommander; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -38,6 +42,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + /** * Flink hudi compaction program that can be executed manually. */ @@ -45,114 +53,252 @@ public class HoodieFlinkCompactor { protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCompactor.class); + /** + * Flink Execution Environment. + */ + private final AsyncCompactionService compactionScheduleService; + + public HoodieFlinkCompactor(AsyncCompactionService service) { + this.compactionScheduleService = service; + } + public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + FlinkCompactionConfig cfg = getFlinkCompactionConfig(args); + Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg); + + AsyncCompactionService service = new AsyncCompactionService(cfg, conf, env); + + new HoodieFlinkCompactor(service).start(cfg.serviceMode); + } + + /** + * Main method to start compaction service. + */ + public void start(boolean serviceMode) throws Exception { + if (serviceMode) { + compactionScheduleService.start(null); + try { + compactionScheduleService.waitForShutdown(); + } catch (Exception e) { + throw new HoodieException(e.getMessage(), e); + } finally { + LOG.info("Shut down hoodie flink compactor"); + } + } else { + LOG.info("Hoodie Flink Compactor running only single round"); + try { + compactionScheduleService.compact(); + } catch (Exception e) { + LOG.error("Got error running delta sync once. Shutting down", e); + throw e; + } finally { + LOG.info("Shut down hoodie flink compactor"); + } + } + } + + public static FlinkCompactionConfig getFlinkCompactionConfig(String[] args) { FlinkCompactionConfig cfg = new FlinkCompactionConfig(); JCommander cmd = new JCommander(cfg, null, args); if (cfg.help || args.length == 0) { cmd.usage(); System.exit(1); } + return cfg; + } - Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg); + // ------------------------------------------------------------------------- + // Inner Class + // ------------------------------------------------------------------------- + + /** + * Schedules compaction in service. + */ + public static class AsyncCompactionService extends HoodieAsyncService { + private static final long serialVersionUID = 1L; + + /** + * Flink Compaction Config. + */ + private final FlinkCompactionConfig cfg; + + /** + * Flink Config. + */ + private final Configuration conf; + + /** + * Meta Client. + */ + private final HoodieTableMetaClient metaClient; + + /** + * Write Client. + */ + private final HoodieFlinkWriteClient writeClient; + + /** + * The hoodie table. + */ + private final HoodieFlinkTable table; + + /** + * Flink Execution Environment. + */ + private final StreamExecutionEnvironment env; + + /** + * Executor Service. + */ + private final ExecutorService executor; + + public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception { + this.cfg = cfg; + this.conf = conf; + this.env = env; + this.executor = Executors.newFixedThreadPool(1); - // create metaClient - HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + // create metaClient + this.metaClient = StreamerUtil.createMetaClient(conf); - // get the table name - conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + // get the table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); - // set table schema - CompactionUtil.setAvroSchema(conf, metaClient); + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); - // infer changelog mode - CompactionUtil.inferChangelogMode(conf, metaClient); + // infer changelog mode + CompactionUtil.inferChangelogMode(conf, metaClient); - HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf); - HoodieFlinkTable table = writeClient.getHoodieTable(); + this.writeClient = StreamerUtil.createWriteClient(conf); + this.table = writeClient.getHoodieTable(); + } + + @Override + protected Pair startService() { + return Pair.of(CompletableFuture.supplyAsync(() -> { + boolean error = false; + + try { + while (!isShutdownRequested()) { + try { + compact(); + Thread.sleep(cfg.minCompactionIntervalSeconds * 1000); + } catch (Exception e) { + LOG.error("Shutting down compaction service due to exception", e); + error = true; + throw new HoodieException(e.getMessage(), e); + } + } + } finally { + shutdownAsyncService(error); + } + return true; + }, executor), executor); + } + + private void compact() throws Exception { + table.getMetaClient().reloadActiveTimeline(); - // judge whether have operation - // to compute the compaction instant time and do compaction. - if (cfg.schedule) { - Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); - if (compactionInstantTimeOption.isPresent()) { - boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); - if (!scheduled) { - // do nothing. - LOG.info("No compaction plan for this job "); - return; + // checks the compaction plan and do compaction. + if (cfg.schedule) { + Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); + if (compactionInstantTimeOption.isPresent()) { + boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); + if (!scheduled) { + // do nothing. + LOG.info("No compaction plan for this job "); + return; + } + table.getMetaClient().reloadActiveTimeline(); } + } + + // fetch the instant based on the configured execution sequence + HoodieTimeline timeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + Option requested = CompactionUtil.isLIFO(cfg.compactionSeq) ? timeline.lastInstant() : timeline.firstInstant(); + if (!requested.isPresent()) { + // do nothing. + LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option"); + return; + } + + String compactionInstantTime = requested.get().getTimestamp(); + + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (timeline.containsInstant(inflightInstant)) { + LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]"); + table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); } - } - // fetch the instant based on the configured execution sequence - HoodieTimeline timeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - Option requested = CompactionUtil.isLIFO(cfg.compactionSeq) ? timeline.lastInstant() : timeline.firstInstant(); - if (!requested.isPresent()) { - // do nothing. - LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option"); - return; - } + // generate compaction plan + // should support configurable commit metadata + HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( + table.getMetaClient(), compactionInstantTime); - String compactionInstantTime = requested.get().getTimestamp(); + if (compactionPlan == null || (compactionPlan.getOperations() == null) + || (compactionPlan.getOperations().isEmpty())) { + // No compaction plan, do nothing and return. + LOG.info("No compaction plan for instant " + compactionInstantTime); + return; + } - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); - if (timeline.containsInstant(inflightInstant)) { - LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]"); - table.rollbackInflightCompaction(inflightInstant); - table.getMetaClient().reloadActiveTimeline(); - } + HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + if (!pendingCompactionTimeline.containsInstant(instant)) { + // this means that the compaction plan was written to auxiliary path(.tmp) + // but not the meta path(.hoodie), this usually happens when the job crush + // exceptionally. - // generate compaction plan - // should support configurable commit metadata - HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( - table.getMetaClient(), compactionInstantTime); + // clean the compaction plan in auxiliary path and cancels the compaction. - if (compactionPlan == null || (compactionPlan.getOperations() == null) - || (compactionPlan.getOperations().isEmpty())) { - // No compaction plan, do nothing and return. - LOG.info("No compaction plan for instant " + compactionInstantTime); - return; - } + LOG.warn("The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n" + + "Clean the compaction plan in auxiliary path and cancels the compaction"); + CompactionUtil.cleanInstant(table.getMetaClient(), instant); + return; + } - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - if (!pendingCompactionTimeline.containsInstant(instant)) { - // this means that the compaction plan was written to auxiliary path(.tmp) - // but not the meta path(.hoodie), this usually happens when the job crush - // exceptionally. + // get compactionParallelism. + int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1 + ? compactionPlan.getOperations().size() : conf.getInteger(FlinkOptions.COMPACTION_TASKS); - // clean the compaction plan in auxiliary path and cancels the compaction. + LOG.info("Start to compaction for instant " + compactionInstantTime); - LOG.warn("The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n" - + "Clean the compaction plan in auxiliary path and cancels the compaction"); - CompactionUtil.cleanInstant(table.getMetaClient(), instant); - return; + // Mark instant as compaction inflight + table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); + table.getMetaClient().reloadActiveTimeline(); + + env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime)) + .name("compaction_source") + .uid("uid_compaction_source") + .rebalance() + .transform("compact_task", + TypeInformation.of(CompactionCommitEvent.class), + new ProcessOperator<>(new CompactFunction(conf))) + .setParallelism(compactionParallelism) + .addSink(new CompactionCommitSink(conf)) + .name("clean_commits") + .uid("uid_clean_commits") + .setParallelism(1); + + env.execute("flink_hudi_compaction_" + compactionInstantTime); } - // get compactionParallelism. - int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1 - ? compactionPlan.getOperations().size() : conf.getInteger(FlinkOptions.COMPACTION_TASKS); - - // Mark instant as compaction inflight - table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); - - env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime)) - .name("compaction_source") - .uid("uid_compaction_source") - .rebalance() - .transform("compact_task", - TypeInformation.of(CompactionCommitEvent.class), - new ProcessOperator<>(new CompactFunction(conf))) - .setParallelism(compactionParallelism) - .addSink(new CompactionCommitSink(conf)) - .name("clean_commits") - .uid("uid_clean_commits") - .setParallelism(1); - - env.execute("flink_hudi_compaction"); - writeClient.close(); + /** + * Shutdown async services like compaction/clustering as DeltaSync is shutdown. + */ + public void shutdownAsyncService(boolean error) { + LOG.info("Gracefully shutting down compactor. Error ?" + error); + executor.shutdown(); + writeClient.close(); + } + + @VisibleForTesting + public void shutDown() { + shutdownAsyncService(false); + } } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java index 33f1dd6204bc..73fd6685539f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java @@ -121,16 +121,16 @@ public void open(Configuration parameters) throws Exception { getRuntimeContext().getIndexOfThisSubtask(), getRuntimeContext().getMaxNumberOfParallelSubtasks(), getRuntimeContext().getNumberOfParallelSubtasks(), - ignoreSmallFiles(writeConfig), + ignoreSmallFiles(), HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)), context, writeConfig); this.payloadCreation = PayloadCreation.instance(this.conf); } - private boolean ignoreSmallFiles(HoodieWriteConfig writeConfig) { + private boolean ignoreSmallFiles() { WriteOperationType operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)); - return WriteOperationType.isOverwrite(operationType) || writeConfig.allowDuplicateInserts(); + return WriteOperationType.isOverwrite(operationType); } @Override diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java index f9d5b1c1faa9..e73890f3b54b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java @@ -22,7 +22,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.sink.partitioner.profile.WriteProfile; import org.apache.hudi.sink.partitioner.profile.WriteProfiles; -import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BucketInfo; import org.apache.hudi.table.action.commit.BucketType; import org.apache.hudi.table.action.commit.SmallFile; @@ -188,6 +187,7 @@ private synchronized SmallFileAssign getSmallFileAssign(String partitionPath) { smallFileAssignMap.put(partitionPath, assign); return assign; } + smallFileAssignMap.put(partitionPath, null); return null; } @@ -211,10 +211,6 @@ public synchronized void reload(long checkpointId) { this.writeProfile.reload(checkpointId); } - public HoodieTable getTable() { - return this.writeProfile.getTable(); - } - private boolean fileIdOfThisTask(String fileId) { // the file id can shuffle to this task return KeyGroupRangeAssignment.assignKeyToParallelOperator(fileId, maxParallelism, numTasks) == taskID; diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java index 922c056d259d..97b6b238814c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.AbstractTableFileSystemView; +import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.action.commit.SmallFile; @@ -50,14 +50,12 @@ protected List smallFilesProfile(String partitionPath) { List smallFileLocations = new ArrayList<>(); // Init here since this class (and member variables) might not have been initialized - HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); + HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); // Find out all eligible small file slices if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - // initialize the filesystem view based on the commit metadata - initFileSystemView(); - // find smallest file in partition and append to it + // find the smallest file in partition and append to it List allSmallFileSlices = new ArrayList<>(); // If we can index log files, we can add more inserts to log files for fileIds including those under // pending compaction. @@ -91,8 +89,8 @@ protected List smallFilesProfile(String partitionPath) { return smallFileLocations; } - protected AbstractTableFileSystemView getFileSystemView() { - return (AbstractTableFileSystemView) this.table.getSliceView(); + protected SyncableFileSystemView getFileSystemView() { + return (SyncableFileSystemView) getTable().getSliceView(); } private long getTotalFileSize(FileSlice fileSlice) { diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java index 1171a54cde92..84fcd03f0833 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java @@ -23,14 +23,16 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.AbstractTableFileSystemView; +import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.sink.partitioner.BucketAssigner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.SmallFile; +import org.apache.hudi.util.StreamerUtil; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; @@ -66,9 +68,9 @@ public class WriteProfile { private final Path basePath; /** - * The hoodie table. + * The meta client. */ - protected final HoodieTable table; + protected final HoodieTableMetaClient metaClient; /** * The average record size. @@ -94,20 +96,27 @@ public class WriteProfile { /** * The file system view cache for one checkpoint interval. */ - protected AbstractTableFileSystemView fsView; + protected SyncableFileSystemView fsView; /** * Metadata cache to reduce IO of metadata files. */ private final Map metadataCache; + /** + * The engine context. + */ + private final HoodieFlinkEngineContext context; + public WriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context) { this.config = config; + this.context = context; this.basePath = new Path(config.getBasePath()); this.smallFilesMap = new HashMap<>(); this.recordsPerBucket = config.getCopyOnWriteInsertSplitSize(); - this.table = HoodieFlinkTable.create(config, context); + this.metaClient = StreamerUtil.createMetaClient(config.getBasePath(), context.getHadoopConf().get()); this.metadataCache = new HashMap<>(); + this.fsView = getFileSystemView(); // profile the record statistics on construction recordProfile(); } @@ -120,8 +129,12 @@ public long getRecordsPerBucket() { return recordsPerBucket; } - public HoodieTable getTable() { - return table; + public HoodieTableMetaClient getMetaClient() { + return this.metaClient; + } + + protected HoodieTable getTable() { + return HoodieFlinkTable.create(config, context); } /** @@ -131,7 +144,7 @@ public long getRecordsPerBucket() { private long averageBytesPerRecord() { long avgSize = config.getCopyOnWriteRecordSizeEstimate(); long fileSizeThreshold = (long) (config.getRecordSizeEstimationThreshold() * config.getParquetSmallFileLimit()); - HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); if (!commitTimeline.empty()) { // Go over the reverse ordered commits to get a more recent estimate of average record size. Iterator instants = commitTimeline.getReverseOrderedInstants().iterator(); @@ -179,12 +192,10 @@ protected List smallFilesProfile(String partitionPath) { // smallFiles only for partitionPath List smallFileLocations = new ArrayList<>(); - HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - // initialize the filesystem view based on the commit metadata - initFileSystemView(); List allFiles = fsView .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); @@ -203,15 +214,8 @@ protected List smallFilesProfile(String partitionPath) { return smallFileLocations; } - @VisibleForTesting - public void initFileSystemView() { - if (fsView == null) { - fsView = getFileSystemView(); - } - } - - protected AbstractTableFileSystemView getFileSystemView() { - return (AbstractTableFileSystemView) this.table.getBaseFileOnlyView(); + protected SyncableFileSystemView getFileSystemView() { + return (SyncableFileSystemView) getTable().getBaseFileOnlyView(); } /** @@ -244,11 +248,10 @@ public synchronized void reload(long checkpointId) { // already reloaded return; } - this.table.getMetaClient().reloadActiveTimeline(); - this.table.getHoodieView().sync(); + this.metaClient.reloadActiveTimeline(); + this.fsView.sync(); recordProfile(); - this.fsView = null; - cleanMetadataCache(this.table.getMetaClient().getCommitsTimeline().filterCompletedInstants().getInstants()); + cleanMetadataCache(this.metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()); this.smallFilesMap.clear(); this.reloadedCheckpointId = checkpointId; } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 855a2ca1b4d0..5f156e839f1e 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.sink.CleanFunction; import org.apache.hudi.sink.StreamWriteOperator; import org.apache.hudi.sink.append.AppendWriteOperator; @@ -129,10 +130,10 @@ public static DataStream bootstrap( final boolean globalIndex = conf.getBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED); if (overwrite) { return rowDataToHoodieRecord(conf, rowType, dataStream); - } else if (bounded && !globalIndex) { + } else if (bounded && !globalIndex && OptionsResolver.isPartitionedTable(conf)) { return boundedBootstrap(conf, rowType, defaultParallelism, dataStream); } else { - return streamBootstrap(conf, rowType, defaultParallelism, dataStream); + return streamBootstrap(conf, rowType, defaultParallelism, dataStream, bounded); } } @@ -140,10 +141,11 @@ private static DataStream streamBootstrap( Configuration conf, RowType rowType, int defaultParallelism, - DataStream dataStream) { + DataStream dataStream, + boolean bounded) { DataStream dataStream1 = rowDataToHoodieRecord(conf, rowType, dataStream); - if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { + if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) { dataStream1 = dataStream1 .transform( "index_bootstrap", @@ -161,13 +163,10 @@ private static DataStream boundedBootstrap( RowType rowType, int defaultParallelism, DataStream dataStream) { - final String[] partitionFields = FilePathUtils.extractPartitionKeys(conf); - if (partitionFields.length > 0) { - RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType); - // shuffle by partition keys - dataStream = dataStream - .keyBy(rowDataKeyGen::getPartitionPath); - } + final RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType); + // shuffle by partition keys + dataStream = dataStream + .keyBy(rowDataKeyGen::getPartitionPath); return rowDataToHoodieRecord(conf, rowType, dataStream) .transform( diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java index 653e182bfad9..58c38ef56744 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java +++ b/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java @@ -44,6 +44,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; @@ -79,16 +80,20 @@ public class IncrementalInputSplits implements Serializable { private final long maxCompactionMemoryInBytes; // for partition pruning private final Set requiredPartitions; + // skip compaction + private final boolean skipCompaction; private IncrementalInputSplits( Configuration conf, Path path, long maxCompactionMemoryInBytes, - @Nullable Set requiredPartitions) { + @Nullable Set requiredPartitions, + boolean skipCompaction) { this.conf = conf; this.path = path; this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes; this.requiredPartitions = requiredPartitions; + this.skipCompaction = skipCompaction; } /** @@ -156,28 +161,52 @@ public Result inputSplits( } String tableName = conf.getString(FlinkOptions.TABLE_NAME); - List activeMetadataList = instants.stream() - .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList()); - List archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName); - if (archivedMetadataList.size() > 0) { - LOG.warn("\n" - + "--------------------------------------------------------------------------------\n" - + "---------- caution: the reader has fall behind too much from the writer,\n" - + "---------- tweak 'read.tasks' option to add parallelism of read tasks.\n" - + "--------------------------------------------------------------------------------"); - } - List metadataList = archivedMetadataList.size() > 0 - // IMPORTANT: the merged metadata list must be in ascending order by instant time - ? mergeList(archivedMetadataList, activeMetadataList) - : activeMetadataList; - - Set writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList); - // apply partition push down - if (this.requiredPartitions != null) { - writePartitions = writePartitions.stream() - .filter(this.requiredPartitions::contains).collect(Collectors.toSet()); + + Set writePartitions; + final FileStatus[] fileStatuses; + + if (instantRange == null) { + // reading from the earliest, scans the partitions and files directly. + FileIndex fileIndex = FileIndex.instance(new org.apache.hadoop.fs.Path(path.toUri()), conf); + if (this.requiredPartitions != null) { + // apply partition push down + fileIndex.setPartitionPaths(this.requiredPartitions); + } + writePartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths()); + if (writePartitions.size() == 0) { + LOG.warn("No partitions found for reading in user provided path."); + return Result.EMPTY; + } + fileStatuses = fileIndex.getFilesInPartitions(); + } else { + List activeMetadataList = instants.stream() + .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList()); + List archivedMetadataList = getArchivedMetadata(metaClient, instantRange, commitTimeline, tableName); + if (archivedMetadataList.size() > 0) { + LOG.warn("\n" + + "--------------------------------------------------------------------------------\n" + + "---------- caution: the reader has fall behind too much from the writer,\n" + + "---------- tweak 'read.tasks' option to add parallelism of read tasks.\n" + + "--------------------------------------------------------------------------------"); + } + List metadataList = archivedMetadataList.size() > 0 + // IMPORTANT: the merged metadata list must be in ascending order by instant time + ? mergeList(archivedMetadataList, activeMetadataList) + : activeMetadataList; + + writePartitions = HoodieInputFormatUtils.getWritePartitionPaths(metadataList); + // apply partition push down + if (this.requiredPartitions != null) { + writePartitions = writePartitions.stream() + .filter(this.requiredPartitions::contains).collect(Collectors.toSet()); + } + if (writePartitions.size() == 0) { + LOG.warn("No partitions found for reading in user provided path."); + return Result.EMPTY; + } + fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType()); } - FileStatus[] fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList, metaClient.getTableType()); + if (fileStatuses.length == 0) { LOG.warn("No files found for reading in user provided path."); return Result.EMPTY; @@ -237,7 +266,7 @@ private List getArchivedMetadata( final String startTs = archivedCompleteTimeline.firstInstant().get().getTimestamp(); archivedTimeline.loadInstantDetailsInMemory(startTs, endTs); } - return instantStream + return maySkipCompaction(instantStream) .map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, archivedTimeline)).collect(Collectors.toList()); } } @@ -257,7 +286,7 @@ private List filterInstantsWithRange( HoodieTimeline completedTimeline = commitTimeline.filterCompletedInstants(); if (issuedInstant != null) { // returns early for streaming mode - return completedTimeline.getInstants() + return maySkipCompaction(completedTimeline.getInstants()) .filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, issuedInstant)) .collect(Collectors.toList()); } @@ -274,7 +303,13 @@ private List filterInstantsWithRange( final String endCommit = this.conf.get(FlinkOptions.READ_END_COMMIT); instantStream = instantStream.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, endCommit)); } - return instantStream.collect(Collectors.toList()); + return maySkipCompaction(instantStream).collect(Collectors.toList()); + } + + private Stream maySkipCompaction(Stream instants) { + return this.skipCompaction + ? instants.filter(instant -> !instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)) + : instants; } private static List mergeList(List list1, List list2) { @@ -327,6 +362,8 @@ public static class Builder { private long maxCompactionMemoryInBytes; // for partition pruning private Set requiredPartitions; + // skip compaction + private boolean skipCompaction = false; public Builder() { } @@ -351,9 +388,14 @@ public Builder requiredPartitions(@Nullable Set requiredPartitions) { return this; } + public Builder skipCompaction(boolean skipCompaction) { + this.skipCompaction = skipCompaction; + return this; + } + public IncrementalInputSplits build() { return new IncrementalInputSplits(Objects.requireNonNull(this.conf), Objects.requireNonNull(this.path), - this.maxCompactionMemoryInBytes, this.requiredPartitions); + this.maxCompactionMemoryInBytes, this.requiredPartitions, this.skipCompaction); } } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java b/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java index c7bcc399ebc9..8138e931e54e 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java @@ -107,7 +107,9 @@ public StreamReadMonitoringFunction( .conf(conf) .path(path) .maxCompactionMemoryInBytes(maxCompactionMemoryInBytes) - .requiredPartitions(requiredPartitionPaths).build(); + .requiredPartitions(requiredPartitionPaths) + .skipCompaction(conf.getBoolean(FlinkOptions.READ_STREAMING_SKIP_COMPACT)) + .build(); } @Override diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index de2218acb0b9..1d7111f495c5 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -365,7 +365,9 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setDouble(FlinkOptions.INDEX_STATE_TTL, config.indexStateTtl); conf.setBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED, config.indexGlobalEnabled); conf.setString(FlinkOptions.INDEX_PARTITION_REGEX, config.indexPartitionRegex); - conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, config.sourceAvroSchemaPath); + if (!StringUtils.isNullOrEmpty(config.sourceAvroSchemaPath)) { + conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, config.sourceAvroSchemaPath); + } conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, config.sourceAvroSchema); conf.setBoolean(FlinkOptions.UTC_TIMEZONE, config.utcTimezone); conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode); diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java index 3e567f31fa63..851931f0d76b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java @@ -18,6 +18,8 @@ package org.apache.hudi.streamer; +import org.apache.hudi.common.config.DFSPropertiesConfiguration; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; @@ -38,8 +40,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; -import java.util.Properties; - /** * A utility which can incrementally consume data from Kafka and apply it to the target table. * It has the similar functionality with SQL data source except that the source is bind to Kafka @@ -65,7 +65,8 @@ public static void main(String[] args) throws Exception { env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); } - Properties kafkaProps = StreamerUtil.appendKafkaProps(cfg); + TypedProperties kafkaProps = DFSPropertiesConfiguration.getGlobalProps(); + kafkaProps.putAll(StreamerUtil.appendKafkaProps(cfg)); // Read from kafka source RowType rowType = diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index f0dbffd4732f..259c2e40cd47 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.HoodieROTablePathFilter; import org.apache.hudi.source.FileIndex; @@ -196,11 +197,9 @@ public DataStream produceDataStream(StreamExecutionEnvironment execEnv) @Override public ChangelogMode getChangelogMode() { - return conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED) - ? ChangelogModes.FULL - // when all the changes are persisted or read as batch, - // use INSERT mode. - : ChangelogMode.insertOnly(); + // when read as streaming and changelog mode is enabled, emit as FULL mode; + // when all the changes are compacted or read as batch, emit as INSERT mode. + return OptionsResolver.emitChangelog(conf) ? ChangelogModes.FULL : ChangelogMode.insertOnly(); } @Override @@ -304,8 +303,8 @@ private List buildFileIndex() { } HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(), fileStatuses); + // file-slice after pending compaction-requested instant-time is also considered valid + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), fileStatuses); String latestCommit = fsView.getLastInstant().get().getTimestamp(); final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE); final AtomicInteger cnt = new AtomicInteger(0); diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java new file mode 100644 index 000000000000..842f2757af39 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java @@ -0,0 +1,56 @@ +/* + * 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 org.apache.hudi.table.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.catalog.CommonCatalogOptions; + +import java.util.Map; + +/** + * Hoodie catalog options. + */ +public class CatalogOptions { + + public static final ConfigOption CATALOG_PATH = + ConfigOptions.key("catalog.path") + .stringType() + .noDefaultValue() + .withDescription("Catalog base DFS path, used for inferring the sink table path. " + + "The default strategy for a table path is: ${catalog.path}/${db_name}/${table_name}"); + + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue("default"); + + /** + * Returns all the common table options that can be shared. + * + * @param catalogOptions The catalog options + */ + public static Map tableCommonOptions(Configuration catalogOptions) { + Configuration copied = new Configuration(catalogOptions); + copied.removeConfig(DEFAULT_DATABASE); + copied.removeConfig(CATALOG_PATH); + return copied.toMap(); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java new file mode 100644 index 000000000000..3ad3deab817a --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java @@ -0,0 +1,508 @@ +/* + * 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 org.apache.hudi.table.catalog; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.avro.Schema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.StringUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; +import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; + +/** + * Catalog that can set up common options for underneath table. + */ +public class HoodieCatalog extends AbstractCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalog.class); + + private final org.apache.hadoop.conf.Configuration hadoopConf; + private final String catalogPathStr; + private final Map tableCommonOptions; + + private Path catalogPath; + private FileSystem fs; + + public HoodieCatalog(String name, Configuration options) { + super(name, options.get(DEFAULT_DATABASE)); + this.catalogPathStr = options.get(CATALOG_PATH); + this.hadoopConf = StreamerUtil.getHadoopConf(); + this.tableCommonOptions = CatalogOptions.tableCommonOptions(options); + } + + @Override + public void open() throws CatalogException { + fs = FSUtils.getFs(catalogPathStr, hadoopConf); + catalogPath = new Path(catalogPathStr); + try { + if (!fs.exists(catalogPath)) { + throw new CatalogException(String.format("Catalog %s path %s does not exist.", getName(), catalogPathStr)); + } + } catch (IOException e) { + throw new CatalogException(String.format("Checking catalog path %s exists exception.", catalogPathStr), e); + } + } + + @Override + public void close() throws CatalogException { + try { + fs.close(); + } catch (IOException e) { + throw new CatalogException("Closing FileSystem exception.", e); + } + } + + // ------ databases ------ + + @Override + public List listDatabases() throws CatalogException { + try { + FileStatus[] fileStatuses = fs.listStatus(catalogPath); + return Arrays.stream(fileStatuses) + .filter(FileStatus::isDirectory) + .map(fileStatus -> fileStatus.getPath().getName()) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new CatalogException("Listing database exception.", e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException { + if (databaseExists(databaseName)) { + return new CatalogDatabaseImpl(Collections.emptyMap(), null); + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName)); + + return listDatabases().contains(databaseName); + } + + @Override + public void createDatabase(String databaseName, CatalogDatabase catalogDatabase, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (databaseExists(databaseName)) { + if (ignoreIfExists) { + return; + } else { + throw new DatabaseAlreadyExistException(getName(), databaseName); + } + } + + if (!CollectionUtil.isNullOrEmpty(catalogDatabase.getProperties())) { + throw new CatalogException("Hudi catalog doesn't support to create database with options."); + } + + Path dbPath = new Path(catalogPath, databaseName); + try { + fs.mkdirs(dbPath); + } catch (IOException e) { + throw new CatalogException(String.format("Creating database %s exception.", databaseName), e); + } + } + + @Override + public void dropDatabase(String databaseName, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + if (!databaseExists(databaseName)) { + if (ignoreIfNotExists) { + return; + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + List tables = listTables(databaseName); + if (!tables.isEmpty() && !cascade) { + throw new DatabaseNotEmptyException(getName(), databaseName); + } + + if (databaseName.equals(getDefaultDatabase())) { + throw new IllegalArgumentException( + "Hudi catalog doesn't support to drop the default database."); + } + + Path dbPath = new Path(catalogPath, databaseName); + try { + fs.delete(dbPath, true); + } catch (IOException e) { + throw new CatalogException(String.format("Dropping database %s exception.", databaseName), e); + } + } + + @Override + public void alterDatabase(String databaseName, CatalogDatabase catalogDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException("Altering database is not implemented."); + } + + // ------ tables ------ + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException, CatalogException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + Path dbPath = new Path(catalogPath, databaseName); + try { + return Arrays.stream(fs.listStatus(dbPath)) + .filter(FileStatus::isDirectory) + .map(fileStatus -> fileStatus.getPath().getName()) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new CatalogException(String.format("Listing table in database %s exception.", dbPath), e); + } + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + + final String path = inferTablePath(catalogPathStr, tablePath); + Map options = TableOptionProperties.loadFromProperties(path, hadoopConf); + final Schema latestSchema = getLatestTableSchema(path); + if (latestSchema != null) { + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder() + .fromRowDataType(AvroSchemaConverter.convertToDataType(latestSchema)); + final String pkConstraintName = TableOptionProperties.getPkConstraintName(options); + if (pkConstraintName != null) { + builder.primaryKeyNamed(pkConstraintName, TableOptionProperties.getPkColumns(options)); + } + final org.apache.flink.table.api.Schema schema = builder.build(); + return CatalogTable.of( + schema, + TableOptionProperties.getComment(options), + TableOptionProperties.getPartitionColumns(options), + TableOptionProperties.getTableOptions(options)); + } else { + throw new TableNotExistException(getName(), tablePath); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + if (tableExists(tablePath)) { + if (ignoreIfExists) { + return; + } else { + throw new TableAlreadyExistException(getName(), tablePath); + } + } + + if (catalogTable instanceof CatalogView) { + throw new UnsupportedOperationException( + "Hudi catalog doesn't support to CREATE VIEW."); + } + + ResolvedCatalogTable resolvedTable = (ResolvedCatalogTable) catalogTable; + final String tablePathStr = inferTablePath(catalogPathStr, tablePath); + Map options = applyOptionsHook(tablePathStr, catalogTable.getOptions()); + Configuration conf = Configuration.fromMap(options); + conf.setString(FlinkOptions.PATH, tablePathStr); + ResolvedSchema resolvedSchema = resolvedTable.getResolvedSchema(); + if (!resolvedSchema.getPrimaryKey().isPresent()) { + throw new CatalogException("Primary key definition is missing"); + } + final String avroSchema = AvroSchemaConverter.convertToSchema(resolvedSchema.toPhysicalRowDataType().getLogicalType()).toString(); + conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema); + + // stores two copies of options: + // - partition keys + // - primary keys + // because the HoodieTableMetaClient is a heavy impl, we try to avoid initializing it + // when calling #getTable. + + final String pkColumns = String.join(",", resolvedSchema.getPrimaryKey().get().getColumns()); + conf.setString(FlinkOptions.RECORD_KEY_FIELD, pkColumns); + options.put(TableOptionProperties.PK_CONSTRAINT_NAME, resolvedSchema.getPrimaryKey().get().getName()); + options.put(TableOptionProperties.PK_COLUMNS, pkColumns); + + if (resolvedTable.isPartitioned()) { + final String partitions = String.join(",", resolvedTable.getPartitionKeys()); + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions); + options.put(TableOptionProperties.PARTITION_COLUMNS, partitions); + } + conf.setString(FlinkOptions.TABLE_NAME, tablePath.getObjectName()); + try { + StreamerUtil.initTableIfNotExists(conf); + // prepare the non-table-options properties + options.put(TableOptionProperties.COMMENT, resolvedTable.getComment()); + TableOptionProperties.createProperties(tablePathStr, hadoopConf, options); + } catch (IOException e) { + throw new CatalogException(String.format("Initialize table path %s exception.", tablePathStr), e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + return StreamerUtil.tableExists(inferTablePath(catalogPathStr, tablePath), hadoopConf); + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + if (!tableExists(tablePath)) { + if (ignoreIfNotExists) { + return; + } else { + throw new TableNotExistException(getName(), tablePath); + } + } + + Path path = new Path(inferTablePath(catalogPathStr, tablePath)); + try { + this.fs.delete(path, true); + } catch (IOException e) { + throw new CatalogException(String.format("Dropping table %s exception.", tablePath), e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + throw new UnsupportedOperationException("renameTable is not implemented."); + } + + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable catalogBaseTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException("alterTable is not implemented."); + } + + @Override + public List listViews(String databaseName) throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) + throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitionsByFilter(ObjectPath tablePath, List filters) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) + throws PartitionNotExistException, CatalogException { + return null; + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) throws CatalogException { + return false; + } + + @Override + public void createPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogPartition catalogPartition, boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, PartitionAlreadyExistsException, CatalogException { + throw new UnsupportedOperationException("createPartition is not implemented."); + } + + @Override + public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("dropPartition is not implemented."); + } + + @Override + public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogPartition catalogPartition, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("alterPartition is not implemented."); + } + + @Override + public List listFunctions(String databaseName) throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException { + return null; + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction(ObjectPath functionPath, CatalogFunction catalogFunction, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException("createFunction is not implemented."); + } + + @Override + public void alterFunction(ObjectPath functionPath, CatalogFunction catalogFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new UnsupportedOperationException("alterFunction is not implemented."); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new UnsupportedOperationException("dropFunction is not implemented."); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics catalogTableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException("alterTableStatistics is not implemented."); + } + + @Override + public void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatistics catalogColumnStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + throw new UnsupportedOperationException("alterTableColumnStatistics is not implemented."); + } + + @Override + public void alterPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogTableStatistics catalogTableStatistics, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("alterPartitionStatistics is not implemented."); + } + + @Override + public void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec catalogPartitionSpec, CatalogColumnStatistics catalogColumnStatistics, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("alterPartitionColumnStatistics is not implemented."); + } + + private @Nullable Schema getLatestTableSchema(String path) { + if (path != null && StreamerUtil.tableExists(path, hadoopConf)) { + try { + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(path, hadoopConf); + return new TableSchemaResolver(metaClient).getTableAvroSchema(false); // change log mode is not supported now + } catch (Throwable throwable) { + LOG.warn("Error while resolving the latest table schema.", throwable); + // ignored + } + } + return null; + } + + private Map applyOptionsHook(String tablePath, Map options) { + Map newOptions = new HashMap<>(options); + newOptions.put("connector", "hudi"); + newOptions.computeIfAbsent(FlinkOptions.PATH.key(), k -> tablePath); + tableCommonOptions.forEach(newOptions::putIfAbsent); + return newOptions; + } + + private String inferTablePath(String catalogPath, ObjectPath tablePath) { + return String.format("%s/%s/%s", catalogPath, tablePath.getDatabaseName(), tablePath.getObjectName()); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java new file mode 100644 index 000000000000..8ab632ba5a5a --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java @@ -0,0 +1,72 @@ +/* + * 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 org.apache.hudi.table.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; +import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; + +/** + * A catalog factory impl that creates {@link HoodieCatalog}. + */ +public class HoodieCatalogFactory implements CatalogFactory { + private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalogFactory.class); + + public static final String IDENTIFIER = "hudi"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = + FactoryUtil.createCatalogFactoryHelper(this, context); + helper.validate(); + + return new HoodieCatalog( + context.getName(), + (Configuration) helper.getOptions()); + } + + @Override + public Set> requiredOptions() { + Set> options = new HashSet<>(); + options.add(CATALOG_PATH); + options.add(DEFAULT_DATABASE); + return options; + } + + @Override + public Set> optionalOptions() { + return Collections.emptySet(); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java new file mode 100644 index 000000000000..ba6ca4efd726 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java @@ -0,0 +1,131 @@ +/* + * 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 org.apache.hudi.table.catalog; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME; + +/** + * Helper class to read/write flink table options as a map. + */ +public class TableOptionProperties { + private static final Logger LOG = LoggerFactory.getLogger(TableOptionProperties.class); + + private static final String FILE_NAME = "table_option.properties"; + + public static final String PK_CONSTRAINT_NAME = "pk.constraint.name"; + public static final String PK_COLUMNS = "pk.columns"; + public static final String COMMENT = "comment"; + public static final String PARTITION_COLUMNS = "partition.columns"; + + public static final List NON_OPTION_KEYS = Arrays.asList(PK_CONSTRAINT_NAME, PK_COLUMNS, COMMENT, PARTITION_COLUMNS); + + /** + * Initialize the {@link #FILE_NAME} meta file. + */ + public static void createProperties(String basePath, + Configuration hadoopConf, + Map options) throws IOException { + Path propertiesFilePath = getPropertiesFilePath(basePath); + FileSystem fs = FSUtils.getFs(basePath, hadoopConf); + try (FSDataOutputStream outputStream = fs.create(propertiesFilePath)) { + Properties properties = new Properties(); + properties.putAll(options); + properties.store(outputStream, + "Table option properties saved on " + new Date(System.currentTimeMillis())); + } + LOG.info(String.format("Create file %s success.", propertiesFilePath)); + } + + /** + * Read table options map from the given table base path. + */ + public static Map loadFromProperties(String basePath, Configuration hadoopConf) { + Path propertiesFilePath = getPropertiesFilePath(basePath); + Map options = new HashMap<>(); + Properties props = new Properties(); + + FileSystem fs = FSUtils.getFs(basePath, hadoopConf); + try (FSDataInputStream inputStream = fs.open(propertiesFilePath)) { + props.load(inputStream); + for (final String name : props.stringPropertyNames()) { + options.put(name, props.getProperty(name)); + } + } catch (IOException e) { + throw new HoodieIOException(String.format("Could not load table option properties from %s", propertiesFilePath), e); + } + LOG.info(String.format("Loading table option properties from %s success.", propertiesFilePath)); + return options; + } + + private static Path getPropertiesFilePath(String basePath) { + String auxPath = basePath + Path.SEPARATOR + AUXILIARYFOLDER_NAME; + return new Path(auxPath, FILE_NAME); + } + + public static String getPkConstraintName(Map options) { + return options.get(PK_CONSTRAINT_NAME); + } + + public static List getPkColumns(Map options) { + if (options.containsKey(PK_COLUMNS)) { + return Arrays.stream(options.get(PK_COLUMNS).split(",")).collect(Collectors.toList()); + } else { + return Collections.emptyList(); + } + } + + public static List getPartitionColumns(Map options) { + if (options.containsKey(PARTITION_COLUMNS)) { + return Arrays.stream(options.get(PARTITION_COLUMNS).split(",")).collect(Collectors.toList()); + } else { + return Collections.emptyList(); + } + } + + public static String getComment(Map options) { + return options.get(COMMENT); + } + + public static Map getTableOptions(Map options) { + Map copied = new HashMap<>(options); + NON_OPTION_KEYS.forEach(copied::remove); + return copied; + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java new file mode 100644 index 000000000000..07a93e19c25c --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java @@ -0,0 +1,85 @@ +/* + * 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 org.apache.hudi.table.format.cow; + +import org.apache.flink.table.data.vector.writable.WritableBytesVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; +import org.apache.flink.table.data.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Fixed length bytes {@code ColumnReader}, just for decimal. + * + *

Note: Reference Flink release 1.13.2 + * {@code org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader} + * to always write as legacy decimal format. + */ +public class FixedLenBytesColumnReader + extends AbstractColumnReader { + + public FixedLenBytesColumnReader( + ColumnDescriptor descriptor, PageReader pageReader, int precision) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + } + + @Override + protected void readBatch(int rowId, int num, V column) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + byte[] bytes = readDataBinary(bytesLen).getBytes(); + bytesVector.appendBytes(rowId + i, bytes, 0, bytes.length); + } else { + bytesVector.setNullAt(rowId + i); + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, int num, V column, WritableIntVector dictionaryIds) { + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!bytesVector.isNullAt(i)) { + byte[] v = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes(); + bytesVector.appendBytes(i, v, 0, v.length); + } + } + } + + private Binary readDataBinary(int len) { + ByteBuffer buffer = readDataBuffer(len); + if (buffer.hasArray()) { + return Binary.fromConstantByteArray( + buffer.array(), buffer.arrayOffset() + buffer.position(), len); + } else { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return Binary.fromConstantByteArray(bytes); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java index 64eb1f4853d7..c615283c7c5a 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java @@ -18,10 +18,11 @@ package org.apache.hudi.table.format.cow; +import org.apache.hudi.table.format.cow.data.ColumnarRowData; +import org.apache.hudi.table.format.cow.vector.VectorizedColumnBatch; + import org.apache.flink.formats.parquet.vector.reader.ColumnReader; -import org.apache.flink.table.data.ColumnarRowData; import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.VectorizedColumnBatch; import org.apache.flink.table.data.vector.writable.WritableColumnVector; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -208,11 +209,14 @@ private static Type[] clipParquetSchema( private WritableColumnVector[] createWritableVectors() { WritableColumnVector[] columns = new WritableColumnVector[requestedTypes.length]; + List types = requestedSchema.getFields(); + List descriptors = requestedSchema.getColumns(); for (int i = 0; i < requestedTypes.length; i++) { columns[i] = createWritableColumnVector( batchSize, requestedTypes[i], - requestedSchema.getColumns().get(i).getPrimitiveType()); + types.get(i), + descriptors); } return columns; } @@ -236,11 +240,6 @@ private void checkSchema() throws IOException, UnsupportedOperationException { * Check that the requested schema is supported. */ for (int i = 0; i < requestedSchema.getFieldCount(); ++i) { - Type t = requestedSchema.getFields().get(i); - if (!t.isPrimitive() || t.isRepetition(Type.Repetition.REPEATED)) { - throw new UnsupportedOperationException("Complex types not supported."); - } - String[] colPath = requestedSchema.getPaths().get(i); if (fileSchema.containsPath(colPath)) { ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); @@ -322,14 +321,16 @@ private void readNextRowGroup() throws IOException { throw new IOException("expecting more rows but reached last block. Read " + rowsReturned + " out of " + totalRowCount); } + List types = requestedSchema.getFields(); List columns = requestedSchema.getColumns(); - columnReaders = new ColumnReader[columns.size()]; - for (int i = 0; i < columns.size(); ++i) { + columnReaders = new ColumnReader[types.size()]; + for (int i = 0; i < types.size(); ++i) { columnReaders[i] = createColumnReader( utcTimestamp, requestedTypes[i], - columns.get(i), - pages.getPageReader(columns.get(i))); + types.get(i), + columns, + pages); } totalCountLoadedSoFar += pages.getRowCount(); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java index 2749f02f36d3..4705b2f633ff 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java @@ -32,9 +32,9 @@ */ public class ParquetDecimalVector implements DecimalColumnVector { - private final ColumnVector vector; + public final ColumnVector vector; - ParquetDecimalVector(ColumnVector vector) { + public ParquetDecimalVector(ColumnVector vector) { this.vector = vector; } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index 6bb514b429b3..10a2dcd5a1c8 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -18,13 +18,21 @@ package org.apache.hudi.table.format.cow; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; +import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; +import org.apache.hudi.table.format.cow.vector.VectorizedColumnBatch; +import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.MapColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.RowColumnReader; + import org.apache.flink.core.fs.Path; import org.apache.flink.formats.parquet.vector.reader.BooleanColumnReader; import org.apache.flink.formats.parquet.vector.reader.ByteColumnReader; import org.apache.flink.formats.parquet.vector.reader.BytesColumnReader; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; import org.apache.flink.formats.parquet.vector.reader.DoubleColumnReader; -import org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader; import org.apache.flink.formats.parquet.vector.reader.FloatColumnReader; import org.apache.flink.formats.parquet.vector.reader.IntColumnReader; import org.apache.flink.formats.parquet.vector.reader.LongColumnReader; @@ -33,7 +41,6 @@ import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.data.vector.ColumnVector; -import org.apache.flink.table.data.vector.VectorizedColumnBatch; import org.apache.flink.table.data.vector.heap.HeapBooleanVector; import org.apache.flink.table.data.vector.heap.HeapByteVector; import org.apache.flink.table.data.vector.heap.HeapBytesVector; @@ -45,16 +52,24 @@ import org.apache.flink.table.data.vector.heap.HeapTimestampVector; import org.apache.flink.table.data.vector.writable.WritableColumnVector; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.ParquetRuntimeException; import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.InvalidSchemaException; import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import java.io.IOException; import java.math.BigDecimal; @@ -62,6 +77,7 @@ import java.sql.Date; import java.time.LocalDate; import java.time.LocalDateTime; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -253,11 +269,40 @@ private static ColumnVector createVectorFromConstant( } } + private static List filterDescriptors(int depth, Type type, List columns) throws ParquetRuntimeException { + List filtered = new ArrayList<>(); + for (ColumnDescriptor descriptor : columns) { + if (depth >= descriptor.getPath().length) { + throw new InvalidSchemaException("Expect depth " + depth + " for schema: " + descriptor); + } + if (type.getName().equals(descriptor.getPath()[depth])) { + filtered.add(descriptor); + } + } + ValidationUtils.checkState(filtered.size() > 0, "Corrupted Parquet schema"); + return filtered; + } + public static ColumnReader createColumnReader( boolean utcTimestamp, LogicalType fieldType, - ColumnDescriptor descriptor, - PageReader pageReader) throws IOException { + Type physicalType, + List descriptors, + PageReadStore pages) throws IOException { + return createColumnReader(utcTimestamp, fieldType, physicalType, descriptors, + pages, 0); + } + + private static ColumnReader createColumnReader( + boolean utcTimestamp, + LogicalType fieldType, + Type physicalType, + List columns, + PageReadStore pages, + int depth) throws IOException { + List descriptors = filterDescriptors(depth, physicalType, columns); + ColumnDescriptor descriptor = descriptors.get(0); + PageReader pageReader = pages.getPageReader(descriptor); switch (fieldType.getTypeRoot()) { case BOOLEAN: return new BooleanColumnReader(descriptor, pageReader); @@ -304,6 +349,45 @@ public static ColumnReader createColumnReader( default: throw new AssertionError(); } + case ARRAY: + return new ArrayColumnReader( + descriptor, + pageReader, + utcTimestamp, + descriptor.getPrimitiveType(), + fieldType); + case MAP: + MapType mapType = (MapType) fieldType; + ArrayColumnReader keyReader = + new ArrayColumnReader( + descriptor, + pageReader, + utcTimestamp, + descriptor.getPrimitiveType(), + new ArrayType(mapType.getKeyType())); + ArrayColumnReader valueReader = + new ArrayColumnReader( + descriptors.get(1), + pages.getPageReader(descriptors.get(1)), + utcTimestamp, + descriptors.get(1).getPrimitiveType(), + new ArrayType(mapType.getValueType())); + return new MapColumnReader(keyReader, valueReader, fieldType); + case ROW: + RowType rowType = (RowType) fieldType; + GroupType groupType = physicalType.asGroupType(); + List fieldReaders = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + fieldReaders.add( + createColumnReader( + utcTimestamp, + rowType.getTypeAt(i), + groupType.getType(i), + descriptors, + pages, + depth + 1)); + } + return new RowColumnReader(fieldReaders); default: throw new UnsupportedOperationException(fieldType + " is not supported now."); } @@ -312,7 +396,19 @@ public static ColumnReader createColumnReader( public static WritableColumnVector createWritableColumnVector( int batchSize, LogicalType fieldType, - PrimitiveType primitiveType) { + Type physicalType, + List descriptors) { + return createWritableColumnVector(batchSize, fieldType, physicalType, descriptors, 0); + } + + private static WritableColumnVector createWritableColumnVector( + int batchSize, + LogicalType fieldType, + Type physicalType, + List columns, + int depth) { + List descriptors = filterDescriptors(depth, physicalType, columns); + PrimitiveType primitiveType = descriptors.get(0).getPrimitiveType(); PrimitiveType.PrimitiveTypeName typeName = primitiveType.getPrimitiveTypeName(); switch (fieldType.getTypeRoot()) { case BOOLEAN: @@ -366,10 +462,55 @@ public static WritableColumnVector createWritableColumnVector( "TIME_MICROS original type is not "); return new HeapTimestampVector(batchSize); case DECIMAL: - checkArgument(typeName == PrimitiveType.PrimitiveTypeName.BINARY + checkArgument( + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY + || typeName == PrimitiveType.PrimitiveTypeName.BINARY) && primitiveType.getOriginalType() == OriginalType.DECIMAL, "Unexpected type: %s", typeName); return new HeapBytesVector(batchSize); + case ARRAY: + ArrayType arrayType = (ArrayType) fieldType; + return new HeapArrayVector( + batchSize, + createWritableColumnVector( + batchSize, + arrayType.getElementType(), + physicalType, + descriptors, + depth)); + case MAP: + MapType mapType = (MapType) fieldType; + GroupType repeatedType = physicalType.asGroupType().getType(0).asGroupType(); + // the map column has three level paths. + return new HeapMapColumnVector( + batchSize, + createWritableColumnVector( + batchSize, + mapType.getKeyType(), + repeatedType.getType(0), + descriptors, + depth + 2), + createWritableColumnVector( + batchSize, + mapType.getValueType(), + repeatedType.getType(1), + descriptors, + depth + 2)); + case ROW: + RowType rowType = (RowType) fieldType; + GroupType groupType = physicalType.asGroupType(); + WritableColumnVector[] columnVectors = + new WritableColumnVector[rowType.getFieldCount()]; + for (int i = 0; i < columnVectors.length; i++) { + columnVectors[i] = + createWritableColumnVector( + batchSize, + rowType.getTypeAt(i), + groupType.getType(i), + descriptors, + depth + 1); + } + return new HeapRowColumnVector(batchSize, columnVectors); default: throw new UnsupportedOperationException(fieldType + " is not supported now."); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarArrayData.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarArrayData.java new file mode 100644 index 000000000000..a16a4dd8d014 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarArrayData.java @@ -0,0 +1,272 @@ +/* + * 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 org.apache.hudi.table.format.cow.data; + +import org.apache.hudi.table.format.cow.vector.MapColumnVector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.binary.TypedSetters; +import org.apache.flink.table.data.vector.ArrayColumnVector; +import org.apache.flink.table.data.vector.BooleanColumnVector; +import org.apache.flink.table.data.vector.ByteColumnVector; +import org.apache.flink.table.data.vector.BytesColumnVector; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.DecimalColumnVector; +import org.apache.flink.table.data.vector.DoubleColumnVector; +import org.apache.flink.table.data.vector.FloatColumnVector; +import org.apache.flink.table.data.vector.IntColumnVector; +import org.apache.flink.table.data.vector.LongColumnVector; +import org.apache.flink.table.data.vector.RowColumnVector; +import org.apache.flink.table.data.vector.ShortColumnVector; +import org.apache.flink.table.data.vector.TimestampColumnVector; + +import java.util.Arrays; + +/** + * Columnar array to support access to vector column data. + * + *

References {@code org.apache.flink.table.data.ColumnarArrayData} to include FLINK-15390. + */ +public final class ColumnarArrayData implements ArrayData, TypedSetters { + + private final ColumnVector data; + private final int offset; + private final int numElements; + + public ColumnarArrayData(ColumnVector data, int offset, int numElements) { + this.data = data; + this.offset = offset; + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int pos) { + return data.isNullAt(offset + pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean getBoolean(int pos) { + return ((BooleanColumnVector) data).getBoolean(offset + pos); + } + + @Override + public byte getByte(int pos) { + return ((ByteColumnVector) data).getByte(offset + pos); + } + + @Override + public short getShort(int pos) { + return ((ShortColumnVector) data).getShort(offset + pos); + } + + @Override + public int getInt(int pos) { + return ((IntColumnVector) data).getInt(offset + pos); + } + + @Override + public long getLong(int pos) { + return ((LongColumnVector) data).getLong(offset + pos); + } + + @Override + public float getFloat(int pos) { + return ((FloatColumnVector) data).getFloat(offset + pos); + } + + @Override + public double getDouble(int pos) { + return ((DoubleColumnVector) data).getDouble(offset + pos); + } + + @Override + public StringData getString(int pos) { + BytesColumnVector.Bytes byteArray = getByteArray(pos); + return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return ((DecimalColumnVector) data).getDecimal(offset + pos, precision, scale); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return ((TimestampColumnVector) data).getTimestamp(offset + pos, precision); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("RawValueData is not supported."); + } + + @Override + public byte[] getBinary(int pos) { + BytesColumnVector.Bytes byteArray = getByteArray(pos); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + return Arrays.copyOfRange(byteArray.data, byteArray.offset, byteArray.len); + } + } + + @Override + public ArrayData getArray(int pos) { + return ((ArrayColumnVector) data).getArray(offset + pos); + } + + @Override + public MapData getMap(int pos) { + return ((MapColumnVector) data).getMap(offset + pos); + } + + @Override + public RowData getRow(int pos, int numFields) { + return ((RowColumnVector) data).getRow(offset + pos); + } + + @Override + public void setBoolean(int pos, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int pos, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int pos, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int pos, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int pos, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int pos, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int pos, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int pos, DecimalData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int pos, TimestampData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] res = new boolean[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getBoolean(i); + } + return res; + } + + @Override + public byte[] toByteArray() { + byte[] res = new byte[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getByte(i); + } + return res; + } + + @Override + public short[] toShortArray() { + short[] res = new short[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getShort(i); + } + return res; + } + + @Override + public int[] toIntArray() { + int[] res = new int[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getInt(i); + } + return res; + } + + @Override + public long[] toLongArray() { + long[] res = new long[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getLong(i); + } + return res; + } + + @Override + public float[] toFloatArray() { + float[] res = new float[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getFloat(i); + } + return res; + } + + @Override + public double[] toDoubleArray() { + double[] res = new double[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getDouble(i); + } + return res; + } + + private BytesColumnVector.Bytes getByteArray(int pos) { + return ((BytesColumnVector) data).getBytes(offset + pos); + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarMapData.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarMapData.java new file mode 100644 index 000000000000..9792e87ec936 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarMapData.java @@ -0,0 +1,75 @@ +/* + * 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 org.apache.hudi.table.format.cow.data; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.ColumnarArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.vector.ColumnVector; + +/** + * Columnar map to support access to vector column data. + * + *

Referenced from flink 1.14.0 {@code org.apache.flink.table.data.ColumnarMapData}. + */ +public final class ColumnarMapData implements MapData { + + private final ColumnVector keyColumnVector; + private final ColumnVector valueColumnVector; + private final int offset; + private final int size; + + public ColumnarMapData( + ColumnVector keyColumnVector, + ColumnVector valueColumnVector, + int offset, + int size) { + this.keyColumnVector = keyColumnVector; + this.valueColumnVector = valueColumnVector; + this.offset = offset; + this.size = size; + } + + @Override + public int size() { + return size; + } + + @Override + public ArrayData keyArray() { + return new ColumnarArrayData(keyColumnVector, offset, size); + } + + @Override + public ArrayData valueArray() { + return new ColumnarArrayData(valueColumnVector, offset, size); + } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException( + "ColumnarMapData do not support equals, please compare fields one by one!"); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException( + "ColumnarMapData do not support hashCode, please hash fields one by one!"); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarRowData.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarRowData.java new file mode 100644 index 000000000000..ebb4ca26fa87 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/data/ColumnarRowData.java @@ -0,0 +1,232 @@ +/* + * 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 org.apache.hudi.table.format.cow.data; + +import org.apache.hudi.table.format.cow.vector.VectorizedColumnBatch; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.binary.TypedSetters; +import org.apache.flink.table.data.vector.BytesColumnVector; +import org.apache.flink.types.RowKind; + +/** + * Columnar row to support access to vector column data. + * It is a row view in {@link VectorizedColumnBatch}. + * + *

References {@code org.apache.flink.table.data.ColumnarRowData} to include FLINK-15390. + */ +public final class ColumnarRowData implements RowData, TypedSetters { + + private RowKind rowKind = RowKind.INSERT; + private VectorizedColumnBatch vectorizedColumnBatch; + private int rowId; + + public ColumnarRowData() { + } + + public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch) { + this(vectorizedColumnBatch, 0); + } + + public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch, int rowId) { + this.vectorizedColumnBatch = vectorizedColumnBatch; + this.rowId = rowId; + } + + public void setVectorizedColumnBatch(VectorizedColumnBatch vectorizedColumnBatch) { + this.vectorizedColumnBatch = vectorizedColumnBatch; + this.rowId = 0; + } + + public void setRowId(int rowId) { + this.rowId = rowId; + } + + @Override + public RowKind getRowKind() { + return rowKind; + } + + @Override + public void setRowKind(RowKind kind) { + this.rowKind = kind; + } + + @Override + public int getArity() { + return vectorizedColumnBatch.getArity(); + } + + @Override + public boolean isNullAt(int pos) { + return vectorizedColumnBatch.isNullAt(rowId, pos); + } + + @Override + public boolean getBoolean(int pos) { + return vectorizedColumnBatch.getBoolean(rowId, pos); + } + + @Override + public byte getByte(int pos) { + return vectorizedColumnBatch.getByte(rowId, pos); + } + + @Override + public short getShort(int pos) { + return vectorizedColumnBatch.getShort(rowId, pos); + } + + @Override + public int getInt(int pos) { + return vectorizedColumnBatch.getInt(rowId, pos); + } + + @Override + public long getLong(int pos) { + return vectorizedColumnBatch.getLong(rowId, pos); + } + + @Override + public float getFloat(int pos) { + return vectorizedColumnBatch.getFloat(rowId, pos); + } + + @Override + public double getDouble(int pos) { + return vectorizedColumnBatch.getDouble(rowId, pos); + } + + @Override + public StringData getString(int pos) { + BytesColumnVector.Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); + return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return vectorizedColumnBatch.getDecimal(rowId, pos, precision, scale); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return vectorizedColumnBatch.getTimestamp(rowId, pos, precision); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("RawValueData is not supported."); + } + + @Override + public byte[] getBinary(int pos) { + BytesColumnVector.Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + byte[] ret = new byte[byteArray.len]; + System.arraycopy(byteArray.data, byteArray.offset, ret, 0, byteArray.len); + return ret; + } + } + + @Override + public RowData getRow(int pos, int numFields) { + return vectorizedColumnBatch.getRow(rowId, pos); + } + + @Override + public ArrayData getArray(int pos) { + return vectorizedColumnBatch.getArray(rowId, pos); + } + + @Override + public MapData getMap(int pos) { + return vectorizedColumnBatch.getMap(rowId, pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setBoolean(int pos, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int pos, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int pos, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int pos, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int pos, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int pos, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int pos, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int pos, DecimalData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int pos, TimestampData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException( + "ColumnarRowData do not support equals, please compare fields one by one!"); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException( + "ColumnarRowData do not support hashCode, please hash fields one by one!"); + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java new file mode 100644 index 000000000000..f4c15b6a9b36 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java @@ -0,0 +1,71 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.hudi.table.format.cow.data.ColumnarArrayData; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.vector.ArrayColumnVector; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap array column vector. + */ +public class HeapArrayVector extends AbstractHeapVector + implements WritableColumnVector, ArrayColumnVector { + + public long[] offsets; + public long[] lengths; + public ColumnVector child; + private int size; + + public HeapArrayVector(int len) { + super(len); + offsets = new long[len]; + lengths = new long[len]; + } + + public HeapArrayVector(int len, ColumnVector vector) { + super(len); + offsets = new long[len]; + lengths = new long[len]; + this.child = vector; + } + + public int getSize() { + return size; + } + + public void setSize(int size) { + this.size = size; + } + + public int getLen() { + return this.isNull.length; + } + + @Override + public ArrayData getArray(int i) { + long offset = offsets[i]; + long length = lengths[i]; + return new ColumnarArrayData(child, (int) offset, (int) length); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java new file mode 100644 index 000000000000..f05a2e73431d --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java @@ -0,0 +1,79 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.hudi.table.format.cow.data.ColumnarMapData; + +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap map column vector. + */ +public class HeapMapColumnVector extends AbstractHeapVector + implements WritableColumnVector, MapColumnVector { + + private long[] offsets; + private long[] lengths; + private int size; + private ColumnVector keys; + private ColumnVector values; + + public HeapMapColumnVector(int len, ColumnVector keys, ColumnVector values) { + super(len); + size = 0; + offsets = new long[len]; + lengths = new long[len]; + this.keys = keys; + this.values = values; + } + + public void setOffsets(long[] offsets) { + this.offsets = offsets; + } + + public void setLengths(long[] lengths) { + this.lengths = lengths; + } + + public void setKeys(ColumnVector keys) { + this.keys = keys; + } + + public void setValues(ColumnVector values) { + this.values = values; + } + + public int getSize() { + return size; + } + + public void setSize(int size) { + this.size = size; + } + + @Override + public MapData getMap(int i) { + long offset = offsets[i]; + long length = lengths[i]; + return new ColumnarMapData(keys, values, (int) offset, (int) length); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java new file mode 100644 index 000000000000..ad05a612c7bd --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java @@ -0,0 +1,45 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.hudi.table.format.cow.data.ColumnarRowData; + +import org.apache.flink.table.data.vector.heap.AbstractHeapVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; + +/** + * This class represents a nullable heap row column vector. + */ +public class HeapRowColumnVector extends AbstractHeapVector + implements WritableColumnVector, RowColumnVector { + + public WritableColumnVector[] vectors; + + public HeapRowColumnVector(int len, WritableColumnVector... vectors) { + super(len); + this.vectors = vectors; + } + + @Override + public ColumnarRowData getRow(int i) { + ColumnarRowData columnarRowData = new ColumnarRowData(new VectorizedColumnBatch(vectors)); + columnarRowData.setRowId(i); + return columnarRowData; + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/MapColumnVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/MapColumnVector.java new file mode 100644 index 000000000000..38424dad7d3a --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/MapColumnVector.java @@ -0,0 +1,29 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.vector.ColumnVector; + +/** + * Map column vector. + */ +public interface MapColumnVector extends ColumnVector { + MapData getMap(int i); +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/RowColumnVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/RowColumnVector.java new file mode 100644 index 000000000000..293af7b9cf2e --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/RowColumnVector.java @@ -0,0 +1,30 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.hudi.table.format.cow.data.ColumnarRowData; + +import org.apache.flink.table.data.vector.ColumnVector; + +/** + * Row column vector. + */ +public interface RowColumnVector extends ColumnVector { + ColumnarRowData getRow(int i); +} \ No newline at end of file diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/VectorizedColumnBatch.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/VectorizedColumnBatch.java new file mode 100644 index 000000000000..9eee55d1eeae --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/VectorizedColumnBatch.java @@ -0,0 +1,148 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.vector.ArrayColumnVector; +import org.apache.flink.table.data.vector.BooleanColumnVector; +import org.apache.flink.table.data.vector.ByteColumnVector; +import org.apache.flink.table.data.vector.BytesColumnVector; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.DecimalColumnVector; +import org.apache.flink.table.data.vector.DoubleColumnVector; +import org.apache.flink.table.data.vector.FloatColumnVector; +import org.apache.flink.table.data.vector.IntColumnVector; +import org.apache.flink.table.data.vector.LongColumnVector; +import org.apache.flink.table.data.vector.ShortColumnVector; +import org.apache.flink.table.data.vector.TimestampColumnVector; + +import java.io.Serializable; +import java.nio.charset.StandardCharsets; + +/** + * A VectorizedColumnBatch is a set of rows, organized with each column as a vector. It is the unit + * of query execution, organized to minimize the cost per row. + * + *

{@code VectorizedColumnBatch}s are influenced by Apache Hive VectorizedRowBatch. + * + *

References {@code org.apache.flink.table.data.vector.VectorizedColumnBatch} to include FLINK-15390. + */ +public class VectorizedColumnBatch implements Serializable { + private static final long serialVersionUID = 1L; + + /** + * This number is carefully chosen to minimize overhead and typically allows one + * VectorizedColumnBatch to fit in cache. + */ + public static final int DEFAULT_SIZE = 2048; + + private int numRows; + public final ColumnVector[] columns; + + public VectorizedColumnBatch(ColumnVector[] vectors) { + this.columns = vectors; + } + + public void setNumRows(int numRows) { + this.numRows = numRows; + } + + public int getNumRows() { + return numRows; + } + + public int getArity() { + return columns.length; + } + + public boolean isNullAt(int rowId, int colId) { + return columns[colId].isNullAt(rowId); + } + + public boolean getBoolean(int rowId, int colId) { + return ((BooleanColumnVector) columns[colId]).getBoolean(rowId); + } + + public byte getByte(int rowId, int colId) { + return ((ByteColumnVector) columns[colId]).getByte(rowId); + } + + public short getShort(int rowId, int colId) { + return ((ShortColumnVector) columns[colId]).getShort(rowId); + } + + public int getInt(int rowId, int colId) { + return ((IntColumnVector) columns[colId]).getInt(rowId); + } + + public long getLong(int rowId, int colId) { + return ((LongColumnVector) columns[colId]).getLong(rowId); + } + + public float getFloat(int rowId, int colId) { + return ((FloatColumnVector) columns[colId]).getFloat(rowId); + } + + public double getDouble(int rowId, int colId) { + return ((DoubleColumnVector) columns[colId]).getDouble(rowId); + } + + public BytesColumnVector.Bytes getByteArray(int rowId, int colId) { + return ((BytesColumnVector) columns[colId]).getBytes(rowId); + } + + private byte[] getBytes(int rowId, int colId) { + BytesColumnVector.Bytes byteArray = getByteArray(rowId, colId); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + return byteArray.getBytes(); + } + } + + public String getString(int rowId, int colId) { + BytesColumnVector.Bytes byteArray = getByteArray(rowId, colId); + return new String(byteArray.data, byteArray.offset, byteArray.len, StandardCharsets.UTF_8); + } + + public DecimalData getDecimal(int rowId, int colId, int precision, int scale) { + return ((DecimalColumnVector) (columns[colId])).getDecimal(rowId, precision, scale); + } + + public TimestampData getTimestamp(int rowId, int colId, int precision) { + return ((TimestampColumnVector) (columns[colId])).getTimestamp(rowId, precision); + } + + public ArrayData getArray(int rowId, int colId) { + return ((ArrayColumnVector) columns[colId]).getArray(rowId); + } + + public RowData getRow(int rowId, int colId) { + return ((RowColumnVector) columns[colId]).getRow(rowId); + } + + public MapData getMap(int rowId, int colId) { + return ((MapColumnVector) columns[colId]).getMap(rowId); + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java new file mode 100644 index 000000000000..256d4c1bbcec --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java @@ -0,0 +1,473 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.ParquetDecimalVector; +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.vector.VectorizedColumnBatch; +import org.apache.flink.table.data.vector.heap.HeapBooleanVector; +import org.apache.flink.table.data.vector.heap.HeapByteVector; +import org.apache.flink.table.data.vector.heap.HeapBytesVector; +import org.apache.flink.table.data.vector.heap.HeapDoubleVector; +import org.apache.flink.table.data.vector.heap.HeapFloatVector; +import org.apache.flink.table.data.vector.heap.HeapIntVector; +import org.apache.flink.table.data.vector.heap.HeapLongVector; +import org.apache.flink.table.data.vector.heap.HeapShortVector; +import org.apache.flink.table.data.vector.heap.HeapTimestampVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Array {@link ColumnReader}. + */ +public class ArrayColumnReader extends BaseVectorizedColumnReader { + + // The value read in last time + private Object lastValue; + + // flag to indicate if there is no data in parquet data page + private boolean eof = false; + + // flag to indicate if it's the first time to read parquet data page with this instance + boolean isFirstRow = true; + + public ArrayColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader, + boolean isUtcTimestamp, + Type type, + LogicalType logicalType) + throws IOException { + super(descriptor, pageReader, isUtcTimestamp, type, logicalType); + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapArrayVector lcv = (HeapArrayVector) vector; + // before readBatch, initial the size of offsets & lengths as the default value, + // the actual size will be assigned in setChildrenInfo() after reading complete. + lcv.offsets = new long[VectorizedColumnBatch.DEFAULT_SIZE]; + lcv.lengths = new long[VectorizedColumnBatch.DEFAULT_SIZE]; + // Because the length of ListColumnVector.child can't be known now, + // the valueList will save all data for ListColumnVector temporary. + List valueList = new ArrayList<>(); + + LogicalType category = ((ArrayType) logicalType).getElementType(); + + // read the first row in parquet data page, this will be only happened once for this + // instance + if (isFirstRow) { + if (!fetchNextValue(category)) { + return; + } + isFirstRow = false; + } + + int index = collectDataFromParquetPage(readNumber, lcv, valueList, category); + + // Convert valueList to array for the ListColumnVector.child + fillColumnVector(category, lcv, valueList, index); + } + + /** + * Reads a single value from parquet page, puts it into lastValue. Returns a boolean indicating + * if there is more values to read (true). + * + * @param category + * @return boolean + * @throws IOException + */ + private boolean fetchNextValue(LogicalType category) throws IOException { + int left = readPageIfNeed(); + if (left > 0) { + // get the values of repetition and definitionLevel + readRepetitionAndDefinitionLevels(); + // read the data if it isn't null + if (definitionLevel == maxDefLevel) { + if (isCurrentPageDictionaryEncoded) { + lastValue = dataColumn.readValueDictionaryId(); + } else { + lastValue = readPrimitiveTypedRow(category); + } + } else { + lastValue = null; + } + return true; + } else { + eof = true; + return false; + } + } + + private int readPageIfNeed() throws IOException { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + // no data left in current page, load data from new page + readPage(); + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + return leftInPage; + } + + // Need to be in consistent with that VectorizedPrimitiveColumnReader#readBatchHelper + // TODO Reduce the duplicated code + private Object readPrimitiveTypedRow(LogicalType category) { + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return dataColumn.readString(); + case BOOLEAN: + return dataColumn.readBoolean(); + case TIME_WITHOUT_TIME_ZONE: + case DATE: + case INTEGER: + return dataColumn.readInteger(); + case TINYINT: + return dataColumn.readTinyInt(); + case SMALLINT: + return dataColumn.readSmallInt(); + case BIGINT: + return dataColumn.readLong(); + case FLOAT: + return dataColumn.readFloat(); + case DOUBLE: + return dataColumn.readDouble(); + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return dataColumn.readInteger(); + case INT64: + return dataColumn.readLong(); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return dataColumn.readString(); + default: + throw new AssertionError(); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return dataColumn.readTimestamp(); + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } + + private Object dictionaryDecodeValue(LogicalType category, Integer dictionaryValue) { + if (dictionaryValue == null) { + return null; + } + + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return dictionary.readString(dictionaryValue); + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTEGER: + return dictionary.readInteger(dictionaryValue); + case BOOLEAN: + return dictionary.readBoolean(dictionaryValue) ? 1 : 0; + case DOUBLE: + return dictionary.readDouble(dictionaryValue); + case FLOAT: + return dictionary.readFloat(dictionaryValue); + case TINYINT: + return dictionary.readTinyInt(dictionaryValue); + case SMALLINT: + return dictionary.readSmallInt(dictionaryValue); + case BIGINT: + return dictionary.readLong(dictionaryValue); + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return dictionary.readInteger(dictionaryValue); + case INT64: + return dictionary.readLong(dictionaryValue); + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return dictionary.readString(dictionaryValue); + default: + throw new AssertionError(); + } + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return dictionary.readTimestamp(dictionaryValue); + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } + + /** + * Collects data from a parquet page and returns the final row index where it stopped. The + * returned index can be equal to or less than total. + * + * @param total maximum number of rows to collect + * @param lcv column vector to do initial setup in data collection time + * @param valueList collection of values that will be fed into the vector later + * @param category + * @return int + * @throws IOException + */ + private int collectDataFromParquetPage( + int total, HeapArrayVector lcv, List valueList, LogicalType category) + throws IOException { + int index = 0; + /* + * Here is a nested loop for collecting all values from a parquet page. + * A column of array type can be considered as a list of lists, so the two loops are as below: + * 1. The outer loop iterates on rows (index is a row index, so points to a row in the batch), e.g.: + * [0, 2, 3] <- index: 0 + * [NULL, 3, 4] <- index: 1 + * + * 2. The inner loop iterates on values within a row (sets all data from parquet data page + * for an element in ListColumnVector), so fetchNextValue returns values one-by-one: + * 0, 2, 3, NULL, 3, 4 + * + * As described below, the repetition level (repetitionLevel != 0) + * can be used to decide when we'll start to read values for the next list. + */ + while (!eof && index < total) { + // add element to ListColumnVector one by one + lcv.offsets[index] = valueList.size(); + /* + * Let's collect all values for a single list. + * Repetition level = 0 means that a new list started there in the parquet page, + * in that case, let's exit from the loop, and start to collect value for a new list. + */ + do { + /* + * Definition level = 0 when a NULL value was returned instead of a list + * (this is not the same as a NULL value in of a list). + */ + if (definitionLevel == 0) { + lcv.setNullAt(index); + } + valueList.add( + isCurrentPageDictionaryEncoded + ? dictionaryDecodeValue(category, (Integer) lastValue) + : lastValue); + } while (fetchNextValue(category) && (repetitionLevel != 0)); + + lcv.lengths[index] = valueList.size() - lcv.offsets[index]; + index++; + } + return index; + } + + /** + * The lengths & offsets will be initialized as default size (1024), it should be set to the + * actual size according to the element number. + */ + private void setChildrenInfo(HeapArrayVector lcv, int itemNum, int elementNum) { + lcv.setSize(itemNum); + long[] lcvLength = new long[elementNum]; + long[] lcvOffset = new long[elementNum]; + System.arraycopy(lcv.lengths, 0, lcvLength, 0, elementNum); + System.arraycopy(lcv.offsets, 0, lcvOffset, 0, elementNum); + lcv.lengths = lcvLength; + lcv.offsets = lcvOffset; + } + + private void fillColumnVector( + LogicalType category, HeapArrayVector lcv, List valueList, int elementNum) { + int total = valueList.size(); + setChildrenInfo(lcv, total, elementNum); + switch (category.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + lcv.child = new HeapBytesVector(total); + ((HeapBytesVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + byte[] src = ((List) valueList).get(i); + if (src == null) { + ((HeapBytesVector) lcv.child).setNullAt(i); + } else { + ((HeapBytesVector) lcv.child).appendBytes(i, src, 0, src.length); + } + } + break; + case BOOLEAN: + lcv.child = new HeapBooleanVector(total); + ((HeapBooleanVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapBooleanVector) lcv.child).setNullAt(i); + } else { + ((HeapBooleanVector) lcv.child).vector[i] = + ((List) valueList).get(i); + } + } + break; + case TINYINT: + lcv.child = new HeapByteVector(total); + ((HeapByteVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapByteVector) lcv.child).setNullAt(i); + } else { + ((HeapByteVector) lcv.child).vector[i] = + (byte) ((List) valueList).get(i).intValue(); + } + } + break; + case SMALLINT: + lcv.child = new HeapShortVector(total); + ((HeapShortVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapShortVector) lcv.child).setNullAt(i); + } else { + ((HeapShortVector) lcv.child).vector[i] = + (short) ((List) valueList).get(i).intValue(); + } + } + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + lcv.child = new HeapIntVector(total); + ((HeapIntVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapIntVector) lcv.child).setNullAt(i); + } else { + ((HeapIntVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case FLOAT: + lcv.child = new HeapFloatVector(total); + ((HeapFloatVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapFloatVector) lcv.child).setNullAt(i); + } else { + ((HeapFloatVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case BIGINT: + lcv.child = new HeapLongVector(total); + ((HeapLongVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapLongVector) lcv.child).setNullAt(i); + } else { + ((HeapLongVector) lcv.child).vector[i] = ((List) valueList).get(i); + } + } + break; + case DOUBLE: + lcv.child = new HeapDoubleVector(total); + ((HeapDoubleVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapDoubleVector) lcv.child).setNullAt(i); + } else { + ((HeapDoubleVector) lcv.child).vector[i] = + ((List) valueList).get(i); + } + } + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + lcv.child = new HeapTimestampVector(total); + ((HeapTimestampVector) lcv.child).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapTimestampVector) lcv.child).setNullAt(i); + } else { + ((HeapTimestampVector) lcv.child) + .setTimestamp(i, ((List) valueList).get(i)); + } + } + break; + case DECIMAL: + PrimitiveType.PrimitiveTypeName primitiveTypeName = + descriptor.getPrimitiveType().getPrimitiveTypeName(); + switch (primitiveTypeName) { + case INT32: + lcv.child = new ParquetDecimalVector(new HeapIntVector(total)); + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapIntVector) ((ParquetDecimalVector) lcv.child).vector) + .vector[i] = + ((List) valueList).get(i); + } + } + break; + case INT64: + lcv.child = new ParquetDecimalVector(new HeapLongVector(total)); + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + if (valueList.get(i) == null) { + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapLongVector) ((ParquetDecimalVector) lcv.child).vector) + .vector[i] = + ((List) valueList).get(i); + } + } + break; + default: + lcv.child = new ParquetDecimalVector(new HeapBytesVector(total)); + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector).reset(); + for (int i = 0; i < valueList.size(); i++) { + byte[] src = ((List) valueList).get(i); + if (valueList.get(i) == null) { + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector) + .setNullAt(i); + } else { + ((HeapBytesVector) ((ParquetDecimalVector) lcv.child).vector) + .appendBytes(i, src, 0, src.length); + } + } + break; + } + break; + default: + throw new RuntimeException("Unsupported type in the list: " + type); + } + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java new file mode 100644 index 000000000000..073c704c4b24 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java @@ -0,0 +1,313 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + +import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL; +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.parquet.column.ValuesType.VALUES; + +/** + * Abstract {@link ColumnReader}. part of the code is referred from Apache Hive and Apache Parquet. + */ +public abstract class BaseVectorizedColumnReader implements ColumnReader { + + private static final Logger LOG = LoggerFactory.getLogger(BaseVectorizedColumnReader.class); + + protected boolean isUtcTimestamp; + + /** + * Total number of values read. + */ + protected long valuesRead; + + /** + * value that indicates the end of the current page. That is, if valuesRead == + * endOfPageValueCount, we are at the end of the page. + */ + protected long endOfPageValueCount; + + /** + * The dictionary, if this column has dictionary encoding. + */ + protected final ParquetDataColumnReader dictionary; + + /** + * If true, the current page is dictionary encoded. + */ + protected boolean isCurrentPageDictionaryEncoded; + + /** + * Maximum definition level for this column. + */ + protected final int maxDefLevel; + + protected int definitionLevel; + protected int repetitionLevel; + + /** + * Repetition/Definition/Value readers. + */ + protected IntIterator repetitionLevelColumn; + + protected IntIterator definitionLevelColumn; + protected ParquetDataColumnReader dataColumn; + + /** + * Total values in the current page. + */ + protected int pageValueCount; + + protected final PageReader pageReader; + protected final ColumnDescriptor descriptor; + protected final Type type; + protected final LogicalType logicalType; + + public BaseVectorizedColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader, + boolean isUtcTimestamp, + Type parquetType, + LogicalType logicalType) + throws IOException { + this.descriptor = descriptor; + this.type = parquetType; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + this.isUtcTimestamp = isUtcTimestamp; + this.logicalType = logicalType; + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = + ParquetDataColumnReaderFactory.getDataColumnReaderByTypeOnDictionary( + parquetType.asPrimitiveType(), + dictionaryPage + .getEncoding() + .initDictionary(descriptor, dictionaryPage), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + } + + protected void readRepetitionAndDefinitionLevels() { + repetitionLevel = repetitionLevelColumn.nextInt(); + definitionLevel = definitionLevelColumn.nextInt(); + valuesRead++; + } + + protected void readPage() throws IOException { + DataPage page = pageReader.readPage(); + + if (page == null) { + return; + } + + page.accept( + new DataPage.Visitor() { + @Override + public Void visit(DataPageV1 dataPageV1) { + readPageV1(dataPageV1); + return null; + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + readPageV2(dataPageV2); + return null; + } + }); + } + + private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) + throws IOException { + this.pageValueCount = valueCount; + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + this.dataColumn = null; + if (dictionary == null) { + throw new IOException( + "could not read page in col " + + descriptor + + " as the dictionary was missing for encoding " + + dataEncoding); + } + dataColumn = + ParquetDataColumnReaderFactory.getDataColumnReaderByType( + type.asPrimitiveType(), + dataEncoding.getDictionaryBasedValuesReader( + descriptor, VALUES, dictionary.getDictionary()), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = true; + } else { + dataColumn = + ParquetDataColumnReaderFactory.getDataColumnReaderByType( + type.asPrimitiveType(), + dataEncoding.getValuesReader(descriptor, VALUES), + isUtcTimestamp); + this.isCurrentPageDictionaryEncoded = false; + } + + try { + dataColumn.initFromPage(pageValueCount, in); + } catch (IOException e) { + throw new IOException("could not read page in col " + descriptor, e); + } + } + + private void readPageV1(DataPageV1 page) { + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL); + this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); + this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); + try { + BytesInput bytes = page.getBytes(); + LOG.debug("page size " + bytes.size() + " bytes and " + pageValueCount + " records"); + ByteBufferInputStream in = bytes.toInputStream(); + LOG.debug("reading repetition levels at " + in.position()); + rlReader.initFromPage(pageValueCount, in); + LOG.debug("reading definition levels at " + in.position()); + dlReader.initFromPage(pageValueCount, in); + LOG.debug("reading data at " + in.position()); + initDataReader(page.getValueEncoding(), in, page.getValueCount()); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) { + this.pageValueCount = page.getValueCount(); + this.repetitionLevelColumn = + newRLEIterator(descriptor.getMaxRepetitionLevel(), page.getRepetitionLevels()); + this.definitionLevelColumn = + newRLEIterator(descriptor.getMaxDefinitionLevel(), page.getDefinitionLevels()); + try { + LOG.debug( + "page data size " + + page.getData().size() + + " bytes and " + + pageValueCount + + " records"); + initDataReader( + page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount()); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read page " + page + " in col " + descriptor, e); + } + } + + private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) { + try { + if (maxLevel == 0) { + return new NullIntIterator(); + } + return new RLEIntIterator( + new RunLengthBitPackingHybridDecoder( + BytesUtils.getWidthFromMaxInt(maxLevel), + new ByteArrayInputStream(bytes.toByteArray()))); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not read levels in page for col " + descriptor, e); + } + } + + /** + * Utility classes to abstract over different way to read ints with different encodings. + */ + abstract static class IntIterator { + abstract int nextInt(); + } + + /** + * read ints from {@link ValuesReader}. + */ + protected static final class ValuesReaderIntIterator extends IntIterator { + ValuesReader delegate; + + public ValuesReaderIntIterator(ValuesReader delegate) { + this.delegate = delegate; + } + + @Override + int nextInt() { + return delegate.readInteger(); + } + } + + /** + * read ints from {@link RunLengthBitPackingHybridDecoder}. + */ + protected static final class RLEIntIterator extends IntIterator { + RunLengthBitPackingHybridDecoder delegate; + + public RLEIntIterator(RunLengthBitPackingHybridDecoder delegate) { + this.delegate = delegate; + } + + @Override + int nextInt() { + try { + return delegate.readInt(); + } catch (IOException e) { + throw new ParquetDecodingException(e); + } + } + } + + /** + * return zero. + */ + protected static final class NullIntIterator extends IntIterator { + @Override + int nextInt() { + return 0; + } + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java new file mode 100644 index 000000000000..015a867c4f22 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java @@ -0,0 +1,76 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; + +import java.io.IOException; + +/** + * Map {@link ColumnReader}. + */ +public class MapColumnReader implements ColumnReader { + + private final LogicalType logicalType; + private final ArrayColumnReader keyReader; + private final ArrayColumnReader valueReader; + + public MapColumnReader( + ArrayColumnReader keyReader, ArrayColumnReader valueReader, LogicalType logicalType) { + this.keyReader = keyReader; + this.valueReader = valueReader; + this.logicalType = logicalType; + } + + public void readBatch(int total, ColumnVector column) throws IOException { + HeapMapColumnVector mapColumnVector = (HeapMapColumnVector) column; + MapType mapType = (MapType) logicalType; + // initialize 2 ListColumnVector for keys and values + HeapArrayVector keyArrayColumnVector = new HeapArrayVector(total); + HeapArrayVector valueArrayColumnVector = new HeapArrayVector(total); + // read the keys and values + keyReader.readToVector(total, keyArrayColumnVector); + valueReader.readToVector(total, valueArrayColumnVector); + + // set the related attributes according to the keys and values + mapColumnVector.setKeys(keyArrayColumnVector.child); + mapColumnVector.setValues(valueArrayColumnVector.child); + mapColumnVector.setOffsets(keyArrayColumnVector.offsets); + mapColumnVector.setLengths(keyArrayColumnVector.lengths); + mapColumnVector.setSize(keyArrayColumnVector.getSize()); + for (int i = 0; i < keyArrayColumnVector.getLen(); i++) { + if (keyArrayColumnVector.isNullAt(i)) { + mapColumnVector.setNullAt(i); + } + } + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + readBatch(readNumber, vector); + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java new file mode 100644 index 000000000000..e96cf22d29ef --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java @@ -0,0 +1,199 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.TimestampData; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.column.Dictionary; + +import java.io.IOException; + +/** + * The interface to wrap the underlying Parquet dictionary and non dictionary encoded page reader. + */ +public interface ParquetDataColumnReader { + + /** + * Initialize the reader by page data. + * + * @param valueCount value count + * @param in page data + * @throws IOException + */ + void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException; + + /** + * @return the next Dictionary ID from the page + */ + int readValueDictionaryId(); + + /** + * @return the next Long from the page + */ + long readLong(); + + /** + * @return the next Integer from the page + */ + int readInteger(); + + /** + * @return the next SmallInt from the page + */ + int readSmallInt(); + + /** + * @return the next TinyInt from the page + */ + int readTinyInt(); + + /** + * @return the next Float from the page + */ + float readFloat(); + + /** + * @return the next Boolean from the page + */ + boolean readBoolean(); + + /** + * @return the next String from the page + */ + byte[] readString(); + + /** + * @return the next Varchar from the page + */ + byte[] readVarchar(); + + /** + * @return the next Char from the page + */ + byte[] readChar(); + + /** + * @return the next Bytes from the page + */ + byte[] readBytes(); + + /** + * @return the next Decimal from the page + */ + byte[] readDecimal(); + + /** + * @return the next Double from the page + */ + double readDouble(); + + /** + * @return the next TimestampData from the page + */ + TimestampData readTimestamp(); + + /** + * @return is data valid + */ + boolean isValid(); + + /** + * @return the underlying dictionary if current reader is dictionary encoded + */ + Dictionary getDictionary(); + + /** + * @param id in dictionary + * @return the Bytes from the dictionary by id + */ + byte[] readBytes(int id); + + /** + * @param id in dictionary + * @return the Float from the dictionary by id + */ + float readFloat(int id); + + /** + * @param id in dictionary + * @return the Double from the dictionary by id + */ + double readDouble(int id); + + /** + * @param id in dictionary + * @return the Integer from the dictionary by id + */ + int readInteger(int id); + + /** + * @param id in dictionary + * @return the Long from the dictionary by id + */ + long readLong(int id); + + /** + * @param id in dictionary + * @return the Small Int from the dictionary by id + */ + int readSmallInt(int id); + + /** + * @param id in dictionary + * @return the tiny int from the dictionary by id + */ + int readTinyInt(int id); + + /** + * @param id in dictionary + * @return the Boolean from the dictionary by id + */ + boolean readBoolean(int id); + + /** + * @param id in dictionary + * @return the Decimal from the dictionary by id + */ + byte[] readDecimal(int id); + + /** + * @param id in dictionary + * @return the TimestampData from the dictionary by id + */ + TimestampData readTimestamp(int id); + + /** + * @param id in dictionary + * @return the String from the dictionary by id + */ + byte[] readString(int id); + + /** + * @param id in dictionary + * @return the Varchar from the dictionary by id + */ + byte[] readVarchar(int id); + + /** + * @param id in dictionary + * @return the Char from the dictionary by id + */ + byte[] readChar(int id); +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java new file mode 100644 index 000000000000..861d5cb00bbe --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java @@ -0,0 +1,304 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.flink.table.data.TimestampData; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; + +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_MILLISECOND; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND; + +/** + * Parquet file has self-describing schema which may differ from the user required schema (e.g. + * schema evolution). This factory is used to retrieve user required typed data via corresponding + * reader which reads the underlying data. + */ +public final class ParquetDataColumnReaderFactory { + + private ParquetDataColumnReaderFactory() { + } + + /** + * default reader for {@link ParquetDataColumnReader}. + */ + public static class DefaultParquetDataColumnReader implements ParquetDataColumnReader { + protected ValuesReader valuesReader; + protected Dictionary dict; + + // After the data is read in the parquet type, isValid will be set to true if the data can + // be returned in the type defined in HMS. Otherwise isValid is set to false. + boolean isValid = true; + + public DefaultParquetDataColumnReader(ValuesReader valuesReader) { + this.valuesReader = valuesReader; + } + + public DefaultParquetDataColumnReader(Dictionary dict) { + this.dict = dict; + } + + @Override + public void initFromPage(int i, ByteBufferInputStream in) throws IOException { + valuesReader.initFromPage(i, in); + } + + @Override + public boolean readBoolean() { + return valuesReader.readBoolean(); + } + + @Override + public boolean readBoolean(int id) { + return dict.decodeToBoolean(id); + } + + @Override + public byte[] readString(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readString() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readVarchar() { + // we need to enforce the size here even the types are the same + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readVarchar(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readChar() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readChar(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readBytes() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readBytes(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public byte[] readDecimal() { + return valuesReader.readBytes().getBytesUnsafe(); + } + + @Override + public byte[] readDecimal(int id) { + return dict.decodeToBinary(id).getBytesUnsafe(); + } + + @Override + public float readFloat() { + return valuesReader.readFloat(); + } + + @Override + public float readFloat(int id) { + return dict.decodeToFloat(id); + } + + @Override + public double readDouble() { + return valuesReader.readDouble(); + } + + @Override + public double readDouble(int id) { + return dict.decodeToDouble(id); + } + + @Override + public TimestampData readTimestamp() { + throw new RuntimeException("Unsupported operation"); + } + + @Override + public TimestampData readTimestamp(int id) { + throw new RuntimeException("Unsupported operation"); + } + + @Override + public int readInteger() { + return valuesReader.readInteger(); + } + + @Override + public int readInteger(int id) { + return dict.decodeToInt(id); + } + + @Override + public boolean isValid() { + return isValid; + } + + @Override + public long readLong(int id) { + return dict.decodeToLong(id); + } + + @Override + public long readLong() { + return valuesReader.readLong(); + } + + @Override + public int readSmallInt() { + return valuesReader.readInteger(); + } + + @Override + public int readSmallInt(int id) { + return dict.decodeToInt(id); + } + + @Override + public int readTinyInt() { + return valuesReader.readInteger(); + } + + @Override + public int readTinyInt(int id) { + return dict.decodeToInt(id); + } + + @Override + public int readValueDictionaryId() { + return valuesReader.readValueDictionaryId(); + } + + public void skip() { + valuesReader.skip(); + } + + @Override + public Dictionary getDictionary() { + return dict; + } + } + + /** + * The reader who reads from the underlying Timestamp value value. + */ + public static class TypesFromInt96PageReader extends DefaultParquetDataColumnReader { + private final boolean isUtcTimestamp; + + public TypesFromInt96PageReader(ValuesReader realReader, boolean isUtcTimestamp) { + super(realReader); + this.isUtcTimestamp = isUtcTimestamp; + } + + public TypesFromInt96PageReader(Dictionary dict, boolean isUtcTimestamp) { + super(dict); + this.isUtcTimestamp = isUtcTimestamp; + } + + private TimestampData convert(Binary binary) { + ByteBuffer buf = binary.toByteBuffer(); + buf.order(ByteOrder.LITTLE_ENDIAN); + long timeOfDayNanos = buf.getLong(); + int julianDay = buf.getInt(); + return int96ToTimestamp(isUtcTimestamp, timeOfDayNanos, julianDay); + } + + @Override + public TimestampData readTimestamp(int id) { + return convert(dict.decodeToBinary(id)); + } + + @Override + public TimestampData readTimestamp() { + return convert(valuesReader.readBytes()); + } + } + + private static ParquetDataColumnReader getDataColumnReaderByTypeHelper( + boolean isDictionary, + PrimitiveType parquetType, + Dictionary dictionary, + ValuesReader valuesReader, + boolean isUtcTimestamp) { + if (parquetType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96) { + return isDictionary + ? new TypesFromInt96PageReader(dictionary, isUtcTimestamp) + : new TypesFromInt96PageReader(valuesReader, isUtcTimestamp); + } else { + return isDictionary + ? new DefaultParquetDataColumnReader(dictionary) + : new DefaultParquetDataColumnReader(valuesReader); + } + } + + public static ParquetDataColumnReader getDataColumnReaderByTypeOnDictionary( + PrimitiveType parquetType, Dictionary realReader, boolean isUtcTimestamp) { + return getDataColumnReaderByTypeHelper(true, parquetType, realReader, null, isUtcTimestamp); + } + + public static ParquetDataColumnReader getDataColumnReaderByType( + PrimitiveType parquetType, ValuesReader realReader, boolean isUtcTimestamp) { + return getDataColumnReaderByTypeHelper( + false, parquetType, null, realReader, isUtcTimestamp); + } + + private static TimestampData int96ToTimestamp( + boolean utcTimestamp, long nanosOfDay, int julianDay) { + long millisecond = julianDayToMillis(julianDay) + (nanosOfDay / NANOS_PER_MILLISECOND); + + if (utcTimestamp) { + int nanoOfMillisecond = (int) (nanosOfDay % NANOS_PER_MILLISECOND); + return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); + } else { + Timestamp timestamp = new Timestamp(millisecond); + timestamp.setNanos((int) (nanosOfDay % NANOS_PER_SECOND)); + return TimestampData.fromTimestamp(timestamp); + } + } + + private static long julianDayToMillis(int julianDay) { + return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY; + } +} + diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java new file mode 100644 index 000000000000..39ebb90ee6c1 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java @@ -0,0 +1,57 @@ +/* + * 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 org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; + +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.vector.writable.WritableColumnVector; + +import java.io.IOException; +import java.util.List; + +/** + * Row {@link ColumnReader}. + */ +public class RowColumnReader implements ColumnReader { + + private final List fieldReaders; + + public RowColumnReader(List fieldReaders) { + this.fieldReaders = fieldReaders; + } + + @Override + public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { + HeapRowColumnVector rowColumnVector = (HeapRowColumnVector) vector; + WritableColumnVector[] vectors = rowColumnVector.vectors; + for (int i = 0; i < vectors.length; i++) { + fieldReaders.get(i).readToVector(readNumber, vectors[i]); + + for (int j = 0; j < readNumber; j++) { + boolean isNull = (i == 0) + ? vectors[i].isNullAt(j) + : rowColumnVector.isNullAt(j) && vectors[i].isNullAt(j); + if (isNull) { + rowColumnVector.setNullAt(j); + } + } + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 2bf5bd58edb1..7a72bca0582f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.format.FilePathUtils; @@ -179,8 +180,7 @@ public void open(MergeOnReadInputSplit split) throws IOException { } } else if (!split.getBasePath().isPresent()) { // log files only - if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING) - && conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) { + if (OptionsResolver.emitChangelog(conf)) { this.iterator = new LogFileOnlyIterator(getUnMergedLogFileIterator(split)); } else { this.iterator = new LogFileOnlyIterator(getLogFileIterator(split)); @@ -683,13 +683,18 @@ public boolean reachedEnd() throws IOException { // deleted continue; } else { + final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos); + if (!emitDelete && rowKind == RowKind.DELETE) { + // deleted + continue; + } GenericRecord avroRecord = buildAvroRecordBySchema( mergedAvroRecord.get(), requiredSchema, requiredPos, recordBuilder); this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord); - FormatUtils.setRowKind(this.currentRecord, mergedAvroRecord.get(), this.operationPos); + this.currentRecord.setRowKind(rowKind); return false; } } @@ -746,9 +751,6 @@ private Option mergeRowWithLog( RowData curRow, String curKey) throws IOException { final HoodieRecord record = scanner.getRecords().get(curKey); - if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) { - return Option.empty(); - } GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java b/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java index df815a82e4fb..8c9537fe3c66 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java @@ -46,7 +46,7 @@ * SQL API. * *

Note: Changes in this class need to be kept in sync with the corresponding runtime classes - * {@link org.apache.flink.formats.avro.AvroRowDeserializationSchema} and {@link org.apache.flink.formats.avro.AvroRowSerializationSchema}. + * {@code org.apache.flink.formats.avro.AvroRowDeserializationSchema} and {@code org.apache.flink.formats.avro.AvroRowSerializationSchema}. * *

NOTE: reference from Flink release 1.12.0, should remove when Flink version upgrade to that. */ @@ -294,7 +294,7 @@ public static Schema convertToSchema(LogicalType logicalType, String rowName) { } } - private static LogicalType extractValueTypeToAvroMap(LogicalType type) { + public static LogicalType extractValueTypeToAvroMap(LogicalType type) { LogicalType keyType; LogicalType valueType; if (type instanceof MapType) { diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java b/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java index 1ce467f5467e..d903632a6bec 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java @@ -49,8 +49,6 @@ import java.util.List; import java.util.Map; -import static org.apache.flink.formats.avro.typeutils.AvroSchemaConverter.extractValueTypeToAvroMap; - /** * Tool class used to convert from Avro {@link GenericRecord} to {@link RowData}. * @@ -188,7 +186,7 @@ private static AvroToRowDataConverter createMapConverter(LogicalType type) { final AvroToRowDataConverter keyConverter = createConverter(DataTypes.STRING().getLogicalType()); final AvroToRowDataConverter valueConverter = - createNullableConverter(extractValueTypeToAvroMap(type)); + createNullableConverter(AvroSchemaConverter.extractValueTypeToAvroMap(type)); return avroObject -> { final Map map = (Map) avroObject; diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java b/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java index c282f5aebb81..26e21770b168 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java @@ -39,8 +39,6 @@ import java.util.List; import java.util.Map; -import static org.apache.flink.formats.avro.typeutils.AvroSchemaConverter.extractValueTypeToAvroMap; - /** * Tool class used to convert from {@link RowData} to Avro {@link GenericRecord}. * @@ -279,7 +277,7 @@ public Object convert(Schema schema, Object object) { } private static RowDataToAvroConverter createMapConverter(LogicalType type) { - LogicalType valueType = extractValueTypeToAvroMap(type); + LogicalType valueType = AvroSchemaConverter.extractValueTypeToAvroMap(type); final ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(valueType); final RowDataToAvroConverter valueConverter = createConverter(valueType); diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 867621a66d8f..98df0bbcfd86 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -28,11 +28,11 @@ import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -101,8 +101,8 @@ public static TypedProperties getProps(FlinkStreamerConfig cfg) { return new TypedProperties(); } return readConfig( - FSUtils.getFs(cfg.propsFilePath, getHadoopConf()), - new Path(cfg.propsFilePath), cfg.configs).getConfig(); + getHadoopConf(), + new Path(cfg.propsFilePath), cfg.configs).getProps(); } public static Schema getSourceSchema(FlinkStreamerConfig cfg) { @@ -126,19 +126,12 @@ public static Schema getSourceSchema(org.apache.flink.configuration.Configuratio /** * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option). */ - public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List overriddenProps) { - DFSPropertiesConfiguration conf; - try { - conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath); - } catch (Exception e) { - conf = new DFSPropertiesConfiguration(); - LOG.warn("Unexpected error read props file at :" + cfgPath, e); - } - + public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List overriddenProps) { + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath); try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); @@ -239,8 +232,6 @@ public static TypedProperties flinkConf2TypedProperties(Configuration conf) { Properties properties = new Properties(); // put all the set options flatConf.addAllToProperties(properties); - // ugly: table keygen clazz, needed by TwoToThreeUpgradeHandler - properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), conf.getString(FlinkOptions.KEYGEN_CLASS_NAME)); // put all the default options for (ConfigOption option : FlinkOptions.optionalOptions()) { if (!flatConf.contains(option) && option.hasDefaultValue()) { @@ -266,6 +257,7 @@ public static HoodieTableMetaClient initTableIfNotExists(Configuration conf) thr final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf(); if (!tableExists(basePath, hadoopConf)) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder() + .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA)) .setTableType(conf.getString(FlinkOptions.TABLE_TYPE)) .setTableName(conf.getString(FlinkOptions.TABLE_NAME)) .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null)) @@ -374,13 +366,25 @@ public static HoodieTableMetaClient createMetaClient(Configuration conf) { * *

This expects to be used by client, the driver should start an embedded timeline server. */ + @SuppressWarnings("rawtypes") public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext) { + return createWriteClient(conf, runtimeContext, true); + } + + /** + * Creates the Flink write client. + * + *

This expects to be used by client, set flag {@code loadFsViewStorageConfig} to use + * remote filesystem view storage config, or an in-memory filesystem view storage is used. + */ + @SuppressWarnings("rawtypes") + public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext, boolean loadFsViewStorageConfig) { HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( new SerializableConfiguration(getHadoopConf()), new FlinkTaskContextSupplier(runtimeContext)); - HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true); + HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, loadFsViewStorageConfig); return new HoodieFlinkWriteClient<>(context, writeConfig); } @@ -391,17 +395,20 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf, Runti * *

The task context supplier is a constant: the write token is always '0-1-0'. */ + @SuppressWarnings("rawtypes") public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throws IOException { HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true, false); + // build the write client to start the embedded timeline server + final HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, writeConfig); // create the filesystem view storage properties for client - FileSystemViewStorageConfig viewStorageConfig = writeConfig.getViewStorageConfig(); + final FileSystemViewStorageConfig viewStorageConfig = writeConfig.getViewStorageConfig(); // rebuild the view storage config with simplified options. FileSystemViewStorageConfig rebuilt = FileSystemViewStorageConfig.newBuilder() .withStorageType(viewStorageConfig.getStorageType()) .withRemoteServerHost(viewStorageConfig.getRemoteViewServerHost()) .withRemoteServerPort(viewStorageConfig.getRemoteViewServerPort()).build(); ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), rebuilt); - return new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, writeConfig); + return writeClient; } /** @@ -409,12 +416,17 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throw */ public static Option medianInstantTime(String highVal, String lowVal) { try { - long high = HoodieActiveTimeline.parseInstantTime(highVal).getTime(); - long low = HoodieActiveTimeline.parseInstantTime(lowVal).getTime(); + long high = HoodieActiveTimeline.parseDateFromInstantTime(highVal).getTime(); + long low = HoodieActiveTimeline.parseDateFromInstantTime(lowVal).getTime(); ValidationUtils.checkArgument(high > low, "Instant [" + highVal + "] should have newer timestamp than instant [" + lowVal + "]"); long median = low + (high - low) / 2; - return low >= median ? Option.empty() : Option.of(HoodieActiveTimeline.formatInstantTime(new Date(median))); + final String instantTime = HoodieActiveTimeline.formatDate(new Date(median)); + if (HoodieTimeline.compareTimestamps(lowVal, HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime) + || HoodieTimeline.compareTimestamps(highVal, HoodieTimeline.LESSER_THAN_OR_EQUALS, instantTime)) { + return Option.empty(); + } + return Option.of(instantTime); } catch (ParseException e) { throw new HoodieException("Get median instant time with interval [" + lowVal + ", " + highVal + "] error", e); } @@ -425,8 +437,8 @@ public static Option medianInstantTime(String highVal, String lowVal) { */ public static long instantTimeDiffSeconds(String newInstantTime, String oldInstantTime) { try { - long newTimestamp = HoodieActiveTimeline.parseInstantTime(newInstantTime).getTime(); - long oldTimestamp = HoodieActiveTimeline.parseInstantTime(oldInstantTime).getTime(); + long newTimestamp = HoodieActiveTimeline.parseDateFromInstantTime(newInstantTime).getTime(); + long oldTimestamp = HoodieActiveTimeline.parseDateFromInstantTime(oldInstantTime).getTime(); return (newTimestamp - oldTimestamp) / 1000; } catch (ParseException e) { throw new HoodieException("Get instant time diff with interval [" + oldInstantTime + ", " + newInstantTime + "] error", e); diff --git a/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 1031e4274d54..47435c745c46 100644 --- a/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -15,3 +15,4 @@ # limitations under the License. org.apache.hudi.table.HoodieTableFactory +org.apache.hudi.table.catalog.HoodieCatalogFactory diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index be2e334a4c96..d86602ea95c5 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -222,6 +222,32 @@ void testSyncMetadataTable() throws Exception { assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(6L)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "001")); assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.COMMIT_ACTION)); + // write another 2 commits + for (int i = 6; i < 8; i++) { + instant = mockWriteWithMetadata(); + metadataTableMetaClient.reloadActiveTimeline(); + completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); + assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(i + 1L)); + assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + } + + // write another commit to trigger clean + instant = mockWriteWithMetadata(); + metadataTableMetaClient.reloadActiveTimeline(); + completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); + assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(10L)); + assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "002")); + assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.CLEAN_ACTION)); + + // write another commit + mockWriteWithMetadata(); + // write another commit to trigger compaction + instant = mockWriteWithMetadata(); + metadataTableMetaClient.reloadActiveTimeline(); + completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); + assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(13L)); + assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant + "001")); + assertThat(completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.COMMIT_ACTION)); } // ------------------------------------------------------------------------- diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index e0c574bd655f..ebe9140adb2e 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -44,6 +44,8 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.util.Arrays; @@ -58,13 +60,23 @@ * IT cases for {@link org.apache.hudi.common.model.HoodieRecord}. */ public class ITTestHoodieFlinkCompactor { - private static final Map> EXPECTED = new HashMap<>(); + + protected static final Logger LOG = LoggerFactory.getLogger(ITTestHoodieFlinkCompactor.class); + + private static final Map> EXPECTED1 = new HashMap<>(); + + private static final Map> EXPECTED2 = new HashMap<>(); static { - EXPECTED.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1")); - EXPECTED.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2")); - EXPECTED.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3")); - EXPECTED.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4")); + EXPECTED1.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1")); + EXPECTED1.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2")); + EXPECTED1.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3")); + EXPECTED1.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4")); + + EXPECTED2.put("par1", Arrays.asList("id1,par1,id1,Danny,24,1000,par1", "id2,par1,id2,Stephen,34,2000,par1")); + EXPECTED2.put("par2", Arrays.asList("id3,par2,id3,Julian,54,3000,par2", "id4,par2,id4,Fabian,32,4000,par2")); + EXPECTED2.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3", "id9,par3,id9,Jane,19,6000,par3")); + EXPECTED2.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4", "id10,par4,id10,Ella,38,7000,par4", "id11,par4,id11,Phoebe,52,8000,par4")); } @TempDir @@ -148,6 +160,48 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { env.execute("flink_hudi_compaction"); writeClient.close(); - TestData.checkWrittenFullData(tempFile, EXPECTED); + TestData.checkWrittenFullData(tempFile, EXPECTED1); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exception { + // Create hoodie table and insert into data. + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + TableEnvironment tableEnv = TableEnvironmentImpl.create(settings); + tableEnv.getConfig().getConfiguration() + .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + Map options = new HashMap<>(); + options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false"); + options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ"); + options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + ""); + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); + tableEnv.executeSql(hoodieTableDDL); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(5); + + // Make configuration and setAvroSchema. + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + FlinkCompactionConfig cfg = new FlinkCompactionConfig(); + cfg.path = tempFile.getAbsolutePath(); + cfg.minCompactionIntervalSeconds = 3; + cfg.schedule = true; + Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg); + conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ"); + + HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new HoodieFlinkCompactor.AsyncCompactionService(cfg, conf, env); + asyncCompactionService.start(null); + + tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(5); + + asyncCompactionService.shutDown(); + + TestData.checkWrittenFullData(tempFile, EXPECTED2); } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java b/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java index 053c2a39c8e0..4f4b5499530c 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java @@ -401,7 +401,7 @@ public void testWriteProfileMetadataCache() throws Exception { } private static String getLastCompleteInstant(WriteProfile profile) { - return StreamerUtil.getLastCompletedInstant(profile.getTable().getMetaClient()); + return StreamerUtil.getLastCompletedInstant(profile.getMetaClient()); } private void assertBucketEquals( diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java index 285df4931e1d..9e0da3adf351 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java @@ -242,6 +242,33 @@ void testStreamWriteBatchReadOptimized() { assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT); } + @Test + void testStreamWriteReadSkippingCompaction() throws Exception { + // create filesystem table named source + String createSource = TestConfigurations.getFileSourceDDL("source", 4); + streamTableEnv.executeSql(createSource); + + String hoodieTableDDL = sql("t1") + .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) + .option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ) + .option(FlinkOptions.READ_AS_STREAMING, true) + .option(FlinkOptions.READ_STREAMING_SKIP_COMPACT, true) + .option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1) + .option(FlinkOptions.COMPACTION_TASKS, 1) + .end(); + streamTableEnv.executeSql(hoodieTableDDL); + String insertInto = "insert into t1 select * from source"; + execInsertSql(streamTableEnv, insertInto); + + String instant = TestUtils.getNthCompleteInstant(tempFile.getAbsolutePath(), 2, true); + + streamTableEnv.getConfig().getConfiguration() + .setBoolean("table.dynamic-table-options.enabled", true); + final String query = String.format("select * from t1/*+ options('read.start-commit'='%s')*/", instant); + List rows = execSelectSql(streamTableEnv, query, 10); + assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT); + } + @Test void testStreamWriteWithCleaning() { // create filesystem table named source @@ -424,6 +451,8 @@ void testBatchModeUpsertWithoutPartition(HoodieTableType tableType) { String hoodieTableDDL = sql("t1") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) .option(FlinkOptions.TABLE_NAME, tableType.name()) + .option("hoodie.parquet.small.file.limit", "0") // invalidate the small file strategy + .option("hoodie.parquet.max.file.size", "0") .noPartition() .end(); tableEnv.executeSql(hoodieTableDDL); @@ -961,8 +990,9 @@ void testWriteAndReadWithTimestampPartitioning(ExecMode execMode) { + "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]"); } - @Test - void testWriteReadDecimals() { + @ParameterizedTest + @ValueSource(strings = {"bulk_insert", "upsert"}) + void testWriteReadDecimals(String operation) { TableEnvironment tableEnv = batchTableEnv; String createTable = sql("decimals") .field("f0 decimal(3, 2)") @@ -970,7 +1000,7 @@ void testWriteReadDecimals() { .field("f2 decimal(20, 2)") .field("f3 decimal(38, 18)") .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) - .option(FlinkOptions.OPERATION, "bulk_insert") + .option(FlinkOptions.OPERATION, operation) .option(FlinkOptions.PRECOMBINE_FIELD, "f1") .pkField("f0") .noPartition() @@ -1051,6 +1081,62 @@ void testReadWithWiderSchema(HoodieTableType tableType) throws Exception { assertRowsEquals(result, expected); } + @ParameterizedTest + @ValueSource(strings = {"insert", "upsert", "bulk_insert"}) + void testParquetComplexTypes(String operation) { + TableEnvironment tableEnv = batchTableEnv; + + String hoodieTableDDL = sql("t1") + .field("f_int int") + .field("f_array array") + .field("f_map map") + .field("f_row row(f_row_f0 int, f_row_f1 varchar(10))") + .pkField("f_int") + .noPartition() + .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) + .option(FlinkOptions.OPERATION, operation) + .end(); + tableEnv.executeSql(hoodieTableDDL); + + execInsertSql(tableEnv, TestSQL.COMPLEX_TYPE_INSERT_T1); + + List result = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1").execute().collect()); + final String expected = "[" + + "+I[1, [abc1, def1], {abc1=1, def1=3}, +I[1, abc1]], " + + "+I[2, [abc2, def2], {def2=3, abc2=1}, +I[2, abc2]], " + + "+I[3, [abc3, def3], {def3=3, abc3=1}, +I[3, abc3]]]"; + assertRowsEquals(result, expected); + } + + @ParameterizedTest + @ValueSource(strings = {"insert", "upsert", "bulk_insert"}) + void testParquetComplexNestedRowTypes(String operation) { + TableEnvironment tableEnv = batchTableEnv; + + String hoodieTableDDL = sql("t1") + .field("f_int int") + .field("f_array array") + .field("f_map map") + .field("f_row row(f_nested_array array, f_nested_row row(f_row_f0 int, f_row_f1 varchar(10)))") + .pkField("f_int") + .noPartition() + .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) + .option(FlinkOptions.OPERATION, operation) + .end(); + tableEnv.executeSql(hoodieTableDDL); + + execInsertSql(tableEnv, TestSQL.COMPLEX_NESTED_ROW_TYPE_INSERT_T1); + + List result = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1").execute().collect()); + final String expected = "[" + + "+I[1, [abc1, def1], {abc1=1, def1=3}, +I[[abc1, def1], +I[1, abc1]]], " + + "+I[2, [abc2, def2], {def2=3, abc2=1}, +I[[abc2, def2], +I[2, abc2]]], " + + "+I[3, [abc3, def3], {def3=3, abc3=1}, +I[[abc3, def3], +I[3, abc3]]]]"; + assertRowsEquals(result, expected); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java b/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java new file mode 100644 index 000000000000..18a2369ffd4e --- /dev/null +++ b/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java @@ -0,0 +1,258 @@ +/* + * 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 org.apache.hudi.table.catalog; + +import org.apache.hudi.configuration.FlinkOptions; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; +import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases for {@link HoodieCatalog}. + */ +public class TestHoodieCatalog { + + private static final String TEST_DEFAULT_DATABASE = "test_db"; + private static final String NONE_EXIST_DATABASE = "none_exist_database"; + private static final List CREATE_COLUMNS = Arrays.asList( + Column.physical("uuid", DataTypes.VARCHAR(20)), + Column.physical("name", DataTypes.VARCHAR(20)), + Column.physical("age", DataTypes.INT()), + Column.physical("tss", DataTypes.TIMESTAMP(3)), + Column.physical("partition", DataTypes.VARCHAR(10)) + ); + private static final UniqueConstraint CONSTRAINTS = UniqueConstraint.primaryKey("uuid", Arrays.asList("uuid")); + private static final ResolvedSchema CREATE_TABLE_SCHEMA = + new ResolvedSchema( + CREATE_COLUMNS, + Collections.emptyList(), + CONSTRAINTS); + + private static final List EXPECTED_TABLE_COLUMNS = + CREATE_COLUMNS.stream() + .map( + col -> { + // Flink char/varchar is transform to string in avro. + if (col.getDataType() + .getLogicalType() + .getTypeRoot() + .equals(LogicalTypeRoot.VARCHAR)) { + return Column.physical(col.getName(), DataTypes.STRING()); + } else { + return col; + } + }) + .collect(Collectors.toList()); + private static final ResolvedSchema EXPECTED_TABLE_SCHEMA = + new ResolvedSchema(EXPECTED_TABLE_COLUMNS, Collections.emptyList(), CONSTRAINTS); + + private static final Map EXPECTED_OPTIONS = new HashMap<>(); + static { + EXPECTED_OPTIONS.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + EXPECTED_OPTIONS.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false"); + EXPECTED_OPTIONS.put(FlinkOptions.PRE_COMBINE.key(), "true"); + } + + private static final ResolvedCatalogTable EXPECTED_CATALOG_TABLE = new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(CREATE_TABLE_SCHEMA).build(), + "test", + Arrays.asList("partition"), + EXPECTED_OPTIONS), + CREATE_TABLE_SCHEMA + ); + + private TableEnvironment streamTableEnv; + private HoodieCatalog catalog; + + @TempDir + File tempFile; + + @BeforeEach + void beforeEach() { + EnvironmentSettings settings = EnvironmentSettings.newInstance().build(); + streamTableEnv = TableEnvironmentImpl.create(settings); + streamTableEnv.getConfig().getConfiguration() + .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + File testDb = new File(tempFile, TEST_DEFAULT_DATABASE); + testDb.mkdir(); + Map catalogOptions = new HashMap<>(); + catalogOptions.put(CATALOG_PATH.key(), tempFile.getAbsolutePath()); + catalogOptions.put(DEFAULT_DATABASE.key(), TEST_DEFAULT_DATABASE); + catalog = new HoodieCatalog("hudi", Configuration.fromMap(catalogOptions)); + catalog.open(); + } + + @Test + public void testListDatabases() { + List actual = catalog.listDatabases(); + assertTrue(actual.contains(TEST_DEFAULT_DATABASE)); + assertFalse(actual.contains(NONE_EXIST_DATABASE)); + } + + @Test + public void testDatabaseExists() { + assertTrue(catalog.databaseExists(TEST_DEFAULT_DATABASE)); + assertFalse(catalog.databaseExists(NONE_EXIST_DATABASE)); + } + + @Test + public void testCreateAndDropDatabase() throws Exception { + CatalogDatabase expected = new CatalogDatabaseImpl(Collections.emptyMap(), null); + catalog.createDatabase("db1", expected, true); + + CatalogDatabase actual = catalog.getDatabase("db1"); + assertTrue(catalog.listDatabases().contains("db1")); + assertEquals(expected.getProperties(), actual.getProperties()); + + // drop exist database + catalog.dropDatabase("db1", true); + assertFalse(catalog.listDatabases().contains("db1")); + + // drop non-exist database + assertThrows(DatabaseNotExistException.class, + () -> catalog.dropDatabase(NONE_EXIST_DATABASE, false)); + } + + @Test + public void testCreateDatabaseWithOptions() { + Map options = new HashMap<>(); + options.put("k1", "v1"); + options.put("k2", "v2"); + + assertThrows( + CatalogException.class, + () -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, null), true)); + } + + @Test + public void testCreateTable() throws Exception { + ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); + // test create table + catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true); + + // test table exist + assertTrue(catalog.tableExists(tablePath)); + + // test create exist table + assertThrows(TableAlreadyExistException.class, + () -> catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, false)); + } + + @Test + public void testListTable() throws Exception { + ObjectPath tablePath1 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); + ObjectPath tablePath2 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb2"); + + // create table + catalog.createTable(tablePath1, EXPECTED_CATALOG_TABLE, true); + catalog.createTable(tablePath2, EXPECTED_CATALOG_TABLE, true); + + // test list table + List tables = catalog.listTables(TEST_DEFAULT_DATABASE); + assertTrue(tables.contains(tablePath1.getObjectName())); + assertTrue(tables.contains(tablePath2.getObjectName())); + + // test list non-exist database table + assertThrows(DatabaseNotExistException.class, + () -> catalog.listTables(NONE_EXIST_DATABASE)); + } + + @Test + public void testGetTable() throws Exception { + ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); + // create table + catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true); + + Map expectedOptions = new HashMap<>(EXPECTED_OPTIONS); + expectedOptions.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + expectedOptions.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false"); + expectedOptions.put(FlinkOptions.PRE_COMBINE.key(), "true"); + expectedOptions.put("connector", "hudi"); + expectedOptions.put( + FlinkOptions.PATH.key(), + String.format("%s/%s/%s", tempFile.getAbsolutePath(), tablePath.getDatabaseName(), tablePath.getObjectName())); + + // test get table + CatalogBaseTable actualTable = catalog.getTable(tablePath); + // validate schema + Schema actualSchema = actualTable.getUnresolvedSchema(); + Schema expectedSchema = Schema.newBuilder().fromResolvedSchema(EXPECTED_TABLE_SCHEMA).build(); + assertEquals(expectedSchema, actualSchema); + // validate options + Map actualOptions = actualTable.getOptions(); + assertEquals(expectedOptions, actualOptions); + // validate comment + assertEquals(EXPECTED_CATALOG_TABLE.getComment(), actualTable.getComment()); + // validate partition key + assertEquals(EXPECTED_CATALOG_TABLE.getPartitionKeys(),((CatalogTable) actualTable).getPartitionKeys()); + } + + @Test + public void dropTable() throws Exception { + ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); + // create table + catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true); + + // test drop table + catalog.dropTable(tablePath, true); + assertFalse(catalog.tableExists(tablePath)); + + // drop non-exist table + assertThrows(TableNotExistException.class, + () -> catalog.dropTable(new ObjectPath(TEST_DEFAULT_DATABASE, "non_exist"), false)); + } +} diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java b/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java index f4da947f3bfc..6fbbab81fa4a 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java @@ -39,6 +39,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; @@ -221,8 +222,9 @@ void testReadBaseAndLogFilesWithDeletes() throws Exception { assertThat(actual2, is(expected2)); } - @Test - void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testReadBaseAndLogFilesWithDisorderUpdateDelete(boolean compact) throws Exception { Map options = new HashMap<>(); options.put(FlinkOptions.CHANGELOG_ENABLED.key(), "true"); beforeEach(HoodieTableType.MERGE_ON_READ, options); @@ -233,7 +235,7 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { TestData.writeData(TestData.DATA_SET_SINGLE_INSERT, conf); // write another commit using logs and read again. - conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); + conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, compact); TestData.writeData(TestData.DATA_SET_DISORDER_UPDATE_DELETE, conf); InputFormat inputFormat = this.tableSource.getInputFormat(); @@ -242,9 +244,11 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { // when isEmitDelete is false. List result1 = readData(inputFormat); + final String rowKind = compact ? "I" : "U"; + final String expected = "[+" + rowKind + "[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; + final String actual1 = TestData.rowDataToString(result1); - final String expected1 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; - assertThat(actual1, is(expected1)); + assertThat(actual1, is(expected)); // refresh the input format and set isEmitDelete to true. this.tableSource.reset(); @@ -254,8 +258,7 @@ void testReadBaseAndLogFilesWithDisorderUpdateDelete() throws Exception { List result2 = readData(inputFormat); final String actual2 = TestData.rowDataToString(result2); - final String expected2 = "[+U[id1, Danny, 22, 1970-01-01T00:00:00.004, par1]]"; - assertThat(actual2, is(expected2)); + assertThat(actual2, is(expected)); } @Test @@ -460,6 +463,32 @@ void testReadWithWiderSchema(HoodieTableType tableType) throws Exception { TestData.assertRowDataEquals(result, TestData.DATA_SET_INSERT); } + /** + * Test reading file groups with compaction plan scheduled and delta logs. + * File-slice after pending compaction-requested instant-time should also be considered valid. + */ + @Test + void testReadMORWithCompactionPlanScheduled() throws Exception { + Map options = new HashMap<>(); + // compact for each commit + options.put(FlinkOptions.COMPACTION_DELTA_COMMITS.key(), "1"); + options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false"); + beforeEach(HoodieTableType.MERGE_ON_READ, options); + + // write three commits + for (int i = 0; i < 6; i += 2) { + List dataset = TestData.dataSetInsert(i + 1, i + 2); + TestData.writeData(dataset, conf); + } + + InputFormat inputFormat1 = this.tableSource.getInputFormat(); + assertThat(inputFormat1, instanceOf(MergeOnReadInputFormat.class)); + + List actual = readData(inputFormat1); + final List expected = TestData.dataSetInsert(1, 2, 3, 4, 5, 6); + TestData.assertRowDataEquals(actual, expected); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java index 0eafb1281ff4..46cad3e826d3 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java @@ -247,6 +247,11 @@ public Sql option(ConfigOption option, Object val) { return this; } + public Sql option(String key, Object val) { + this.options.put(key, val.toString()); + return this; + } + public Sql options(Map options) { this.options.putAll(options); return this; diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java index 9dc78aa4cf27..595d142b7cc0 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java @@ -51,4 +51,14 @@ private TestSQL() { + "('id9','Jane',19,TIMESTAMP '1970-01-01 00:00:06','par3'),\n" + "('id10','Ella',38,TIMESTAMP '1970-01-01 00:00:07','par4'),\n" + "('id11','Phoebe',52,TIMESTAMP '1970-01-01 00:00:08','par4')"; + + public static final String COMPLEX_TYPE_INSERT_T1 = "insert into t1 values\n" + + "(1, array['abc1', 'def1'], map['abc1', 1, 'def1', 3], row(1, 'abc1')),\n" + + "(2, array['abc2', 'def2'], map['abc2', 1, 'def2', 3], row(2, 'abc2')),\n" + + "(3, array['abc3', 'def3'], map['abc3', 1, 'def3', 3], row(3, 'abc3'))"; + + public static final String COMPLEX_NESTED_ROW_TYPE_INSERT_T1 = "insert into t1 values\n" + + "(1, array['abc1', 'def1'], map['abc1', 1, 'def1', 3], row(array['abc1', 'def1'], row(1, 'abc1'))),\n" + + "(2, array['abc2', 'def2'], map['abc2', 1, 'def2', 3], row(array['abc2', 'def2'], row(2, 'abc2'))),\n" + + "(3, array['abc3', 'def3'], map['abc3', 1, 'def3', 3], row(array['abc3', 'def3'], row(3, 'abc3')))"; } diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java index c05e5b056344..57297c50ee82 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java @@ -19,9 +19,12 @@ package org.apache.hudi.utils; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.util.StreamerUtil; +import org.apache.hudi.util.ViewStorageProperties; import org.apache.flink.configuration.Configuration; import org.junit.jupiter.api.Test; @@ -80,12 +83,15 @@ void testInitTableIfNotExists() throws IOException { void testMedianInstantTime() { String higher = "20210705125921"; String lower = "20210705125806"; + String expectedMedianInstant = "20210705125844499"; String median1 = StreamerUtil.medianInstantTime(higher, lower).get(); - assertThat(median1, is("20210705125843")); + assertThat(median1, is(expectedMedianInstant)); // test symmetry assertThrows(IllegalArgumentException.class, () -> StreamerUtil.medianInstantTime(lower, higher), "The first argument should have newer instant time"); + // test very near instant time + assertFalse(StreamerUtil.medianInstantTime("20211116115634", "20211116115633").isPresent()); } @Test @@ -95,5 +101,13 @@ void testInstantTimeDiff() { long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower); assertThat(diff, is(75L)); } + + @Test + void testDumpRemoteViewStorageConfig() throws IOException { + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + StreamerUtil.createWriteClient(conf); + FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH)); + assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST)); + } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java b/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java index 92e16cd1059b..466ccdfd01e7 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java +++ b/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java @@ -29,6 +29,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import javax.annotation.Nullable; + import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -64,6 +66,17 @@ public static String getFirstCompleteInstant(String basePath) { .map(HoodieInstant::getTimestamp).orElse(null); } + @Nullable + public static String getNthCompleteInstant(String basePath, int n, boolean isDelta) { + final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + return metaClient.getActiveTimeline() + .filterCompletedInstants() + .filter(instant -> isDelta ? HoodieTimeline.DELTA_COMMIT_ACTION.equals(instant.getAction()) : HoodieTimeline.COMMIT_ACTION.equals(instant.getAction())) + .nthInstant(n).map(HoodieInstant::getTimestamp) + .orElse(null); + } + public static String getSplitPartitionPath(MergeOnReadInputSplit split) { assertTrue(split.getLogPaths().isPresent()); final String logPath = split.getLogPaths().get().get(0); diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index a33b2e292560..bf87bfaa36a8 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java index 34e0a392bd9d..c9afa9119c0c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java @@ -18,6 +18,8 @@ package org.apache.hudi.hadoop; +import org.apache.hudi.common.model.HoodieLogFile; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.FileSplit; @@ -36,7 +38,7 @@ public class BaseFileWithLogsSplit extends FileSplit { // a flag to mark this split is produced by incremental query or not. private boolean belongToIncrementalSplit = false; // the log file paths of this split. - private List deltaLogPaths = new ArrayList<>(); + private List deltaLogFiles = new ArrayList<>(); // max commit time of current split. private String maxCommitTime = ""; // the basePath of current hoodie table. @@ -55,9 +57,10 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, maxCommitTime); Text.writeString(out, basePath); Text.writeString(out, baseFilePath); - out.writeInt(deltaLogPaths.size()); - for (String logPath : deltaLogPaths) { - Text.writeString(out, logPath); + out.writeInt(deltaLogFiles.size()); + for (HoodieLogFile logFile : deltaLogFiles) { + Text.writeString(out, logFile.getPath().toString()); + out.writeLong(logFile.getFileSize()); } } @@ -69,11 +72,13 @@ public void readFields(DataInput in) throws IOException { basePath = Text.readString(in); baseFilePath = Text.readString(in); int deltaLogSize = in.readInt(); - List tempDeltaLogs = new ArrayList<>(); + List tempDeltaLogs = new ArrayList<>(); for (int i = 0; i < deltaLogSize; i++) { - tempDeltaLogs.add(Text.readString(in)); + String logPath = Text.readString(in); + long logFileSize = in.readLong(); + tempDeltaLogs.add(new HoodieLogFile(new Path(logPath), logFileSize)); } - deltaLogPaths = tempDeltaLogs; + deltaLogFiles = tempDeltaLogs; } public boolean getBelongToIncrementalSplit() { @@ -84,12 +89,12 @@ public void setBelongToIncrementalSplit(boolean belongToIncrementalSplit) { this.belongToIncrementalSplit = belongToIncrementalSplit; } - public List getDeltaLogPaths() { - return deltaLogPaths; + public List getDeltaLogFiles() { + return deltaLogFiles; } - public void setDeltaLogPaths(List deltaLogPaths) { - this.deltaLogPaths = deltaLogPaths; + public void setDeltaLogFiles(List deltaLogFiles) { + this.deltaLogFiles = deltaLogFiles; } public String getMaxCommitTime() { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 810e6ecb413f..c067042857d9 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -59,7 +59,7 @@ /** * HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths * that does not correspond to a hoodie table then they are passed in as is (as what FileInputFormat.listStatus() - * would do). The JobConf could have paths from multipe Hoodie/Non-Hoodie tables + * would do). The JobConf could have paths from multiple Hoodie/Non-Hoodie tables */ @UseRecordReaderFromInputFormat @UseFileSplitsFromInputFormat diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index c797f59efc03..32dfd7c1e4ec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -71,18 +71,13 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial /** * Paths that are known to be non-hoodie tables. */ - private Set nonHoodiePathCache; + Set nonHoodiePathCache; /** * Table Meta Client Cache. */ Map metaClientCache; - /** - * HoodieTableFileSystemView Cache. - */ - private Map hoodieTableFileSystemViewCache; - /** * Hadoop configurations for the FileSystem. */ @@ -102,7 +97,6 @@ public HoodieROTablePathFilter(Configuration conf) { this.nonHoodiePathCache = new HashSet<>(); this.conf = new SerializableConfiguration(conf); this.metaClientCache = new HashMap<>(); - this.hoodieTableFileSystemViewCache = new HashMap<>(); } /** @@ -173,6 +167,13 @@ public boolean accept(Path path) { } if (baseDir != null) { + // Check whether baseDir in nonHoodiePathCache + if (nonHoodiePathCache.contains(baseDir.toString())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Accepting non-hoodie path from cache: " + path); + } + return true; + } HoodieTableFileSystemView fsView = null; try { HoodieTableMetaClient metaClient = metaClientCache.get(baseDir.toString()); @@ -181,15 +182,8 @@ public boolean accept(Path path) { metaClientCache.put(baseDir.toString(), metaClient); } - HoodieTableMetaClient finalMetaClient = metaClient; - fsView = hoodieTableFileSystemViewCache.computeIfAbsent(baseDir.toString(), key -> - FileSystemViewManager.createInMemoryFileSystemView( - engineContext, - finalMetaClient, - HoodieInputFormatUtils.buildMetadataConfig(getConf()) - ) - ); - + fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf())); String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder); List latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList()); // populate the cache @@ -211,10 +205,15 @@ public boolean accept(Path path) { } catch (TableNotFoundException e) { // Non-hoodie path, accept it. if (LOG.isDebugEnabled()) { - LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString())); + LOG.debug(String.format("(1) Caching non-hoodie path under %s with basePath %s \n", folder.toString(), baseDir.toString())); } nonHoodiePathCache.add(folder.toString()); + nonHoodiePathCache.add(baseDir.toString()); return true; + } finally { + if (fsView != null) { + fsView.close(); + } } } else { // files is at < 3 level depth in FS tree, can't be hoodie dataset diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java index 5b4e535e62d1..8f9ac8b03d57 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java @@ -18,6 +18,8 @@ package org.apache.hudi.hadoop; +import org.apache.hudi.common.model.HoodieLogFile; + import org.apache.hadoop.fs.Path; import java.util.ArrayList; @@ -31,7 +33,7 @@ public class PathWithLogFilePath extends Path { // a flag to mark this split is produced by incremental query or not. private boolean belongToIncrementalPath = false; // the log files belong this path. - private List deltaLogPaths = new ArrayList<>(); + private List deltaLogFiles = new ArrayList<>(); // max commit time of current path. private String maxCommitTime = ""; // the basePath of current hoodie table. @@ -50,12 +52,12 @@ public void setBelongToIncrementalPath(boolean belongToIncrementalPath) { this.belongToIncrementalPath = belongToIncrementalPath; } - public List getDeltaLogPaths() { - return deltaLogPaths; + public List getDeltaLogFiles() { + return deltaLogFiles; } - public void setDeltaLogPaths(List deltaLogPaths) { - this.deltaLogPaths = deltaLogPaths; + public void setDeltaLogFiles(List deltaLogFiles) { + this.deltaLogFiles = deltaLogFiles; } public String getMaxCommitTime() { @@ -97,7 +99,7 @@ public boolean includeBootstrapFilePath() { public BaseFileWithLogsSplit buildSplit(Path file, long start, long length, String[] hosts) { BaseFileWithLogsSplit bs = new BaseFileWithLogsSplit(file, start, length, hosts); bs.setBelongToIncrementalSplit(belongToIncrementalPath); - bs.setDeltaLogPaths(deltaLogPaths); + bs.setDeltaLogFiles(deltaLogFiles); bs.setMaxCommitTime(maxCommitTime); bs.setBasePath(basePath); bs.setBaseFilePath(baseFilePath); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index 542720b4919b..e8e1a28987c5 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -18,6 +18,8 @@ package org.apache.hudi.hadoop; +import org.apache.hudi.common.model.HoodieLogFile; + import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -35,7 +37,7 @@ public class RealtimeFileStatus extends FileStatus { // a flag to mark this split is produced by incremental query or not. private boolean belongToIncrementalFileStatus = false; // the log files belong this fileStatus. - private List deltaLogPaths = new ArrayList<>(); + private List deltaLogFiles = new ArrayList<>(); // max commit time of current fileStatus. private String maxCommitTime = ""; // the basePath of current hoodie table. @@ -55,7 +57,7 @@ public Path getPath() { Path path = super.getPath(); PathWithLogFilePath pathWithLogFilePath = new PathWithLogFilePath(path.getParent(), path.getName()); pathWithLogFilePath.setBelongToIncrementalPath(belongToIncrementalFileStatus); - pathWithLogFilePath.setDeltaLogPaths(deltaLogPaths); + pathWithLogFilePath.setDeltaLogFiles(deltaLogFiles); pathWithLogFilePath.setMaxCommitTime(maxCommitTime); pathWithLogFilePath.setBasePath(basePath); pathWithLogFilePath.setBaseFilePath(baseFilePath); @@ -69,12 +71,12 @@ public void setBelongToIncrementalFileStatus(boolean belongToIncrementalFileStat this.belongToIncrementalFileStatus = belongToIncrementalFileStatus; } - public List getDeltaLogPaths() { - return deltaLogPaths; + public List getDeltaLogFiles() { + return deltaLogFiles; } - public void setDeltaLogPaths(List deltaLogPaths) { - this.deltaLogPaths = deltaLogPaths; + public void setDeltaLogFiles(List deltaLogFiles) { + this.deltaLogFiles = deltaLogFiles; } public String getMaxCommitTime() { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java index 7173c9a8f0d7..c24c75359f58 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java @@ -26,7 +26,6 @@ import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.common.StringInternUtils; @@ -183,8 +182,8 @@ private InputSplit[] getCombineSplits(JobConf job, int numSplits, Map " + writerSchema.getFields()); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index cb529cbbf7cf..e683840c6f4d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -189,7 +190,7 @@ private List collectAllIncrementalFiles(List fileGr fileStatus.setBelongToIncrementalFileStatus(true); fileStatus.setBasePath(basePath); fileStatus.setBaseFilePath(baseFilePath); - fileStatus.setDeltaLogPaths(f.getLatestFileSlice().get().getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList())); + fileStatus.setDeltaLogFiles(f.getLatestFileSlice().get().getLogFiles().collect(Collectors.toList())); // try to set bootstrapfileStatus if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { fileStatus.setBootStrapFileStatus(baseFileStatus); @@ -202,7 +203,7 @@ private List collectAllIncrementalFiles(List fileGr if (logFileStatus.size() > 0) { RealtimeFileStatus fileStatus = new RealtimeFileStatus(logFileStatus.get(0)); fileStatus.setBelongToIncrementalFileStatus(true); - fileStatus.setDeltaLogPaths(logFileStatus.stream().map(l -> l.getPath().toString()).collect(Collectors.toList())); + fileStatus.setDeltaLogFiles(logFileStatus.stream().map(l -> new HoodieLogFile(l.getPath(), l.getLen())).collect(Collectors.toList())); fileStatus.setMaxCommitTime(maxCommitTime); fileStatus.setBasePath(basePath); result.add(fileStatus); @@ -256,7 +257,7 @@ private FileSplit doMakeSplitForPathWithLogFilePath(PathWithLogFilePath path, lo ? super.makeSplit(path.getPathWithBootstrapFileStatus(), start, length, hosts) : super.makeSplit(path.getPathWithBootstrapFileStatus(), start, length, hosts, inMemoryHosts); return HoodieRealtimeInputFormatUtils - .createRealtimeBoostrapBaseFileSplit((BootstrapBaseFileSplit) bf, path.getBasePath(), path.getDeltaLogPaths(), path.getMaxCommitTime()); + .createRealtimeBoostrapBaseFileSplit((BootstrapBaseFileSplit) bf, path.getBasePath(), path.getDeltaLogFiles(), path.getMaxCommitTime()); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 6423f2cfd46e..a39ec35507a7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hadoop.mapred.FileSplit; @@ -25,7 +26,9 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; /** * Filesplit that wraps the base split and a list of log files to merge deltas from. @@ -33,6 +36,7 @@ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { private List deltaLogPaths; + private List deltaLogFiles = new ArrayList<>(); private String maxCommitTime; @@ -44,11 +48,12 @@ public HoodieRealtimeFileSplit() { super(); } - public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogPaths, String maxCommitTime, + public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxCommitTime, Option hoodieVirtualKeyInfo) throws IOException { super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); - this.deltaLogPaths = deltaLogPaths; + this.deltaLogFiles = deltaLogFiles; + this.deltaLogPaths = deltaLogFiles.stream().map(entry -> entry.getPath().toString()).collect(Collectors.toList()); this.maxCommitTime = maxCommitTime; this.basePath = basePath; this.hoodieVirtualKeyInfo = hoodieVirtualKeyInfo; @@ -58,6 +63,10 @@ public List getDeltaLogPaths() { return deltaLogPaths; } + public List getDeltaLogFiles() { + return deltaLogFiles; + } + public String getMaxCommitTime() { return maxCommitTime; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java index 4da310da4fba..79d2d815ee80 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; @@ -26,7 +27,9 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; /** * Realtime File Split with external base file. @@ -34,6 +37,7 @@ public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { private List deltaLogPaths; + private List deltaLogFiles = new ArrayList<>(); private String maxInstantTime; @@ -43,11 +47,12 @@ public RealtimeBootstrapBaseFileSplit() { super(); } - public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, String basePath, List deltaLogPaths, + public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxInstantTime, FileSplit externalFileSplit) throws IOException { super(baseSplit, externalFileSplit); this.maxInstantTime = maxInstantTime; - this.deltaLogPaths = deltaLogPaths; + this.deltaLogFiles = deltaLogFiles; + this.deltaLogPaths = deltaLogFiles.stream().map(entry -> entry.getPath().toString()).collect(Collectors.toList()); this.basePath = basePath; } @@ -68,6 +73,11 @@ public List getDeltaLogPaths() { return deltaLogPaths; } + @Override + public List getDeltaLogFiles() { + return deltaLogFiles; + } + @Override public String getMaxCommitTime() { return maxInstantTime; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index bad5e1982e0f..f00efa5efaaa 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -53,6 +53,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader private final Map> deltaRecordMap; private final Set deltaRecordKeys; + private final HoodieMergedLogRecordScanner mergedLogRecordScanner; private int recordKeyIndex = HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS; private Iterator deltaItr; @@ -60,7 +61,8 @@ public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, RecordReader realReader) throws IOException { super(split, job); this.parquetReader = realReader; - this.deltaRecordMap = getMergedLogRecordScanner().getRecords(); + this.mergedLogRecordScanner = getMergedLogRecordScanner(); + this.deltaRecordMap = mergedLogRecordScanner.getRecords(); this.deltaRecordKeys = new HashSet<>(this.deltaRecordMap.keySet()); if (split.getHoodieVirtualKeyInfo().isPresent()) { this.recordKeyIndex = split.getHoodieVirtualKeyInfo().get().getRecordKeyFieldIndex(); @@ -189,6 +191,9 @@ public long getPos() throws IOException { @Override public void close() throws IOException { parquetReader.close(); + // need clean the tmp file which created by logScanner + // Otherwise, for resident process such as presto, the /tmp directory will overflow + mergedLogRecordScanner.close(); } @Override diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java index 108613c18282..a7f0d2cc2f5e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.InputSplitUtils; @@ -41,6 +42,8 @@ public interface RealtimeSplit extends InputSplitWithLocationInfo { */ List getDeltaLogPaths(); + List getDeltaLogFiles(); + /** * Return Max Instant Time. * @return diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 25dde840c13b..4f4b69f9813a 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -466,7 +466,7 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map< HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient -> FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline)); List filteredBaseFiles = new ArrayList<>(); - Map> filteredLogs = new HashMap<>(); + Map> filteredLogs = new HashMap<>(); for (Path p : entry.getValue()) { String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p); List matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList()); @@ -476,9 +476,8 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map< .filter(f -> !f.getBaseFile().isPresent() && f.getLatestLogFile().isPresent()) .collect(Collectors.toList()); logMatched.forEach(f -> { - List logPaths = f.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) - .map(log -> log.getPath().toString()).collect(Collectors.toList()); - filteredLogs.put(f.getLatestLogFile().get().getFileStatus(), logPaths); + List logPathSizePairs = f.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); + filteredLogs.put(f.getLatestLogFile().get().getFileStatus(), logPathSizePairs); }); } } @@ -492,9 +491,9 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map< returns.add(getFileStatus(filteredFile)); } - for (Map.Entry> filterLogEntry : filteredLogs.entrySet()) { + for (Map.Entry> filterLogEntry : filteredLogs.entrySet()) { RealtimeFileStatus rs = new RealtimeFileStatus(filterLogEntry.getKey()); - rs.setDeltaLogPaths(filterLogEntry.getValue()); + rs.setDeltaLogFiles(filterLogEntry.getValue()); returns.add(rs); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 9140969c6021..09338d330430 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -120,17 +120,13 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream dataFileSplits = groupedInputSplits.getOrDefault(fileSlice.getFileId(), new ArrayList<>()); dataFileSplits.forEach(split -> { try { - List logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) - .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()); + List logFiles = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) + .collect(Collectors.toList()); if (split instanceof BootstrapBaseFileSplit) { BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit) split; - rtSplits.add(createRealtimeBoostrapBaseFileSplit(eSplit, metaClient.getBasePath(), logFilePaths, maxCommitTime)); - } else if (split instanceof BaseFileWithLogsSplit) { - BaseFileWithLogsSplit bs = (BaseFileWithLogsSplit)split; - HoodieRealtimeFileSplit hoodieRealtimeFileSplit = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogPaths(), bs.getMaxCommitTime(), finalHoodieVirtualKeyInfo); - rtSplits.add(hoodieRealtimeFileSplit); + rtSplits.add(createRealtimeBoostrapBaseFileSplit(eSplit, metaClient.getBasePath(), logFiles, maxCommitTime)); } else { - rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime, finalHoodieVirtualKeyInfo)); + rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFiles, maxCommitTime, finalHoodieVirtualKeyInfo)); } } catch (IOException e) { throw new HoodieIOException("Error creating hoodie real time split ", e); @@ -166,7 +162,7 @@ public static InputSplit[] getIncrementalRealtimeSplits(Configuration conf, Stre if (s instanceof BaseFileWithLogsSplit) { BaseFileWithLogsSplit bs = (BaseFileWithLogsSplit)s; if (bs.getBelongToIncrementalSplit()) { - rtSplits.add(new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogPaths(), bs.getMaxCommitTime(), finalHoodieVirtualKeyInfo)); + rtSplits.add(new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), finalHoodieVirtualKeyInfo)); } } else if (s instanceof RealtimeBootstrapBaseFileSplit) { rtSplits.add(s); @@ -210,7 +206,7 @@ public static boolean isIncrementalQuerySplits(List fileSplits) { } public static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit( - BootstrapBaseFileSplit split, String basePath, List deltaLogPaths, String maxInstantTime) { + BootstrapBaseFileSplit split, String basePath, List logFiles, String maxInstantTime) { try { String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; @@ -218,7 +214,7 @@ public static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; FileSplit baseSplit = new FileSplit(split.getPath(), split.getStart(), split.getLength(), hosts, inMemoryHosts); - return new RealtimeBootstrapBaseFileSplit(baseSplit, basePath, deltaLogPaths, maxInstantTime, split.getBootstrapFileSplit()); + return new RealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit()); } catch (IOException e) { throw new HoodieIOException("Error creating hoodie real time split ", e); } @@ -241,7 +237,7 @@ public static List, List>> groupLogsByBaseFi try { // Both commit and delta-commits are included - pick the latest completed one Option latestCompletedInstant = - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants().lastInstant(); Stream latestFileSlices = latestCompletedInstant .map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp())) diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java index ba88df3e4c4e..260afd5ced03 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java @@ -68,9 +68,11 @@ public void testHoodiePaths() throws Exception { assertFalse(pathFilter.accept(testTable.getInflightCommitFilePath("003"))); assertFalse(pathFilter.accept(testTable.getRequestedCompactionFilePath("004"))); assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"))); + assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/hoodie.properties"))); assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME))); assertEquals(1, pathFilter.metaClientCache.size()); + assertEquals(0, pathFilter.nonHoodiePathCache.size(), "NonHoodiePathCache size should be 0"); } @Test @@ -82,6 +84,7 @@ public void testNonHoodiePaths() throws IOException { java.nio.file.Path path2 = Paths.get(basePath, "nonhoodiefolder/somefile"); Files.createFile(path2); assertTrue(pathFilter.accept(new Path(path2.toUri()))); + assertEquals(2, pathFilter.nonHoodiePathCache.size(), "NonHoodiePathCache size should be 2"); } @Test @@ -93,5 +96,6 @@ public void testPartitionPathsAsNonHoodiePaths() throws Exception { Path partitionPath2 = testTable.getPartitionPath(p2).getParent(); assertTrue(pathFilter.accept(partitionPath1), "Directories should be accepted"); assertTrue(pathFilter.accept(partitionPath2), "Directories should be accepted"); + assertEquals(2, pathFilter.nonHoodiePathCache.size(), "NonHoodiePathCache size should be 2"); } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java index ac857868c041..9d3855c47d66 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.Path; @@ -56,6 +57,7 @@ public class TestHoodieRealtimeFileSplit { private HoodieRealtimeFileSplit split; private String basePath; + private List deltaLogFiles; private List deltaLogPaths; private String fileSplitName; private FileSplit baseFileSplit; @@ -64,12 +66,13 @@ public class TestHoodieRealtimeFileSplit { @BeforeEach public void setUp(@TempDir java.nio.file.Path tempDir) throws Exception { basePath = tempDir.toAbsolutePath().toString(); + deltaLogFiles = Collections.singletonList(new HoodieLogFile(new Path(basePath + "/1.log"), 0L)); deltaLogPaths = Collections.singletonList(basePath + "/1.log"); fileSplitName = basePath + "/test.file"; baseFileSplit = new FileSplit(new Path(fileSplitName), 0, 100, new String[] {}); maxCommitTime = "10001"; - split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogPaths, maxCommitTime, Option.empty()); + split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFiles, maxCommitTime, Option.empty()); } @Test diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 8375dd3afca6..ede76dc3490f 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -82,6 +82,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -221,7 +222,7 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf), basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) - .map(h -> h.getPath().toString()).collect(Collectors.toList()), + .collect(Collectors.toList()), instantTime, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader @@ -235,6 +236,7 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, jobConf.setBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), isCompressionEnabled); // validate record reader compaction + long logTmpFileStartTime = System.currentTimeMillis(); HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); // use reader to read base Parquet File and log file, merge in flight and return latest commit @@ -255,6 +257,8 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, assertEquals(1.0, recordReader.getProgress(), 0.05); assertEquals(120, recordCnt); recordReader.close(); + // the temp file produced by logScanner should be deleted + assertTrue(!getLogTempFile(logTmpFileStartTime, System.currentTimeMillis(), diskMapType.toString()).exists()); } catch (Exception ioe) { throw new HoodieException(ioe.getMessage(), ioe); } @@ -264,6 +268,13 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, } + private File getLogTempFile(long startTime, long endTime, String diskType) { + return Arrays.stream(new File("/tmp").listFiles()) + .filter(f -> f.isDirectory() && f.getName().startsWith("hudi-" + diskType) && f.lastModified() > startTime && f.lastModified() < endTime) + .findFirst() + .orElse(new File("")); + } + @Test public void testUnMergedReader() throws Exception { // initial commit @@ -290,10 +301,9 @@ public void testUnMergedReader() throws Exception { FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime); // create a split with baseFile (parquet file written earlier) and new log file(s) - String logFilePath = writer.getLogFile().getPath().toString(); HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty()); + basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -370,10 +380,9 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp InputFormatTestUtil.deltaCommit(basePath, newCommitTime); // create a split with baseFile (parquet file written earlier) and new log file(s) - String logFilePath = writer.getLogFile().getPath().toString(); HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), Collections.singletonList(logFilePath), newCommitTime, Option.empty()); + basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -475,6 +484,7 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp assertEquals("stringArray" + i + recordCommitTimeSuffix, arrayValues[i].toString(), "test value for field: stringArray"); } + reader.close(); } } @@ -483,7 +493,7 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled) throws Exception { // initial commit - List logFilePaths = new ArrayList<>(); + List logFiles = new ArrayList<>(); Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ); String instantTime = "100"; @@ -504,7 +514,7 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime, numberOfLogRecords, 0, 1); long size = writer.getCurrentSize(); - logFilePaths.add(writer.getLogFile().getPath().toString()); + logFiles.add(writer.getLogFile()); writer.close(); assertTrue(size > 0, "block - size should be > 0"); @@ -512,14 +522,14 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa newCommitTime = "102"; writer = InputFormatTestUtil.writeRollbackBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime, "101", 1); - logFilePaths.add(writer.getLogFile().getPath().toString()); + logFiles.add(writer.getLogFile()); writer.close(); InputFormatTestUtil.deltaCommit(basePath, newCommitTime); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), logFilePaths, newCommitTime, Option.empty()); + basePath.toUri().toString(), logFiles, newCommitTime, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -554,6 +564,7 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa while (recordReader.next(key, value)) { // keep reading } + reader.close(); } private static Stream testArguments() { @@ -695,6 +706,7 @@ private HoodieWriteStat createHoodieWriteStat(java.nio.file.Path basePath, Strin writeStat.setNumUpdateWrites(100); writeStat.setNumWrites(100); writeStat.setPath(filePath); + writeStat.setFileSizeInBytes(new File(new Path(basePath.toString(), filePath).toString()).length()); writeStat.setPartitionPath(partitionPath); writeStat.setTotalLogFilesCompacted(100L); HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats(); @@ -750,14 +762,14 @@ public void testLogOnlyReader() throws Exception { assertTrue(size > 0, "block - size should be > 0"); FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); // create a split with new log file(s) - fileSlice.addLogFile(writer.getLogFile()); - RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(0, false, 1, 1, 0, writer.getLogFile().getPath())); + fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size)); + RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath())); realtimeFileStatus.setMaxCommitTime(instantTime); realtimeFileStatus.setBasePath(basePath.toString()); - realtimeFileStatus.setDeltaLogPaths(fileSlice.getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList())); + realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); PathWithLogFilePath pathWithLogFileStatus = (PathWithLogFilePath) realtimeFileStatus.getPath(); BaseFileWithLogsSplit bs = pathWithLogFileStatus.buildSplit(pathWithLogFileStatus, 0, 0, new String[] {""}); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogPaths(), bs.getMaxCommitTime(), Option.empty()); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), Option.empty()); JobConf newJobConf = new JobConf(baseJobConf); List fields = schema.getFields(); diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index c697d6cc0102..7ca976f9f80a 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../pom.xml hudi-integ-test diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index d8ed649d9b08..fe81f0c075c9 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -103,7 +103,7 @@ public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throw cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString(); this.sparkSession = SparkSession.builder().config(jsc.getConf()).enableHiveSupport().getOrCreate(); this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration()); - this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + this.props = UtilHelpers.readConfig(fs.getConf(), new Path(cfg.propsFilePath), cfg.configs).getProps(); log.info("Creating workload generator with configs : {}", props.toString()); this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration()); this.keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java index c94b5deb9996..a7c3245f6992 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java @@ -84,29 +84,25 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E String metadataPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/.hoodie"; FileStatus[] metaFileStatuses = fs.listStatus(new Path(metadataPath)); - boolean archFound = false; boolean cleanFound = false; for (FileStatus fileStatus : metaFileStatuses) { - Matcher archFileMatcher = ARCHIVE_FILE_PATTERN.matcher(fileStatus.getPath().getName()); - if (archFileMatcher.matches()) { - archFound = true; - if (config.validateArchival() && !config.validateClean()) { - break; - } - } Matcher cleanFileMatcher = CLEAN_FILE_PATTERN.matcher(fileStatus.getPath().getName()); if (cleanFileMatcher.matches()) { cleanFound = true; - if (!config.validateArchival() && config.validateClean()) { - break; - } + break; } - if (config.validateClean() && config.validateArchival()) { - if (archFound && cleanFound) { - break; - } + } + + String archivalPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/.hoodie/archived"; + metaFileStatuses = fs.listStatus(new Path(archivalPath)); + boolean archFound = false; + for (FileStatus fileStatus : metaFileStatuses) { + Matcher archFileMatcher = ARCHIVE_FILE_PATTERN.matcher(fileStatus.getPath().getName()); + if (archFileMatcher.matches()) { + archFound = true; } } + if (config.validateArchival() && !archFound) { throw new AssertionError("Archival NotFound in " + metadataPath); } diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/utils/SparkSqlUtils.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/utils/SparkSqlUtils.scala index fa16eae06b17..ca7ca6f26a4e 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/utils/SparkSqlUtils.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/utils/SparkSqlUtils.scala @@ -21,18 +21,19 @@ package org.apache.hudi.integ.testsuite.utils import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord -import org.apache.hudi.HoodieSparkUtils + +import org.apache.hudi.{AvroConversionUtils, HoodieSparkUtils} import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.util.Option import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config import org.apache.hudi.integ.testsuite.generator.GenericRecordFullPayloadGenerator -import org.apache.hudi.integ.testsuite.utils.SparkSqlUtils.getFieldNamesAndTypes import org.apache.hudi.utilities.schema.RowBasedSchemaProvider + import org.apache.spark.api.java.JavaRDD -import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel + import org.slf4j.Logger import scala.math.BigDecimal.RoundingMode.RoundingMode @@ -139,7 +140,7 @@ object SparkSqlUtils { */ def getFieldNamesAndTypes(avroSchemaString: String): Array[(String, String)] = { val schema = new Schema.Parser().parse(avroSchemaString) - val structType = SchemaConverters.toSqlType(schema).dataType.asInstanceOf[StructType] + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) structType.fields.map(field => (field.name, field.dataType.simpleString)) } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 97ec316a19eb..ccdf4679b7fd 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -18,9 +18,9 @@ package org.apache.hudi.integ; -import java.util.concurrent.TimeoutException; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import com.github.dockerjava.api.DockerClient; import com.github.dockerjava.api.command.DockerCmdExecFactory; @@ -42,6 +42,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import static java.util.concurrent.TimeUnit.SECONDS; @@ -130,10 +131,14 @@ public void init() { DockerClientConfig config = DefaultDockerClientConfig.createDefaultConfigBuilder().withDockerHost(dockerHost).build(); // using jaxrs/jersey implementation here (netty impl is also available) - DockerCmdExecFactory dockerCmdExecFactory = new JerseyDockerCmdExecFactory().withConnectTimeout(1000) - .withMaxTotalConnections(100).withMaxPerRouteConnections(10); + DockerCmdExecFactory dockerCmdExecFactory = new JerseyDockerCmdExecFactory().withConnectTimeout(10000) + .withMaxTotalConnections(100).withMaxPerRouteConnections(50); dockerClient = DockerClientBuilder.getInstance(config).withDockerCmdExecFactory(dockerCmdExecFactory).build(); - await().atMost(60, SECONDS).until(this::servicesUp); + LOG.info("Start waiting for all the containers and services to be ready"); + long currTs = System.currentTimeMillis(); + await().atMost(300, SECONDS).until(this::servicesUp); + LOG.info(String.format("Waiting for all the containers and services finishes in %d ms", + System.currentTimeMillis() - currTs)); } private boolean servicesUp() { @@ -144,8 +149,29 @@ private boolean servicesUp() { return false; } } - runningContainers = containerList.stream().map(c -> Pair.of(c.getNames()[0], c)) - .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + + if (runningContainers == null) { + runningContainers = containerList.stream().map(c -> Pair.of(c.getNames()[0], c)) + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + } + + return checkHealth(ADHOC_1_CONTAINER, "namenode", 8020); + } + + private boolean checkHealth(String fromContainerName, String hostname, int port) { + try { + String command = String.format("nc -z -v %s %d", hostname, port); + TestExecStartResultCallback resultCallback = + executeCommandStringInDocker(fromContainerName, command, false, true); + String stderrString = resultCallback.getStderr().toString().trim(); + if (!stderrString.contains("open")) { + Thread.sleep(1000); + return false; + } + } catch (Exception e) { + throw new HoodieException(String.format("Exception thrown while checking health from %s for %s:%d", + fromContainerName, hostname, port), e); + } return true; } @@ -153,8 +179,13 @@ private String singleSpace(String str) { return str.replaceAll("[\\s]+", " "); } - private TestExecStartResultCallback executeCommandInDocker(String containerName, String[] command, - boolean expectedToSucceed) throws Exception { + private TestExecStartResultCallback executeCommandInDocker( + String containerName, String[] command, boolean expectedToSucceed) throws Exception { + return executeCommandInDocker(containerName, command, true, expectedToSucceed); + } + + private TestExecStartResultCallback executeCommandInDocker( + String containerName, String[] command, boolean checkIfSucceed, boolean expectedToSucceed) throws Exception { Container sparkWorkerContainer = runningContainers.get(containerName); ExecCreateCmd cmd = dockerClient.execCreateCmd(sparkWorkerContainer.getId()).withCmd(command).withAttachStdout(true) .withAttachStderr(true); @@ -171,11 +202,11 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, if (!completed) { callback.getStderr().flush(); callback.getStdout().flush(); - LOG.error("\n\n ###### Timed Out Command : " + Arrays.asList(command)); + LOG.error("\n\n ###### Timed Out Command : " + Arrays.asList(command)); LOG.error("\n\n ###### Stderr of timed-out command #######\n" + callback.getStderr().toString()); LOG.error("\n\n ###### stdout of timed-out command #######\n" + callback.getStdout().toString()); - throw new TimeoutException("Command " + command + " has been running for more than 9 minutes. " - + "Killing and failing !!"); + throw new TimeoutException("Command " + command + " has been running for more than 9 minutes. " + + "Killing and failing !!"); } int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); LOG.info("Exit code for command : " + exitCode); @@ -184,10 +215,12 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, } LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); - if (expectedToSucceed) { - assertEquals(0, exitCode, "Command (" + Arrays.toString(command) + ") expected to succeed. Exit (" + exitCode + ")"); - } else { - assertNotEquals(0, exitCode, "Command (" + Arrays.toString(command) + ") expected to fail. Exit (" + exitCode + ")"); + if (checkIfSucceed) { + if (expectedToSucceed) { + assertEquals(0, exitCode, "Command (" + Arrays.toString(command) + ") expected to succeed. Exit (" + exitCode + ")"); + } else { + assertNotEquals(0, exitCode, "Command (" + Arrays.toString(command) + ") expected to fail. Exit (" + exitCode + ")"); + } } cmd.close(); return callback; @@ -199,14 +232,20 @@ void executeCommandStringsInDocker(String containerName, List commands) } } - protected TestExecStartResultCallback executeCommandStringInDocker(String containerName, String cmd, boolean expectedToSucceed) + protected TestExecStartResultCallback executeCommandStringInDocker( + String containerName, String cmd, boolean expectedToSucceed) throws Exception { + return executeCommandStringInDocker(containerName, cmd, true, expectedToSucceed); + } + + protected TestExecStartResultCallback executeCommandStringInDocker( + String containerName, String cmd, boolean checkIfSucceed, boolean expectedToSucceed) throws Exception { LOG.info("\n\n#################################################################################################"); LOG.info("Container : " + containerName + ", Running command :" + cmd); LOG.info("\n#################################################################################################"); String[] cmdSplits = singleSpace(cmd).split(" "); - return executeCommandInDocker(containerName, cmdSplits, expectedToSucceed); + return executeCommandInDocker(containerName, cmdSplits, checkIfSucceed, expectedToSucceed); } protected Pair executeHiveCommand(String hiveCommand) throws Exception { diff --git a/hudi-kafka-connect/README.md b/hudi-kafka-connect/README.md index e2abab1ccbab..9d63f5a546e5 100644 --- a/hudi-kafka-connect/README.md +++ b/hudi-kafka-connect/README.md @@ -27,55 +27,55 @@ The first thing you need to do to start using this connector is building it. In - [Java 1.8+](https://openjdk.java.net/) - [Apache Maven](https://maven.apache.org/) - Install [kcat](https://github.com/edenhill/kcat) += Install jq. `brew install jq` -After installing these dependencies, execute the following commands. This will install all the Hudi dependency jars, -including the fat packaged jar that contains all the dependencies required for a functional Hudi Kafka Connect Sink. + +## Trying the connector -```bash -cd $HUDI_DIR -mvn clean -DskipTests install -``` +After installing these dependencies, follow steps based on your requirement. -Henceforth, incremental builds can be performed as follows. +### 1 - Starting the environment -```bash -mvn clean -pl hudi-kafka-connect install -DskipTests -mvn clean -pl packaging/hudi-kafka-connect-bundle install -``` +For runtime dependencies, we encourage using the confluent HDFS connector jars. We have tested our setup with version `10.1.0`. +Either use confluent-hub to install the connector or download it from [here](https://tinyurl.com/yb472f79). -Next, we need to make sure that the hudi sink connector bundle jar is in Kafka Connect classpath. Note that the connect -classpath should be same as the one configured in the connector configuration file. +Copy the entire folder to the classpath that will be used by the Hudi Kafka Connector. ```bash -cp $HUDI_DIR/packaging/hudi-kafka-connect-bundle/target/hudi-kafka-connect-bundle-0.10.0-SNAPSHOT.jar /usr/local/share/java/hudi-kafka-connect/ +confluent-hub install confluentinc/kafka-connect-hdfs:10.1.0 +cp confluentinc-kafka-connect-hdfs-10.1.0/* /usr/local/share/kafka/plugins/ ``` -## Trying the connector - -After building the package, we need to install the Apache Kafka - -### 1 - Starting the environment +Now, build the packaged jar that contains all the hudi classes, including the Hudi Kafka Connector. And copy it +to the plugin path that contains all the other jars (`/usr/local/share/kafka/plugins/lib`) +```bash +cd ${HUDI_DIR} +mvn package -DskipTests -pl packaging/hudi-kafka-connect-bundle -am +cp $HUDI_DIR/packaging/hudi-kafka-connect-bundle/target/hudi-kafka-connect-bundle-0.10.0-SNAPSHOT.jar /usr/local/share/kafka/plugins/lib +``` -To try out the Connect Sink locally, set up a Kafka broker locally. Download the latest apache kafka from https://kafka.apache.org/downloads. +Set up a Kafka broker locally. Download the latest apache kafka from [here](https://kafka.apache.org/downloads). Once downloaded and built, run the Zookeeper server and Kafka server using the command line tools. - ```bash export KAFKA_HOME=/path/to/kafka_install_dir -cd $KAFKA_KAFKA_HOME +cd $KAFKA_HOME ./bin/zookeeper-server-start.sh ./config/zookeeper.properties ./bin/kafka-server-start.sh ./config/server.properties ``` - Wait until the kafka cluster is up and running. ### 2 - Set up the schema registry Hudi leverages schema registry to obtain the latest schema when writing records. While it supports most popular schema -registries, we use Confluent schema registry. Download the latest confluent platform and run the schema registry -service. +registries, we use Confluent schema registry. Download the +latest [confluent platform](https://docs.confluent.io/platform/current/installation/index.html) and run the schema +registry service. + +NOTE: You must change the port from `8081` (default) to `8082` to avoid conflict. ```bash cd $CONFLUENT_DIR +/bin/kafka-configs --zookeeper localhost --entity-type topics --entity-name _schemas --alter --add-config cleanup.policy=compact ./bin/schema-registry-start etc/schema-registry/schema-registry.properties ``` @@ -98,7 +98,14 @@ cd $HUDI_DIR/hudi-kafka-connect/demo/ bash setupKafka.sh -n ``` -### 4 - Run the Sink connector worker (multiple workers can be run) +To generate data for long-running tests, you can add `-b` option to specify the number of batches of data +to generate, with each batch containing a number of messages and idle time between batches, as follows: + +```bash +bash setupKafka.sh -n -b +``` + +### 5 - Run the Sink connector worker (multiple workers can be run) The Kafka connect is a distributed platform, with the ability to run one or more workers (each running multiple tasks) that parallely process the records from the Kafka partitions for the same topic. We provide a properties file with @@ -108,11 +115,11 @@ Note that if multiple workers need to be run, the webserver needs to be reconfig successful running of the workers. ```bash -cd $KAFKA_HOME -./bin/connect-distributed.sh $HUDI_DIR/hudi-kafka-connect/demo/connect-distributed.properties +cd ${KAFKA_HOME} +./bin/connect-distributed.sh ${HUDI_DIR}/hudi-kafka-connect/demo/connect-distributed.properties ``` -### 5- To add the Hudi Sink to the Connector (delete it if you want to re-configure) +### 6 - To add the Hudi Sink to the Connector (delete it if you want to re-configure) Once the Connector has started, it will not run the Sink, until the Hudi sink is added using the web api. The following curl APIs can be used to delete and add a new Hudi Sink. Again, a default configuration is provided for the Hudi Sink, @@ -126,6 +133,7 @@ curl -X POST -H "Content-Type:application/json" -d @$HUDI_DIR/hudi-kafka-connect Now, you should see that the connector is created and tasks are running. ```bash +mkdir /tmp/hoodie/ curl -X GET -H "Content-Type:application/json" http://localhost:8083/connectors ["hudi-sink"] curl -X GET -H "Content-Type:application/json" http://localhost:8083/connectors/hudi-sink/status | jq @@ -136,8 +144,8 @@ Note: HUDI-2325 tracks Hive sync, which will unlock pretty much every other quer ```bash ls -a /tmp/hoodie/hudi-test-topic -. .hoodie partition-1 partition-3 -.. partition-0 partition-2 partition-4 +. .hoodie partition_1 partition_3 +.. partition_0 partition_2 partition_4 ls -lt /tmp/hoodie/hudi-test-topic/.hoodie total 72 @@ -152,7 +160,7 @@ total 72 -rw-r--r-- 1 user wheel 0 Sep 13 21:41 20210913214114.commit.requested drwxr-xr-x 2 user wheel 64 Sep 13 21:41 archived -ls -l /tmp/hoodie/hudi-test-topic/partition-0 +ls -l /tmp/hoodie/hudi-test-topic/partition_0 total 5168 -rw-r--r-- 1 user wheel 439332 Sep 13 21:43 2E0E6DB44ACC8479059574A2C71C7A7E-0_0-0-0_20210913214114.parquet -rw-r--r-- 1 user wheel 440179 Sep 13 21:42 3B56FAAAE2BDD04E480C1CBACD463D3E-0_0-0-0_20210913214114.parquet @@ -162,4 +170,222 @@ total 5168 -rw-r--r-- 1 user wheel 440214 Sep 13 21:43 E200FA75DCD1CED60BE86BCE6BF5D23A-0_0-0-0_20210913214114.parquet ``` +### 7 - Run async compaction and clustering if scheduled + +When using Merge-On-Read (MOR) as the table type, async compaction and clustering can be scheduled when the Sink is +running. Inline compaction and clustering are disabled by default due to performance reason. By default, async +compaction scheduling is enabled, and you can disable it by setting `hoodie.kafka.compaction.async.enable` to `false`. +Async clustering scheduling is disabled by default, and you can enable it by setting `hoodie.clustering.async.enabled` +to `true`. + +The Sink only schedules the compaction and clustering if necessary and does not execute them for performance. You need +to execute the scheduled compaction and clustering using separate Spark jobs or Hudi CLI. + +After the compaction is scheduled, you can see the requested compaction instant (`20211111111410.compaction.requested`) +below: + +``` +ls -l /tmp/hoodie/hudi-test-topic/.hoodie +total 280 +-rw-r--r-- 1 user wheel 21172 Nov 11 11:09 20211111110807.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:08 20211111110807.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:08 20211111110807.deltacommit.requested +-rw-r--r-- 1 user wheel 22458 Nov 11 11:11 20211111110940.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:09 20211111110940.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:09 20211111110940.deltacommit.requested +-rw-r--r-- 1 user wheel 21445 Nov 11 11:13 20211111111110.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:11 20211111111110.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:11 20211111111110.deltacommit.requested +-rw-r--r-- 1 user wheel 24943 Nov 11 11:14 20211111111303.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:13 20211111111303.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:13 20211111111303.deltacommit.requested +-rw-r--r-- 1 user wheel 9885 Nov 11 11:14 20211111111410.compaction.requested +-rw-r--r-- 1 user wheel 21192 Nov 11 11:15 20211111111411.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:14 20211111111411.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:14 20211111111411.deltacommit.requested +-rw-r--r-- 1 user wheel 0 Nov 11 11:15 20211111111530.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:15 20211111111530.deltacommit.requested +drwxr-xr-x 2 user wheel 64 Nov 11 11:08 archived +-rw-r--r-- 1 user wheel 387 Nov 11 11:08 hoodie.properties +``` + +Then you can run async compaction job with `HoodieCompactor` and `spark-submit` by: + +``` +spark-submit \ + --class org.apache.hudi.utilities.HoodieCompactor \ + hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.10.0-SNAPSHOT.jar \ + --base-path /tmp/hoodie/hudi-test-topic \ + --table-name hudi-test-topic \ + --schema-file /Users/user/repo/hudi/docker/demo/config/schema.avsc \ + --instant-time 20211111111410 \ + --parallelism 2 \ + --spark-memory 1g +``` + +Note that you don't have to provide the instant time through `--instant-time`. In that case, the earliest scheduled +compaction is going to be executed. + +Alternatively, you can use Hudi CLI to execute compaction: + +``` +hudi-> connect --path /tmp/hoodie/hudi-test-topic +hudi:hudi-test-topic-> compactions show all +╔═════════════════════════╤═══════════╤═══════════════════════════════╗ +║ Compaction Instant Time │ State │ Total FileIds to be Compacted ║ +╠═════════════════════════╪═══════════╪═══════════════════════════════╣ +║ 20211111111410 │ REQUESTED │ 12 ║ +╚═════════════════════════╧═══════════╧═══════════════════════════════╝ + +compaction validate --instant 20211111111410 +compaction run --compactionInstant 20211111111410 --parallelism 2 --schemaFilePath /Users/user/repo/hudi/docker/demo/config/schema.avsc +``` + +Similarly, you can see the requested clustering instant (`20211111111813.replacecommit.requested`) after it is scheduled +by the Sink: + +``` +ls -l /tmp/hoodie/hudi-test-topic/.hoodie +total 736 +-rw-r--r-- 1 user wheel 24943 Nov 11 11:14 20211111111303.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:13 20211111111303.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:13 20211111111303.deltacommit.requested +-rw-r--r-- 1 user wheel 18681 Nov 11 11:17 20211111111410.commit +-rw-r--r-- 1 user wheel 0 Nov 11 11:17 20211111111410.compaction.inflight +-rw-r--r-- 1 user wheel 9885 Nov 11 11:14 20211111111410.compaction.requested +-rw-r--r-- 1 user wheel 21192 Nov 11 11:15 20211111111411.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:14 20211111111411.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:14 20211111111411.deltacommit.requested +-rw-r--r-- 1 user wheel 22460 Nov 11 11:17 20211111111530.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:15 20211111111530.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:15 20211111111530.deltacommit.requested +-rw-r--r-- 1 user wheel 21357 Nov 11 11:18 20211111111711.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:17 20211111111711.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:17 20211111111711.deltacommit.requested +-rw-r--r-- 1 user wheel 6516 Nov 11 11:18 20211111111813.replacecommit.requested +-rw-r--r-- 1 user wheel 26070 Nov 11 11:20 20211111111815.deltacommit +-rw-r--r-- 1 user wheel 0 Nov 11 11:18 20211111111815.deltacommit.inflight +-rw-r--r-- 1 user wheel 0 Nov 11 11:18 20211111111815.deltacommit.requested +``` + +Then you can run async clustering job with `HoodieClusteringJob` and `spark-submit` by: + +``` +spark-submit \ + --class org.apache.hudi.utilities.HoodieClusteringJob \ + hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.10.0-SNAPSHOT.jar \ + --props clusteringjob.properties \ + --mode execute \ + --base-path /tmp/hoodie/hudi-test-topic \ + --table-name sample_table \ + --instant-time 20211111111813 \ + --spark-memory 1g +``` + +Sample `clusteringjob.properties`: + +``` +hoodie.datasource.write.recordkey.field=volume +hoodie.datasource.write.partitionpath.field=date +hoodie.deltastreamer.schemaprovider.registry.url=http://localhost:8081/subjects/hudi-test-topic/versions/latest + +hoodie.clustering.plan.strategy.target.file.max.bytes=1073741824 +hoodie.clustering.plan.strategy.small.file.limit=629145600 +hoodie.clustering.execution.strategy.class=org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy +hoodie.clustering.plan.strategy.sort.columns=volume + +hoodie.write.concurrency.mode=single_writer +``` + +Note that you don't have to provide the instant time through `--instant-time`. In that case, the earliest scheduled +clustering is going to be executed. + +### 8- Querying via Hive + +In this section we explain how one can test syncing of the Hudi table with Hive server/ Hive Metastore, +that enable querying via Hive, Presto etc. + +To ease the deployment of HDFS, Hive Server, Hive Metastore etc. for testing hive sync, we use the docker +containers from the hudi docker demo. Refer to [this link for the setup](https://hudi.apache.org/docs/docker_demo). +Additionally, the docker deploys kafka and zookeeper too, so you do not need to run them explicitly in this setup. + +Essentially, follow the steps listed here: + +/etc/hosts : The demo references many services running in container by the hostname. Add the following settings to /etc/hosts +```bash +127.0.0.1 adhoc-1 +127.0.0.1 adhoc-2 +127.0.0.1 namenode +127.0.0.1 datanode1 +127.0.0.1 hiveserver +127.0.0.1 hivemetastore +127.0.0.1 kafkabroker +127.0.0.1 sparkmaster +127.0.0.1 zookeeper +``` + +Bring up the docker containers +```bash +cd ${HUDI_DIR}/docker +./setup_demo.sh +``` + +Firstly, (re)-install a different connector that is configured to write the Hudi table to Hdfs instead of local filesystem. + +```bash +curl -X DELETE http://localhost:8083/connectors/hudi-sink +curl -X POST -H "Content-Type:application/json" -d @$HUDI_DIR/hudi-kafka-connect/demo/config-sink-hive.json http://localhost:8083/connectors +``` + +After running the connector, you can query the hive server using the following steps: + +```bash +docker exec -it adhoc-2 /bin/bash +beeline -u jdbc:hive2://hiveserver:10000 \ + --hiveconf hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat \ + --hiveconf hive.stats.autogather=false + + +# List Tables +0: jdbc:hive2://hiveserver:10000> show tables; ++---------------------+--+ +| tab_name | ++---------------------+--+ +| huditesttopic_ro | +| huditesttopic_rt | ++---------------------+--+ +3 rows selected (1.199 seconds) +0: jdbc:hive2://hiveserver:10000> + + +# Look at partitions that were added +0: jdbc:hive2://hiveserver:10000> show partitions huditesttopic_ro; ++-------------------+--+ +| partition | ++-------------------+--+ +| date=partition_0 | +| date=partition_1 | +| date=partition_2 | +| date=partition_3 | +| date=partition_4 | ++-------------------+--+ +1 row selected (0.24 seconds) + + +0: jdbc:hive2://hiveserver:10000> select `_hoodie_commit_time`, symbol, ts, volume, open, close from huditesttopic_ro; ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| _hoodie_commit_time | symbol | ts | volume | open | close | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +| 20180924222155 | GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 | +| 20180924222155 | GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 | ++----------------------+---------+----------------------+---------+------------+-----------+--+ +``` + +`Current Limitation:` The Hudi Kafka Connect sink uses `Merge-On-Read` by default, and inserts/ appends the kafka records +directly to the log file(s). Asynchronously, compaction service can be executed to merge the log files into base file (Parquet format). +Generally, we support both Read-Optimized that reads only parquet base files and Snapshot queries that read and merge +records across base and log files. However, currently there is a limitation where we are not able to read records from +only log files. Hence, the queries for Hudi Kafka Connect will only work after compaction merges the records into base files. Alternatively, +users have the option to reconfigure the table type to `COPY_ON_WRITE` in config-sink.json. + diff --git a/hudi-kafka-connect/demo/config-sink-hive.json b/hudi-kafka-connect/demo/config-sink-hive.json new file mode 100644 index 000000000000..214fd1891906 --- /dev/null +++ b/hudi-kafka-connect/demo/config-sink-hive.json @@ -0,0 +1,30 @@ +{ + "name": "hudi-sink", + "config": { + "bootstrap.servers": "kafkabroker:9092", + "connector.class": "org.apache.hudi.connect.HoodieSinkConnector", + "tasks.max": "4", + "key.converter": "org.apache.kafka.connect.storage.StringConverter", + "value.converter": "org.apache.kafka.connect.storage.StringConverter", + "value.converter.schemas.enable": "false", + "topics": "hudi-test-topic", + "hoodie.table.name": "hudi-test-topic", + "hoodie.table.type": "MERGE_ON_READ", + "hoodie.base.path": "hdfs://namenode:8020/user/hive/warehouse/hudi-test-topic", + "hoodie.datasource.write.recordkey.field": "volume", + "hoodie.datasource.write.partitionpath.field": "date", + "hoodie.schemaprovider.class": "org.apache.hudi.schema.SchemaRegistryProvider", + "hoodie.deltastreamer.schemaprovider.registry.url": "http://localhost:8082/subjects/hudi-test-topic/versions/latest", + "hoodie.kafka.commit.interval.secs": 60, + "hoodie.meta.sync.enable": "true", + "hoodie.meta.sync.classes": "org.apache.hudi.hive.HiveSyncTool", + "hoodie.datasource.hive_sync.table": "huditesttopic", + "hoodie.datasource.hive_sync.partition_fields": "date", + "hoodie.datasource.hive_sync.partition_extractor_class": "org.apache.hudi.hive.MultiPartKeysValueExtractor", + "hoodie.datasource.hive_sync.use_jdbc": "false", + "hoodie.datasource.hive_sync.mode": "hms", + "dfs.client.use.datanode.hostname": "true", + "hive.metastore.uris": "thrift://hivemetastore:9083", + "hive.metastore.client.socket.timeout": "1500s" + } +} diff --git a/hudi-kafka-connect/demo/config-sink.json b/hudi-kafka-connect/demo/config-sink.json index 2d2be00f8935..105c4d491e77 100644 --- a/hudi-kafka-connect/demo/config-sink.json +++ b/hudi-kafka-connect/demo/config-sink.json @@ -1,7 +1,7 @@ { "name": "hudi-sink", "config": { - "bootstrap.servers": "localhost:9092", + "bootstrap.servers": "kafkabroker:9092", "connector.class": "org.apache.hudi.connect.HoodieSinkConnector", "tasks.max": "4", "key.converter": "org.apache.kafka.connect.storage.StringConverter", @@ -14,6 +14,7 @@ "hoodie.datasource.write.recordkey.field": "volume", "hoodie.datasource.write.partitionpath.field": "date", "hoodie.schemaprovider.class": "org.apache.hudi.schema.SchemaRegistryProvider", - "hoodie.deltastreamer.schemaprovider.registry.url": "http://localhost:8081/subjects/hudi-test-topic/versions/latest" - } + "hoodie.deltastreamer.schemaprovider.registry.url": "http://localhost:8082/subjects/hudi-test-topic/versions/latest", + "hoodie.kafka.commit.interval.secs": 60 + } } diff --git a/hudi-kafka-connect/demo/connect-distributed.properties b/hudi-kafka-connect/demo/connect-distributed.properties index 9e3cec149238..1c28bc60d52b 100644 --- a/hudi-kafka-connect/demo/connect-distributed.properties +++ b/hudi-kafka-connect/demo/connect-distributed.properties @@ -15,7 +15,7 @@ # limitations under the License. ## -bootstrap.servers=localhost:9092 +bootstrap.servers=kafkabroker:9092 group.id=hudi-connect-cluster key.converter=org.apache.kafka.connect.json.JsonConverter value.converter=org.apache.kafka.connect.json.JsonConverter @@ -30,4 +30,4 @@ status.storage.replication.factor=1 offset.flush.interval.ms=60000 listeners=HTTP://:8083 -plugin.path=/usr/local/share/java +plugin.path=/usr/local/share/kafka/plugins diff --git a/hudi-kafka-connect/demo/setupKafka.sh b/hudi-kafka-connect/demo/setupKafka.sh index 20edb1ceb2ea..49155649cea3 100755 --- a/hudi-kafka-connect/demo/setupKafka.sh +++ b/hudi-kafka-connect/demo/setupKafka.sh @@ -21,11 +21,14 @@ ######################### usage() { echo "Usage: $0" - echo " -n |--num-kafka-records, (required) number of kafka records to generate" + echo " -n |--num-kafka-records, (required) number of kafka records to generate in a batch" + echo " -b |--num-batch, (optional) number of batches of records to generate (default is 1)" + echo " -t |--reuse-topic, (optional) reuses the Kafka topic (default deletes and recreate the topic)" echo " -f |--raw-file, (optional) raw file for the kafka records" echo " -k |--kafka-topic, (optional) Topic name for Kafka" echo " -m |--num-kafka-partitions, (optional) number of kafka partitions" echo " -r |--record-key, (optional) field to use as record key" + echo " -o |--record-key-offset, (optional) record key offset to start with (default is 0)" echo " -l |--num-hudi-partitions, (optional) number of hudi partitions" echo " -p |--partition-key, (optional) field to use as partition" echo " -s |--schema-file, (optional) path of the file containing the schema of the records" @@ -47,18 +50,30 @@ fi ## defaults rawDataFile=${HUDI_DIR}/docker/demo/data/batch_1.json +kafkaBrokerHostname=localhost kafkaTopicName=hudi-test-topic numKafkaPartitions=4 recordKey=volume numHudiPartitions=5 partitionField=date schemaFile=${HUDI_DIR}/docker/demo/config/schema.avsc +numBatch=1 +recordValue=0 +recreateTopic="Y" -while getopts ":n:f:k:m:r:l:p:s:-:" opt; do +while getopts ":n:b:tf:k:m:r:o:l:p:s:-:" opt; do case $opt in n) - num_records="$OPTARG" - printf "Argument num-kafka-records is %s\n" "$num_records" + numRecords="$OPTARG" + printf "Argument num-kafka-records is %s\n" "$numRecords" + ;; + b) + numBatch="$OPTARG" + printf "Argument num-batch is %s\n" "$numBatch" + ;; + t) + recreateTopic="N" + printf "Argument recreate-topic is N (reuse Kafka topic) \n" ;; k) rawDataFile="$OPTARG" @@ -76,6 +91,10 @@ while getopts ":n:f:k:m:r:l:p:s:-:" opt; do recordKey="$OPTARG" printf "Argument record-key is %s\n" "$recordKey" ;; + o) + recordValue="$OPTARG" + printf "Argument record-key-offset is %s\n" "$recordValue" + ;; l) numHudiPartitions="$OPTARG" printf "Argument num-hudi-partitions is %s\n" "$numHudiPartitions" @@ -84,7 +103,7 @@ while getopts ":n:f:k:m:r:l:p:s:-:" opt; do partitionField="$OPTARG" printf "Argument partition-key is %s\n" "$partitionField" ;; - p) + s) schemaFile="$OPTARG" printf "Argument schema-file is %s\n" "$schemaFile" ;; @@ -94,53 +113,63 @@ while getopts ":n:f:k:m:r:l:p:s:-:" opt; do esac done -# First delete the existing topic -${KAFKA_HOME}/bin/kafka-topics.sh --delete --topic ${kafkaTopicName} --bootstrap-server localhost:9092 +if [ $recreateTopic = "Y" ]; then + # First delete the existing topic + echo "Delete Kafka topic $kafkaTopicName ..." + ${KAFKA_HOME}/bin/kafka-topics.sh --delete --topic ${kafkaTopicName} --bootstrap-server ${kafkaBrokerHostname}:9092 -# Create the topic with 4 partitions -${KAFKA_HOME}/bin/kafka-topics.sh --create --topic ${kafkaTopicName} --partitions $numKafkaPartitions --replication-factor 1 --bootstrap-server localhost:9092 + # Create the topic with 4 partitions + echo "Create Kafka topic $kafkaTopicName ..." + ${KAFKA_HOME}/bin/kafka-topics.sh --create --topic ${kafkaTopicName} --partitions $numKafkaPartitions --replication-factor 1 --bootstrap-server ${kafkaBrokerHostname}:9092 +fi # Setup the schema registry export SCHEMA=$(sed 's|/\*|\n&|g;s|*/|&\n|g' ${schemaFile} | sed '/\/\*/,/*\//d' | jq tostring) -curl -X POST -H "Content-Type: application/vnd.schemaregistry.v1+json" --data "{\"schema\": $SCHEMA}" http://localhost:8081/subjects/${kafkaTopicName}/versions +curl -X POST -H "Content-Type: application/vnd.schemaregistry.v1+json" --data "{\"schema\": $SCHEMA}" http://localhost:8082/subjects/${kafkaTopicName}/versions curl -X GET http://localhost:8081/subjects/${kafkaTopicName}/versions/latest # Generate kafka messages from raw records # Each records with unique keys and generate equal messages across each hudi partition partitions={} for ((i = 0; i < ${numHudiPartitions}; i++)); do - partitions[$i]="partition-"$i + partitions[$i]="partition_"$i done events_file=/tmp/kcat-input.events rm -f ${events_file} -recordValue=0 -num_records=$((num_records + 0)) - -for (( ; ; )); do - while IFS= read line; do - for partitionValue in "${partitions[@]}"; do - echo $line | jq --arg recordKey $recordKey --arg recordValue $recordValue --arg partitionField $partitionField --arg partitionValue $partitionValue -c '.[$recordKey] = $recordValue | .[$partitionField] = $partitionValue' >>${events_file} - ((recordValue = recordValue + 1)) - - if [ $recordValue -gt $num_records ]; then +totalNumRecords=$((numRecords + recordValue)) + +for ((i = 1;i<=numBatch;i++)); do + rm -f ${events_file} + date + echo "Start batch $i ..." + batchRecordSeq=0 + for (( ; ; )); do + while IFS= read line; do + for partitionValue in "${partitions[@]}"; do + echo $line | jq --arg recordKey $recordKey --arg recordValue $recordValue --arg partitionField $partitionField --arg partitionValue $partitionValue -c '.[$recordKey] = $recordValue | .[$partitionField] = $partitionValue' >>${events_file} + ((recordValue = recordValue + 1)) + ((batchRecordSeq = batchRecordSeq + 1)) + + if [ $batchRecordSeq -eq $numRecords ]; then + break + fi + done + + if [ $batchRecordSeq -eq $numRecords ]; then break fi - done - - if [ $recordValue -gt $num_records ]; then - break - fi + done <"$rawDataFile" - if [ $(($recordValue % 1000)) -eq 0 ]; then - sleep 1 - fi - done <"$rawDataFile" + if [ $batchRecordSeq -eq $numRecords ]; then + date + echo " Record key until $recordValue" + sleep 20 + break + fi + done - if [ $recordValue -gt $num_records ]; then - break - fi + echo "publish to Kafka ..." + grep -v '^$' ${events_file} | kcat -P -b ${kafkaBrokerHostname}:9092 -t ${kafkaTopicName} done - -grep -v '^$' ${events_file} | kcat -P -b localhost:9092 -t hudi-test-topic diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml index 7a79f265c971..b12468579161 100644 --- a/hudi-kafka-connect/pom.xml +++ b/hudi-kafka-connect/pom.xml @@ -19,13 +19,13 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-kafka-connect Kafka Connect Sink Connector for Hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT jar @@ -148,7 +148,7 @@ org.apache.flink flink-core - 1.12.1 + ${flink.version} com.esotericsoftware.kryo @@ -156,6 +156,11 @@ + + org.apache.flink + flink-hadoop-compatibility_${scala.binary.version} + ${flink.version} + @@ -195,6 +200,19 @@ ${hadoop.version} + + + org.apache.hive + hive-common + ${hive.version} + + + ${hive.groupid} + hive-metastore + ${hive.version} + + + org.apache.hudi diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/KafkaConnectFileIdPrefixProvider.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/KafkaConnectFileIdPrefixProvider.java index 9c4674706a6a..436366709d5b 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/KafkaConnectFileIdPrefixProvider.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/KafkaConnectFileIdPrefixProvider.java @@ -18,6 +18,7 @@ package org.apache.hudi.connect; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.connect.utils.KafkaConnectUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.FileIdPrefixProvider; @@ -25,8 +26,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.util.Properties; - public class KafkaConnectFileIdPrefixProvider extends FileIdPrefixProvider { public static final String KAFKA_CONNECT_PARTITION_ID = "hudi.kafka.connect.partition"; @@ -34,7 +33,7 @@ public class KafkaConnectFileIdPrefixProvider extends FileIdPrefixProvider { private final String kafkaPartition; - public KafkaConnectFileIdPrefixProvider(Properties props) { + public KafkaConnectFileIdPrefixProvider(TypedProperties props) { super(props); if (!props.containsKey(KAFKA_CONNECT_PARTITION_ID)) { LOG.error("Fatal error due to Kafka Connect Partition Id is not set"); diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionCoordinator.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionCoordinator.java index 7acd875b6bee..14fd880b1c05 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionCoordinator.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionCoordinator.java @@ -280,26 +280,56 @@ private void endExistingCommit() { private void onReceiveWriteStatus(ControlMessage message) { ControlMessage.ParticipantInfo participantInfo = message.getParticipantInfo(); - int partition = message.getSenderPartition(); - partitionsWriteStatusReceived.put(partition, KafkaConnectUtils.getWriteStatuses(participantInfo)); - currentConsumedKafkaOffsets.put(partition, participantInfo.getKafkaOffset()); + int partitionId = message.getSenderPartition(); + partitionsWriteStatusReceived.put(partitionId, KafkaConnectUtils.getWriteStatuses(participantInfo)); + currentConsumedKafkaOffsets.put(partitionId, participantInfo.getKafkaOffset()); if (partitionsWriteStatusReceived.size() >= numPartitions && currentState.equals(State.ENDED_COMMIT)) { // Commit the kafka offsets to the commit file try { List allWriteStatuses = new ArrayList<>(); partitionsWriteStatusReceived.forEach((key, value) -> allWriteStatuses.addAll(value)); - // Commit the last write in Hudi, along with the latest kafka offset - if (!allWriteStatuses.isEmpty()) { - transactionServices.endCommit(currentCommitTime, + + long totalErrorRecords = (long) allWriteStatuses.stream().mapToDouble(WriteStatus::getTotalErrorRecords).sum(); + long totalRecords = (long) allWriteStatuses.stream().mapToDouble(WriteStatus::getTotalRecords).sum(); + boolean hasErrors = totalErrorRecords > 0; + + if ((!hasErrors || configs.allowCommitOnErrors()) && !allWriteStatuses.isEmpty()) { + boolean success = transactionServices.endCommit(currentCommitTime, allWriteStatuses, transformKafkaOffsets(currentConsumedKafkaOffsets)); + + if (success) { + LOG.info("Commit " + currentCommitTime + " successful!"); + currentState = State.WRITE_STATUS_RCVD; + globalCommittedKafkaOffsets.putAll(currentConsumedKafkaOffsets); + submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.ACK_COMMIT, + message.getTopicName(), + currentCommitTime)); + return; + } else { + LOG.error("Commit " + currentCommitTime + " failed!"); + } + } else if (hasErrors) { + LOG.error("Coordinator found errors when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords); + LOG.error("Printing out the top 100 errors"); + allWriteStatuses.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> { + LOG.error("Global error :", ws.getGlobalError()); + if (ws.getErrors().size() > 0) { + ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " is " + value)); + } + }); + } else { + LOG.warn("Empty write statuses were received from all Participants"); } - currentState = State.WRITE_STATUS_RCVD; - globalCommittedKafkaOffsets.putAll(currentConsumedKafkaOffsets); - submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.ACK_COMMIT, - message.getTopicName(), - currentCommitTime)); + + // Submit the next start commit, that will rollback the current commit. + currentState = State.FAILED_COMMIT; + LOG.warn("Current commit " + currentCommitTime + " failed, so starting a new commit after recovery delay"); + submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.START_COMMIT, + partition.topic(), + StringUtils.EMPTY_STRING), + RESTART_COMMIT_DELAY_MS, TimeUnit.MILLISECONDS); } catch (Exception exception) { LOG.error("Fatal error while committing file", exception); } @@ -310,7 +340,7 @@ private void handleWriteStatusTimeout() { // If we are still stuck in ENDED_STATE if (currentState.equals(State.ENDED_COMMIT)) { currentState = State.WRITE_STATUS_TIMEDOUT; - LOG.warn("Did not receive the Write Status from all partitions"); + LOG.warn("Current commit " + currentCommitTime + " failed after a write status timeout, so starting a new commit after recovery delay"); // Submit the next start commit submitEvent(new CoordinatorEvent(CoordinatorEvent.CoordinatorEventType.START_COMMIT, partition.topic(), @@ -365,6 +395,7 @@ private enum State { INIT, STARTED_COMMIT, ENDED_COMMIT, + FAILED_COMMIT, WRITE_STATUS_RCVD, WRITE_STATUS_TIMEDOUT, ACKED_COMMIT, diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionParticipant.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionParticipant.java index 19556dca45ea..1d32d03c371b 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionParticipant.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/ConnectTransactionParticipant.java @@ -149,7 +149,7 @@ private void handleStartCommit(ControlMessage message) { LOG.info("Started a new transaction after receiving START_COMMIT for commit " + currentCommitTime); try { ongoingTransactionInfo = new TransactionInfo<>(currentCommitTime, writerProvider.getWriter(currentCommitTime)); - ongoingTransactionInfo.setLastWrittenKafkaOffset(committedKafkaOffset); + ongoingTransactionInfo.setExpectedKafkaOffset(committedKafkaOffset); } catch (Exception exception) { LOG.warn("Error received while starting a new transaction", exception); } @@ -188,7 +188,7 @@ private void handleEndCommit(ControlMessage message) { .setParticipantInfo( ControlMessage.ParticipantInfo.newBuilder() .setWriteStatus(KafkaConnectUtils.buildWriteStatuses(writeStatuses)) - .setKafkaOffset(ongoingTransactionInfo.getLastWrittenKafkaOffset()) + .setKafkaOffset(ongoingTransactionInfo.getExpectedKafkaOffset()) .build() ).build(); @@ -201,9 +201,9 @@ private void handleEndCommit(ControlMessage message) { } private void handleAckCommit(ControlMessage message) { - // Update lastKafkCommitedOffset locally. - if (ongoingTransactionInfo != null && committedKafkaOffset < ongoingTransactionInfo.getLastWrittenKafkaOffset()) { - committedKafkaOffset = ongoingTransactionInfo.getLastWrittenKafkaOffset(); + // Update committedKafkaOffset that tracks the last committed kafka offset locally. + if (ongoingTransactionInfo != null && committedKafkaOffset < ongoingTransactionInfo.getExpectedKafkaOffset()) { + committedKafkaOffset = ongoingTransactionInfo.getExpectedKafkaOffset(); } syncKafkaOffsetWithLeader(message); cleanupOngoingTransaction(); @@ -215,12 +215,22 @@ private void writeRecords() { try { SinkRecord record = buffer.peek(); if (record != null - && record.kafkaOffset() >= ongoingTransactionInfo.getLastWrittenKafkaOffset()) { + && record.kafkaOffset() == ongoingTransactionInfo.getExpectedKafkaOffset()) { ongoingTransactionInfo.getWriter().writeRecord(record); - ongoingTransactionInfo.setLastWrittenKafkaOffset(record.kafkaOffset() + 1); - } else if (record != null && record.kafkaOffset() < committedKafkaOffset) { - LOG.warn(String.format("Received a kafka record with offset %s prior to last committed offset %s for partition %s", - record.kafkaOffset(), ongoingTransactionInfo.getLastWrittenKafkaOffset(), + ongoingTransactionInfo.setExpectedKafkaOffset(record.kafkaOffset() + 1); + } else if (record != null && record.kafkaOffset() > ongoingTransactionInfo.getExpectedKafkaOffset()) { + LOG.warn(String.format("Received a kafka record with offset %s above the next expected kafka offset %s for partition %s, " + + "hence resetting the kafka offset to %s", + record.kafkaOffset(), + ongoingTransactionInfo.getExpectedKafkaOffset(), + partition, + ongoingTransactionInfo.getExpectedKafkaOffset())); + context.offset(partition, ongoingTransactionInfo.getExpectedKafkaOffset()); + } else if (record != null && record.kafkaOffset() < ongoingTransactionInfo.getExpectedKafkaOffset()) { + LOG.warn(String.format("Received a kafka record with offset %s below the next expected kafka offset %s for partition %s, " + + "no action will be taken but this record will be ignored since its already written", + record.kafkaOffset(), + ongoingTransactionInfo.getExpectedKafkaOffset(), partition)); } buffer.poll(); @@ -250,13 +260,24 @@ private void syncKafkaOffsetWithLeader(ControlMessage message) { // Recover kafka committed offsets, treating the commit offset from the coordinator // as the source of truth if (coordinatorCommittedKafkaOffset != null && coordinatorCommittedKafkaOffset >= 0) { + // Debug only messages if (coordinatorCommittedKafkaOffset != committedKafkaOffset) { - LOG.warn(String.format("Recovering the kafka offset for partition %s to offset %s instead of local offset %s", - partition.partition(), coordinatorCommittedKafkaOffset, committedKafkaOffset)); - context.offset(partition, coordinatorCommittedKafkaOffset); + LOG.warn(String.format("The coordinator offset for kafka partition %s is %d while the locally committed offset is %d, " + + "hence resetting the local committed offset to the coordinator provided one to ensure consistency", + partition, + coordinatorCommittedKafkaOffset, + committedKafkaOffset)); } committedKafkaOffset = coordinatorCommittedKafkaOffset; + return; } + } else { + LOG.warn(String.format("The coordinator offset for kafka partition %s is not present while the locally committed offset is %d, " + + "hence resetting the local committed offset to 0 to avoid data loss", + partition, + committedKafkaOffset)); } + // If the coordinator does not have a committed offset for this partition, reset to zero offset. + committedKafkaOffset = 0; } } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/TransactionInfo.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/TransactionInfo.java index 9c7bbf1e8365..7c1852e5fa5c 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/TransactionInfo.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/transaction/TransactionInfo.java @@ -29,13 +29,13 @@ public class TransactionInfo { private final String commitTime; private final ConnectWriter writer; - private long lastWrittenKafkaOffset; + private long expectedKafkaOffset; private boolean commitInitiated; public TransactionInfo(String commitTime, ConnectWriter writer) { this.commitTime = commitTime; this.writer = writer; - this.lastWrittenKafkaOffset = 0; + this.expectedKafkaOffset = 0; this.commitInitiated = false; } @@ -47,16 +47,16 @@ public ConnectWriter getWriter() { return writer; } - public long getLastWrittenKafkaOffset() { - return lastWrittenKafkaOffset; + public long getExpectedKafkaOffset() { + return expectedKafkaOffset; } public boolean isCommitInitiated() { return commitInitiated; } - public void setLastWrittenKafkaOffset(long lastWrittenKafkaOffset) { - this.lastWrittenKafkaOffset = lastWrittenKafkaOffset; + public void setExpectedKafkaOffset(long expectedKafkaOffset) { + this.expectedKafkaOffset = expectedKafkaOffset; } public void commitInitiated() { diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java index 3c77063ddf2f..cf60b9e5c191 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.util.SerializationUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.connect.ControlMessage; +import org.apache.hudi.connect.writers.KafkaConnectConfigs; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.CustomAvroKeyGenerator; @@ -63,6 +64,7 @@ public class KafkaConnectUtils { private static final Logger LOG = LogManager.getLogger(KafkaConnectUtils.class); + private static final String HOODIE_CONF_PREFIX = "hoodie."; public static int getLatestNumPartitions(String bootstrapServers, String topicName) { Properties props = new Properties(); @@ -85,9 +87,15 @@ public static int getLatestNumPartitions(String bootstrapServers, String topicNa * * @return */ - public static Configuration getDefaultHadoopConf() { + public static Configuration getDefaultHadoopConf(KafkaConnectConfigs connectConfigs) { Configuration hadoopConf = new Configuration(); - hadoopConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + connectConfigs.getProps().keySet().stream().filter(prop -> { + // In order to prevent printing unnecessary warn logs, here filter out the hoodie + // configuration items before passing to hadoop/hive configs + return !prop.toString().startsWith(HOODIE_CONF_PREFIX); + }).forEach(prop -> { + hadoopConf.set(prop.toString(), connectConfigs.getProps().get(prop.toString()).toString()); + }); return hadoopConf; } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/ConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/ConnectTransactionServices.java index b36e1f1c7ab5..2ce44ff802bc 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/ConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/ConnectTransactionServices.java @@ -32,7 +32,7 @@ public interface ConnectTransactionServices { String startCommit(); - void endCommit(String commitTime, List writeStatuses, Map extraMetadata); + boolean endCommit(String commitTime, List writeStatuses, Map extraMetadata); Map fetchLatestExtraCommitMetadata(); } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java index 773ce1e048a7..ec0345104680 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java @@ -73,6 +73,11 @@ public class KafkaConnectConfigs extends HoodieConfig { + "the coordinator will wait for the write statuses from all the partitions" + "to ignore the current commit and start a new commit."); + public static final ConfigProperty ASYNC_COMPACT_ENABLE = ConfigProperty + .key("hoodie.kafka.compaction.async.enable") + .defaultValue("true") + .withDocumentation("Controls whether async compaction should be turned on for MOR table writing."); + public static final ConfigProperty META_SYNC_ENABLE = ConfigProperty .key("hoodie.meta.sync.enable") .defaultValue("false") @@ -83,14 +88,17 @@ public class KafkaConnectConfigs extends HoodieConfig { .defaultValue(HiveSyncTool.class.getName()) .withDocumentation("Meta sync client tool, using comma to separate multi tools"); + public static final ConfigProperty ALLOW_COMMIT_ON_ERRORS = ConfigProperty + .key("hoodie.kafka.allow.commit.on.errors") + .defaultValue(true) + .withDocumentation("Commit even when some records failed to be written"); + protected KafkaConnectConfigs() { super(); } protected KafkaConnectConfigs(Properties props) { super(props); - Properties newProps = new Properties(); - newProps.putAll(props); } public static KafkaConnectConfigs.Builder newBuilder() { @@ -121,6 +129,10 @@ public String getKafkaValueConverter() { return getString(KAFKA_VALUE_CONVERTER); } + public Boolean isAsyncCompactEnabled() { + return getBoolean(ASYNC_COMPACT_ENABLE); + } + public Boolean isMetaSyncEnabled() { return getBoolean(META_SYNC_ENABLE); } @@ -129,6 +141,10 @@ public String getMetaSyncClasses() { return getString(META_SYNC_CLASSES); } + public Boolean allowCommitOnErrors() { + return getBoolean(ALLOW_COMMIT_ON_ERRORS); + } + public static class Builder { protected final KafkaConnectConfigs connectConfigs = new KafkaConnectConfigs(); @@ -153,6 +169,11 @@ public Builder withCoordinatorWriteTimeoutSecs(Long coordinatorWriteTimeoutSecs) return this; } + public Builder withAllowCommitOnErrors(Boolean allowCommitOnErrors) { + connectConfigs.setValue(ALLOW_COMMIT_ON_ERRORS, String.valueOf(allowCommitOnErrors)); + return this; + } + // Kafka connect task are passed with props with type Map<> public Builder withProperties(Map properties) { connectConfigs.getProps().putAll(properties); diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java index 8039e56d37ba..cca738a70eba 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java @@ -18,30 +18,43 @@ package org.apache.hudi.connect.writers; +import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.connect.transaction.TransactionCoordinator; import org.apache.hudi.connect.utils.KafkaConnectUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory; +import org.apache.hudi.sync.common.AbstractSyncTool; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Properties; +import java.util.Set; /** * Implementation of Transaction service APIs used by @@ -52,8 +65,10 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic private static final Logger LOG = LogManager.getLogger(KafkaConnectTransactionServices.class); + private final KafkaConnectConfigs connectConfigs; private final Option tableMetaClient; private final Configuration hadoopConf; + private final HoodieWriteConfig writeConfig; private final String tableBasePath; private final String tableName; private final HoodieEngineContext context; @@ -61,12 +76,16 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic private final HoodieJavaWriteClient javaClient; public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throws HoodieException { - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() - .withProperties(connectConfigs.getProps()).build(); + this.connectConfigs = connectConfigs; + // This is the writeConfig for the Transaction Coordinator + this.writeConfig = HoodieWriteConfig.newBuilder() + .withEngineType(EngineType.JAVA) + .withProperties(connectConfigs.getProps()) + .build(); tableBasePath = writeConfig.getBasePath(); tableName = writeConfig.getTableName(); - hadoopConf = KafkaConnectUtils.getDefaultHadoopConf(); + hadoopConf = KafkaConnectUtils.getDefaultHadoopConf(connectConfigs); context = new HoodieJavaEngineContext(hadoopConf); try { @@ -95,6 +114,7 @@ public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throw } } + @Override public String startCommit() { String newCommitTime = javaClient.startCommit(); javaClient.transitionInflight(newCommitTime); @@ -102,11 +122,27 @@ public String startCommit() { return newCommitTime; } - public void endCommit(String commitTime, List writeStatuses, Map extraMetadata) { - javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata)); - LOG.info("Ending Hudi commit " + commitTime); + @Override + public boolean endCommit(String commitTime, List writeStatuses, Map extraMetadata) { + boolean success = javaClient.commit(commitTime, writeStatuses, Option.of(extraMetadata)); + if (success) { + LOG.info("Ending Hudi commit " + commitTime); + + // Schedule clustering and compaction as needed. + if (writeConfig.isAsyncClusteringEnabled()) { + javaClient.scheduleClustering(Option.empty()).ifPresent( + instantTs -> LOG.info("Scheduled clustering at instant time:" + instantTs)); + } + if (isAsyncCompactionEnabled()) { + javaClient.scheduleCompaction(Option.empty()).ifPresent( + instantTs -> LOG.info("Scheduled compaction at instant time:" + instantTs)); + } + syncMeta(); + } + return success; } + @Override public Map fetchLatestExtraCommitMetadata() { if (tableMetaClient.isPresent()) { Option metadata = KafkaConnectUtils.getCommitMetadataForLatestInstant(tableMetaClient.get()); @@ -119,4 +155,50 @@ public Map fetchLatestExtraCommitMetadata() { } throw new HoodieException("Fatal error retrieving Hoodie Extra Metadata since Table Meta Client is absent"); } + + private boolean isAsyncCompactionEnabled() { + return tableMetaClient.isPresent() + && HoodieTableType.MERGE_ON_READ.equals(tableMetaClient.get().getTableType()) + && connectConfigs.isAsyncCompactEnabled(); + } + + private void syncMeta() { + Set syncClientToolClasses = new HashSet<>( + Arrays.asList(connectConfigs.getMetaSyncClasses().split(","))); + if (connectConfigs.isMetaSyncEnabled()) { + for (String impl : syncClientToolClasses) { + impl = impl.trim(); + switch (impl) { + case "org.apache.hudi.hive.HiveSyncTool": + syncHive(); + break; + default: + FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration()); + Properties properties = new Properties(); + properties.putAll(connectConfigs.getProps()); + properties.put("basePath", tableBasePath); + AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[] {Properties.class, FileSystem.class}, properties, fs); + syncTool.syncHoodieTable(); + } + } + } + } + + private void syncHive() { + HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig( + new TypedProperties(connectConfigs.getProps()), + tableBasePath, + "PARQUET"); + LOG.info("Syncing target hoodie table with hive table(" + + hiveSyncConfig.tableName + + "). Hive metastore URL :" + + hiveSyncConfig.jdbcUrl + + ", basePath :" + tableBasePath); + LOG.info("Hive Sync Conf => " + hiveSyncConfig.toString()); + FileSystem fs = FSUtils.getFs(tableBasePath, hadoopConf); + HiveConf hiveConf = new HiveConf(); + hiveConf.addResource(fs.getConf()); + LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString()); + new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable(); + } } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java index 9d007dd09344..6ab0469738df 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java @@ -22,10 +22,13 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.connect.KafkaConnectFileIdPrefixProvider; @@ -62,7 +65,7 @@ public KafkaConnectWriterProvider( KafkaConnectConfigs connectConfigs, TopicPartition partition) throws HoodieException { this.connectConfigs = connectConfigs; - Configuration hadoopConf = KafkaConnectUtils.getDefaultHadoopConf(); + Configuration hadoopConf = KafkaConnectUtils.getDefaultHadoopConf(connectConfigs); try { this.schemaProvider = StringUtils.isNullOrEmpty(connectConfigs.getSchemaProviderClass()) ? null @@ -72,8 +75,9 @@ public KafkaConnectWriterProvider( this.keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator( new TypedProperties(connectConfigs.getProps())); - // Create the write client to write some records in + // This is the writeConfig for the writers for the individual Transaction Coordinators writeConfig = HoodieWriteConfig.newBuilder() + .withEngineType(EngineType.JAVA) .withProperties(connectConfigs.getProps()) .withFileIdPrefixProviderClassName(KafkaConnectFileIdPrefixProvider.class.getName()) .withProps(Collections.singletonMap( @@ -82,6 +86,14 @@ public KafkaConnectWriterProvider( .withSchema(schemaProvider.getSourceSchema().toString()) .withAutoCommit(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + // participants should not trigger table services, and leave it to the coordinator + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withAutoClean(false) + .withAutoArchive(false) + .withInlineCompaction(false).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClustering(false) + .build()) .build(); context = new HoodieJavaEngineContext(hadoopConf); diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionCoordinator.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionCoordinator.java index 6e049c6118b0..f003fe9fa645 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionCoordinator.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionCoordinator.java @@ -50,7 +50,7 @@ public class TestConnectTransactionCoordinator { private static final String TOPIC_NAME = "kafka-connect-test-topic"; - private static final int NUM_PARTITIONS = 4; + private static final int TOTAL_KAFKA_PARTITIONS = 4; private static final int MAX_COMMIT_ROUNDS = 5; private static final int TEST_TIMEOUT_SECS = 60; @@ -63,10 +63,6 @@ public class TestConnectTransactionCoordinator { @BeforeEach public void setUp() throws Exception { transactionServices = new MockConnectTransactionServices(); - configs = KafkaConnectConfigs.newBuilder() - .withCommitIntervalSecs(1L) - .withCoordinatorWriteTimeoutSecs(1L) - .build(); latch = new CountDownLatch(1); } @@ -77,13 +73,22 @@ public void testSingleCommitScenario(MockParticipant.TestScenarios scenario) thr participant = new MockParticipant(kafkaControlAgent, latch, scenario, MAX_COMMIT_ROUNDS); participant.start(); + KafkaConnectConfigs.Builder configBuilder = KafkaConnectConfigs.newBuilder() + .withCommitIntervalSecs(1L) + .withCoordinatorWriteTimeoutSecs(1L); + + if (scenario.equals(MockParticipant.TestScenarios.SUBSET_WRITE_STATUS_FAILED)) { + configBuilder.withAllowCommitOnErrors(false); + } + configs = configBuilder.build(); + // Test the coordinator using the mock participant TransactionCoordinator coordinator = new ConnectTransactionCoordinator( configs, new TopicPartition(TOPIC_NAME, 0), kafkaControlAgent, transactionServices, - (bootstrapServers, topicName) -> NUM_PARTITIONS); + (bootstrapServers, topicName) -> TOTAL_KAFKA_PARTITIONS); coordinator.start(); latch.await(TEST_TIMEOUT_SECS, TimeUnit.SECONDS); @@ -119,7 +124,7 @@ public MockParticipant(MockKafkaControlAgent kafkaControlAgent, this.latch = latch; this.testScenario = testScenario; this.maxNumberCommitRounds = maxNumberCommitRounds; - this.partition = new TopicPartition(TOPIC_NAME, (NUM_PARTITIONS - 1)); + this.partition = new TopicPartition(TOPIC_NAME, (TOTAL_KAFKA_PARTITIONS - 1)); this.kafkaOffsetsCommitted = new HashMap<>(); expectedMsgType = ControlMessage.EventType.START_COMMIT; numberCommitRounds = 0; @@ -162,39 +167,40 @@ public long getLastKafkaCommittedOffset() { private void testScenarios(ControlMessage message) { assertEquals(expectedMsgType, message.getType()); - switch (message.getType()) { case START_COMMIT: expectedMsgType = ControlMessage.EventType.END_COMMIT; break; case END_COMMIT: assertEquals(kafkaOffsetsCommitted, message.getCoordinatorInfo().getGlobalKafkaCommitOffsets()); - int numSuccessPartitions; + int numPartitionsThatReportWriteStatus; Map kafkaOffsets = new HashMap<>(); List controlEvents = new ArrayList<>(); - // Prepare the WriteStatuses for all partitions - for (int i = 1; i <= NUM_PARTITIONS; i++) { - try { - long kafkaOffset = (long) (Math.random() * 10000); - kafkaOffsets.put(i, kafkaOffset); - ControlMessage event = successWriteStatus( - message.getCommitTime(), - new TopicPartition(TOPIC_NAME, i), - kafkaOffset); - controlEvents.add(event); - } catch (Exception exception) { - throw new HoodieException("Fatal error sending control event to Coordinator"); - } - } - switch (testScenario) { case ALL_CONNECT_TASKS_SUCCESS: - numSuccessPartitions = NUM_PARTITIONS; + composeControlEvent(message.getCommitTime(), false, kafkaOffsets, controlEvents); + numPartitionsThatReportWriteStatus = TOTAL_KAFKA_PARTITIONS; + // This commit round should succeed, and the kafka offsets getting committed + kafkaOffsetsCommitted.putAll(kafkaOffsets); + expectedMsgType = ControlMessage.EventType.ACK_COMMIT; + break; + case SUBSET_WRITE_STATUS_FAILED_BUT_IGNORED: + composeControlEvent(message.getCommitTime(), true, kafkaOffsets, controlEvents); + numPartitionsThatReportWriteStatus = TOTAL_KAFKA_PARTITIONS; + // Despite error records, this commit round should succeed, and the kafka offsets getting committed kafkaOffsetsCommitted.putAll(kafkaOffsets); expectedMsgType = ControlMessage.EventType.ACK_COMMIT; break; + case SUBSET_WRITE_STATUS_FAILED: + composeControlEvent(message.getCommitTime(), true, kafkaOffsets, controlEvents); + numPartitionsThatReportWriteStatus = TOTAL_KAFKA_PARTITIONS; + // This commit round should fail, and a new commit round should start without kafka offsets getting committed + expectedMsgType = ControlMessage.EventType.START_COMMIT; + break; case SUBSET_CONNECT_TASKS_FAILED: - numSuccessPartitions = NUM_PARTITIONS / 2; + composeControlEvent(message.getCommitTime(), false, kafkaOffsets, controlEvents); + numPartitionsThatReportWriteStatus = TOTAL_KAFKA_PARTITIONS / 2; + // This commit round should fail, and a new commit round should start without kafka offsets getting committed expectedMsgType = ControlMessage.EventType.START_COMMIT; break; default: @@ -202,7 +208,7 @@ private void testScenarios(ControlMessage message) { } // Send events based on test scenario - for (int i = 0; i < numSuccessPartitions; i++) { + for (int i = 0; i < numPartitionsThatReportWriteStatus; i++) { kafkaControlAgent.publishMessage(controlEvents.get(i)); } break; @@ -227,18 +233,36 @@ private void testScenarios(ControlMessage message) { public enum TestScenarios { SUBSET_CONNECT_TASKS_FAILED, + SUBSET_WRITE_STATUS_FAILED, + SUBSET_WRITE_STATUS_FAILED_BUT_IGNORED, ALL_CONNECT_TASKS_SUCCESS } - private static ControlMessage successWriteStatus(String commitTime, - TopicPartition partition, - long kafkaOffset) throws Exception { - // send WS - WriteStatus writeStatus = new WriteStatus(); - WriteStatus status = new WriteStatus(false, 1.0); - for (int i = 0; i < 1000; i++) { - status.markSuccess(mock(HoodieRecord.class), Option.empty()); + private static void composeControlEvent(String commitTime, boolean shouldIncludeFailedRecords, Map kafkaOffsets, List controlEvents) { + // Prepare the WriteStatuses for all partitions + for (int i = 1; i <= TOTAL_KAFKA_PARTITIONS; i++) { + try { + long kafkaOffset = (long) (Math.random() * 10000); + kafkaOffsets.put(i, kafkaOffset); + ControlMessage event = composeWriteStatusResponse( + commitTime, + new TopicPartition(TOPIC_NAME, i), + kafkaOffset, + shouldIncludeFailedRecords); + controlEvents.add(event); + } catch (Exception exception) { + throw new HoodieException("Fatal error sending control event to Coordinator"); + } } + } + + private static ControlMessage composeWriteStatusResponse(String commitTime, + TopicPartition partition, + long kafkaOffset, + boolean includeFailedRecords) throws Exception { + // send WS + WriteStatus writeStatus = includeFailedRecords ? getSubsetFailedRecordsWriteStatus() : getAllSuccessfulRecordsWriteStatus(); + return ControlMessage.newBuilder() .setType(ControlMessage.EventType.WRITE_STATUS) .setTopicName(partition.topic()) @@ -255,4 +279,27 @@ private static ControlMessage successWriteStatus(String commitTime, ).build(); } } + + private static WriteStatus getAllSuccessfulRecordsWriteStatus() { + // send WS + WriteStatus status = new WriteStatus(false, 0.0); + for (int i = 0; i < 1000; i++) { + status.markSuccess(mock(HoodieRecord.class), Option.empty()); + } + return status; + } + + private static WriteStatus getSubsetFailedRecordsWriteStatus() { + // send WS + WriteStatus status = new WriteStatus(false, 0.0); + for (int i = 0; i < 1000; i++) { + if (i % 10 == 0) { + status.markFailure(mock(HoodieRecord.class), new Throwable("Error writing record on disk"), Option.empty()); + } else { + status.markSuccess(mock(HoodieRecord.class), Option.empty()); + } + } + status.setGlobalError(new Throwable("More than one records failed to be written to storage")); + return status; + } } diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionParticipant.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionParticipant.java index 5d551a79fa03..36da6ac32e01 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionParticipant.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/connect/TestConnectTransactionParticipant.java @@ -26,7 +26,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.helper.MockKafkaControlAgent; import org.apache.hudi.helper.TestHudiWriterProvider; -import org.apache.hudi.helper.TestKafkaConnect; +import org.apache.hudi.helper.MockKafkaConnect; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.BeforeEach; @@ -41,23 +41,24 @@ public class TestConnectTransactionParticipant { private static final String TOPIC_NAME = "kafka-connect-test-topic"; + private static final int NUM_RECORDS_BATCH = 5; private static final int PARTITION_NUMBER = 4; private ConnectTransactionParticipant participant; - private MockCoordinator coordinator; + private MockCoordinator mockCoordinator; private TopicPartition partition; private KafkaConnectConfigs configs; private KafkaControlAgent kafkaControlAgent; private TestHudiWriterProvider testHudiWriterProvider; - private TestKafkaConnect testKafkaConnect; + private MockKafkaConnect mockKafkaConnect; @BeforeEach public void setUp() throws Exception { partition = new TopicPartition(TOPIC_NAME, PARTITION_NUMBER); kafkaControlAgent = new MockKafkaControlAgent(); - testKafkaConnect = new TestKafkaConnect(partition); - coordinator = new MockCoordinator(kafkaControlAgent); - coordinator.start(); + mockKafkaConnect = new MockKafkaConnect(partition); + mockCoordinator = new MockCoordinator(kafkaControlAgent); + mockCoordinator.start(); configs = KafkaConnectConfigs.newBuilder() .build(); initializeParticipant(); @@ -66,26 +67,19 @@ public void setUp() throws Exception { @ParameterizedTest @EnumSource(value = CoordinatorFailureTestScenarios.class) public void testAllCoordinatorFailureScenarios(CoordinatorFailureTestScenarios testScenario) { - int expectedRecordsWritten = 0; try { + assertTrue(mockKafkaConnect.isPaused()); switch (testScenario) { case REGULAR_SCENARIO: - expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isPaused()); break; case COORDINATOR_FAILED_AFTER_START_COMMIT: - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - testKafkaConnect.putRecordsToParticipant(); + triggerAndProcessStartCommit(); // Coordinator Failed initializeCoordinator(); break; case COORDINATOR_FAILED_AFTER_END_COMMIT: - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant(); + triggerAndProcessStartCommit(); + triggerAndProcessEndCommit(); // Coordinator Failed initializeCoordinator(); break; @@ -93,18 +87,8 @@ public void testAllCoordinatorFailureScenarios(CoordinatorFailureTestScenarios t throw new HoodieException("Unknown test scenario " + testScenario); } - // Regular Case or Coordinator Recovery Case - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isResumed()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isPaused()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten); - // Ensure Coordinator and participant are in sync in the kafka offsets - assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset()); + // Despite failures in the previous commit, a fresh 2-phase commit should PASS. + testTwoPhaseCommit(0); } catch (Exception exception) { throw new HoodieException("Unexpected test failure ", exception); } @@ -114,62 +98,38 @@ public void testAllCoordinatorFailureScenarios(CoordinatorFailureTestScenarios t @ParameterizedTest @EnumSource(value = ParticipantFailureTestScenarios.class) public void testAllParticipantFailureScenarios(ParticipantFailureTestScenarios testScenario) { - int expectedRecordsWritten = 0; try { + int currentKafkaOffset = 0; switch (testScenario) { case FAILURE_BEFORE_START_COMMIT: - testKafkaConnect.putRecordsToParticipant(); - // Participant fails + // Participant failing after START_COMMIT will not write any data in this commit cycle. initializeParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - expectedRecordsWritten += testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isResumed()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isPaused()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten); - // Ensure Coordinator and participant are in sync in the kafka offsets - assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset()); break; case FAILURE_AFTER_START_COMMIT: - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - // Participant fails + triggerAndProcessStartCommit(); + // Participant failing after START_COMMIT will not write any data in this commit cycle. initializeParticipant(); - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isPaused()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten); - // Ensure Coordinator and participant are in sync in the kafka offsets - assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset()); + triggerAndProcessEndCommit(); + triggerAndProcessAckCommit(); break; case FAILURE_AFTER_END_COMMIT: - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - // Participant fails + // Regular Case or Coordinator Recovery Case + triggerAndProcessStartCommit(); + triggerAndProcessEndCommit(); initializeParticipant(); - testKafkaConnect.putRecordsToParticipant(); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertTrue(testKafkaConnect.isPaused()); - coordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT); - testKafkaConnect.putRecordsToParticipant(); - assertEquals(testHudiWriterProvider.getLatestNumberWrites(), expectedRecordsWritten); - // Ensure Coordinator and participant are in sync in the kafka offsets - assertEquals(participant.getLastKafkaCommittedOffset(), coordinator.getCommittedKafkaOffset()); + triggerAndProcessAckCommit(); + + // Participant failing after and END_COMMIT should not cause issues with the present commit, + // since the data would have been written by previous participant before failing + // and hence moved the kafka offset. + currentKafkaOffset = NUM_RECORDS_BATCH; break; default: throw new HoodieException("Unknown test scenario " + testScenario); } + + // Despite failures in the previous commit, a fresh 2-phase commit should PASS. + testTwoPhaseCommit(currentKafkaOffset); } catch (Exception exception) { throw new HoodieException("Unexpected test failure ", exception); } @@ -180,15 +140,49 @@ private void initializeParticipant() { participant = new ConnectTransactionParticipant( partition, kafkaControlAgent, - testKafkaConnect, + mockKafkaConnect, testHudiWriterProvider); - testKafkaConnect.setParticipant(participant); + mockKafkaConnect.setParticipant(participant); participant.start(); } private void initializeCoordinator() { - coordinator = new MockCoordinator(kafkaControlAgent); - coordinator.start(); + mockCoordinator = new MockCoordinator(kafkaControlAgent); + mockCoordinator.start(); + } + + // Test and validate result of a single 2 Phase commit from START_COMMIT to ACK_COMMIT. + // Validates that NUM_RECORDS_BATCH number of kafka records are written, + // and the kafka offset only increments by NUM_RECORDS_BATCH. + private void testTwoPhaseCommit(long currentKafkaOffset) { + triggerAndProcessStartCommit(); + triggerAndProcessEndCommit(); + triggerAndProcessAckCommit(); + + // Validate records written, current kafka offset and kafka offsets committed across + // coordinator and participant are in sync despite failure scenarios. + assertEquals(NUM_RECORDS_BATCH, testHudiWriterProvider.getLatestNumberWrites()); + assertEquals((currentKafkaOffset + NUM_RECORDS_BATCH), mockKafkaConnect.getCurrentKafkaOffset()); + // Ensure Coordinator and participant are in sync in the kafka offsets + assertEquals(participant.getLastKafkaCommittedOffset(), mockCoordinator.getCommittedKafkaOffset()); + } + + private void triggerAndProcessStartCommit() { + mockCoordinator.sendEventFromCoordinator(ControlMessage.EventType.START_COMMIT); + mockKafkaConnect.publishBatchRecordsToParticipant(NUM_RECORDS_BATCH); + assertTrue(mockKafkaConnect.isResumed()); + } + + private void triggerAndProcessEndCommit() { + mockCoordinator.sendEventFromCoordinator(ControlMessage.EventType.END_COMMIT); + mockKafkaConnect.publishBatchRecordsToParticipant(0); + assertTrue(mockKafkaConnect.isPaused()); + } + + private void triggerAndProcessAckCommit() { + mockCoordinator.sendEventFromCoordinator(ControlMessage.EventType.ACK_COMMIT); + mockKafkaConnect.publishBatchRecordsToParticipant(0); + assertTrue(mockKafkaConnect.isPaused()); } private static class MockCoordinator implements TransactionCoordinator { @@ -279,5 +273,4 @@ private enum ParticipantFailureTestScenarios { FAILURE_AFTER_START_COMMIT, FAILURE_AFTER_END_COMMIT, } - } diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockConnectTransactionServices.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockConnectTransactionServices.java index 6994c6554d9a..b3314ade3d5b 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockConnectTransactionServices.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockConnectTransactionServices.java @@ -46,8 +46,9 @@ public String startCommit() { } @Override - public void endCommit(String commitTime, List writeStatuses, Map extraMetadata) { + public boolean endCommit(String commitTime, List writeStatuses, Map extraMetadata) { assertEquals(String.valueOf(this.commitTime), commitTime); + return true; } @Override diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/TestKafkaConnect.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockKafkaConnect.java similarity index 73% rename from hudi-kafka-connect/src/test/java/org/apache/hudi/helper/TestKafkaConnect.java rename to hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockKafkaConnect.java index 6e947de07221..113b93ef8712 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/TestKafkaConnect.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/helper/MockKafkaConnect.java @@ -25,7 +25,6 @@ import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import java.io.IOException; import java.util.Arrays; import java.util.Map; import java.util.Set; @@ -33,20 +32,26 @@ /** * Helper class that emulates the Kafka Connect f/w and additionally * implements {@link SinkTaskContext} for testing purposes. + * + * Everytime the consumer (Participant) calls resume, a fixed + * batch of kafka records from the current offset are pushed. If + * the consumer resets the offsets, then a fresh batch of records + * are sent from the new offset. */ -public class TestKafkaConnect implements SinkTaskContext { +public class MockKafkaConnect implements SinkTaskContext { - private static final int NUM_RECORDS_BATCH = 5; private final TopicPartition testPartition; private TransactionParticipant participant; private long currentKafkaOffset; private boolean isPaused; + private boolean isResetOffset; - public TestKafkaConnect(TopicPartition testPartition) { + public MockKafkaConnect(TopicPartition testPartition) { this.testPartition = testPartition; isPaused = false; currentKafkaOffset = 0L; + isResetOffset = false; } public void setParticipant(TransactionParticipant participant) { @@ -61,23 +66,6 @@ public boolean isResumed() { return !isPaused; } - public int putRecordsToParticipant() throws IOException { - for (int i = 1; i <= NUM_RECORDS_BATCH; i++) { - participant.buffer(getNextKafkaRecord()); - } - participant.processRecords(); - return NUM_RECORDS_BATCH; - } - - public SinkRecord getNextKafkaRecord() { - return new SinkRecord(testPartition.topic(), - testPartition.partition(), - Schema.OPTIONAL_BYTES_SCHEMA, - ("key-" + currentKafkaOffset).getBytes(), - Schema.OPTIONAL_BYTES_SCHEMA, - "value".getBytes(), currentKafkaOffset++); - } - public long getCurrentKafkaOffset() { return currentKafkaOffset; } @@ -100,7 +88,7 @@ public void resume(TopicPartition... partitions) { public void offset(Map offsets) { for (TopicPartition tp : offsets.keySet()) { if (tp.equals(testPartition)) { - currentKafkaOffset = offsets.get(tp); + resetOffset(offsets.get(tp)); } } } @@ -108,7 +96,7 @@ public void offset(Map offsets) { @Override public void offset(TopicPartition tp, long offset) { if (tp.equals(testPartition)) { - currentKafkaOffset = offset; + resetOffset(offset); } } @@ -129,6 +117,33 @@ public Set assignment() { @Override public void requestCommit() { + } + + public int publishBatchRecordsToParticipant(int numRecords) { + // Send NUM_RECORDS_BATCH to participant + // If client resets offset, send another batch starting + // from the new reset offset value + do { + isResetOffset = false; + for (int i = 1; i <= numRecords; i++) { + participant.buffer(getNextKafkaRecord()); + } + participant.processRecords(); + } while (isResetOffset); + return numRecords; + } + + private SinkRecord getNextKafkaRecord() { + return new SinkRecord(testPartition.topic(), + testPartition.partition(), + Schema.OPTIONAL_BYTES_SCHEMA, + ("key-" + currentKafkaOffset).getBytes(), + Schema.OPTIONAL_BYTES_SCHEMA, + "value".getBytes(), currentKafkaOffset++); + } + private void resetOffset(long newOffset) { + currentKafkaOffset = newOffset; + isResetOffset = true; } } diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestBufferedConnectWriter.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestBufferedConnectWriter.java index b0dcf38f4b9d..458c79a31062 100644 --- a/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestBufferedConnectWriter.java +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestBufferedConnectWriter.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; @@ -62,6 +63,7 @@ public void setUp() throws Exception { configs = KafkaConnectConfigs.newBuilder().build(); schemaProvider = new TestAbstractConnectWriter.TestSchemaProvider(); writeConfig = HoodieWriteConfig.newBuilder() + .withEngineType(EngineType.JAVA) .withPath("/tmp") .withSchema(schemaProvider.getSourceSchema().toString()) .build(); diff --git a/hudi-kafka-connect/src/test/resources/log4j-surefire-quiet.properties b/hudi-kafka-connect/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 000000000000..ca0a50c84270 --- /dev/null +++ b/hudi-kafka-connect/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,30 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# CONSOLE is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# CONSOLE uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-kafka-connect/src/test/resources/log4j-surefire.properties b/hudi-kafka-connect/src/test/resources/log4j-surefire.properties index 9ee04e1a3bf2..32af462093ae 100644 --- a/hudi-kafka-connect/src/test/resources/log4j-surefire.properties +++ b/hudi-kafka-connect/src/test/resources/log4j-surefire.properties @@ -29,4 +29,3 @@ log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true log4j.appender.CONSOLE.filter.a.LevelMin=WARN log4j.appender.CONSOLE.filter.a.LevelMax=FATAL - diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml index c0ea130fc4b6..caa218a70c66 100644 --- a/hudi-spark-datasource/hudi-spark-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark-common/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-spark-common_${scala.binary.version} - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-spark-common_${scala.binary.version} jar diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index b98417ef2b8f..1e076b1968a7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -44,14 +44,15 @@ import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; -import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.util.DataTypeUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; import java.io.IOException; import java.util.ArrayList; @@ -183,7 +184,6 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base } return builder.forTable(tblName) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) .withInlineCompaction(inlineCompact).build()) @@ -307,6 +307,21 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().defaultValue())); hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().key(), DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().defaultValue())); + hiveSyncConfig.bucketSpec = props.getBoolean(DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().key(), + (boolean) DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue()) + ? HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), + props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())) : null; return hiveSyncConfig; } + + // Now by default ParquetWriteSupport will write DecimalType to parquet as int32/int64 when the scale of decimalType < Decimal.MAX_LONG_DIGITS(), + // but AvroParquetReader which used by HoodieParquetReader cannot support read int32/int64 as DecimalType. + // try to find current schema whether contains that DecimalType, and auto set the value of "hoodie.parquet.writeLegacyFormat.enabled" + public static void mayBeOverwriteParquetWriteLegacyFormatProp(Map properties, StructType schema) { + if (DataTypeUtils.foundSmallPrecisionDecimalType(schema) + && !Boolean.parseBoolean(properties.getOrDefault("hoodie.parquet.writeLegacyFormat.enabled", "false"))) { + properties.put("hoodie.parquet.writeLegacyFormat.enabled", "true"); + LOG.warn("Small Decimal Type found in current schema, auto set the value of hoodie.parquet.writeLegacyFormat.enabled to true"); + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 94bcc0d0de85..dbcc847fc1c1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -18,18 +18,23 @@ package org.apache.hudi import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL} -import org.apache.hudi.common.config.ConfigProperty +import org.apache.hudi.common.config.{ConfigProperty, HoodieConfig} import org.apache.hudi.common.fs.ConsistencyGuardConfig import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.Option import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hive.util.ConfigUtils -import org.apache.hudi.hive.{HiveSyncTool, SlashEncodedDayPartitionValueExtractor} +import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor} import org.apache.hudi.keygen.constant.KeyGeneratorOptions -import org.apache.hudi.keygen.{CustomKeyGenerator, SimpleKeyGenerator} +import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.log4j.LogManager import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils} +import java.util.function.{Function => JavaFunction} +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + /** * List of options that can be passed to the Hoodie datasource, * in addition to the hoodie client configs @@ -211,7 +216,7 @@ object DataSourceWriteOptions { .map(SparkDataSourceUtils.decodePartitioningColumns) .getOrElse(Nil) val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key(), - DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL) + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.defaultValue) val partitionPathField = keyGeneratorClass match { @@ -273,8 +278,26 @@ object DataSourceWriteOptions { */ val HIVE_STYLE_PARTITIONING = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE - val KEYGENERATOR_CLASS_NAME = ConfigProperty.key("hoodie.datasource.write.keygenerator.class") + /** + * Key generator class, that implements will extract the key out of incoming record + * + */ + val keyGeneraterInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { + if (!p.contains(PARTITIONPATH_FIELD)) { + Option.of(classOf[NonpartitionedKeyGenerator].getName) + } else { + val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length + if (numOfPartFields == 1) { + Option.of(classOf[SimpleKeyGenerator].getName) + } else { + Option.of(classOf[ComplexKeyGenerator].getName) + } + } + }) + val KEYGENERATOR_CLASS_NAME: ConfigProperty[String] = ConfigProperty + .key("hoodie.datasource.write.keygenerator.class") .defaultValue(classOf[SimpleKeyGenerator].getName) + .withInferFunction(keyGeneraterInferFunc) .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator`") val ENABLE_ROW_WRITER: ConfigProperty[String] = ConfigProperty @@ -364,9 +387,19 @@ object DataSourceWriteOptions { .defaultValue("default") .withDocumentation("database to sync to") + val hiveTableOptKeyInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { + if (p.contains(TABLE_NAME)) { + Option.of(p.getString(TABLE_NAME)) + } else if (p.contains(HoodieWriteConfig.TBL_NAME)) { + Option.of(p.getString(HoodieWriteConfig.TBL_NAME)) + } else { + Option.empty[String]() + } + }) val HIVE_TABLE: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.hive_sync.table") .defaultValue("unknown") + .withInferFunction(hiveTableOptKeyInferFunc) .withDocumentation("table to sync to") val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = ConfigProperty @@ -389,16 +422,37 @@ object DataSourceWriteOptions { .defaultValue("jdbc:hive2://localhost:10000") .withDocumentation("Hive metastore url") + val hivePartitionFieldsInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { + if (p.contains(PARTITIONPATH_FIELD)) { + Option.of(p.getString(PARTITIONPATH_FIELD)) + } else { + Option.empty[String]() + } + }) val HIVE_PARTITION_FIELDS: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.hive_sync.partition_fields") .defaultValue("") .withDocumentation("Field in the table to use for determining hive partition columns.") - + .withInferFunction(hivePartitionFieldsInferFunc) + + val hivePartitionExtractorInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { + if (!p.contains(PARTITIONPATH_FIELD)) { + Option.of(classOf[NonPartitionedExtractor].getName) + } else { + val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length + if (numOfPartFields == 1 && p.contains(HIVE_STYLE_PARTITIONING) && p.getString(HIVE_STYLE_PARTITIONING) == "true") { + Option.of(classOf[HiveStylePartitionValueExtractor].getName) + } else { + Option.of(classOf[MultiPartKeysValueExtractor].getName) + } + } + }) val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.hive_sync.partition_extractor_class") .defaultValue(classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName) .withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, " + "default 'SlashEncodedDayPartitionValueExtractor'.") + .withInferFunction(hivePartitionExtractorInferFunc) val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.hive_sync.assume_date_partitioning") @@ -472,6 +526,12 @@ object DataSourceWriteOptions { .noDefaultValue() .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.") + val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[Boolean] = ConfigProperty + .key("hoodie.datasource.hive_sync.bucket_sync") + .defaultValue(false) + .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." + + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'") + // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.compaction.async.enable") @@ -622,7 +682,7 @@ object DataSourceWriteOptions { val PARTITIONPATH_FIELD_OPT_KEY = KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key() /** @deprecated Use {@link PARTITIONPATH_FIELD} and its methods instead */ @Deprecated - val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = PARTITIONPATH_FIELD.defaultValue() + val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = null /** @deprecated Use {@link TABLE_NAME} and its methods instead */ @Deprecated @@ -755,7 +815,7 @@ object DataSourceOptionsHelper { // maps the deprecated config name to its latest name val allAlternatives: Map[String, String] = { val alterMap = scala.collection.mutable.Map[String, String]() - allConfigsWithAlternatives.foreach(cfg => cfg.getAlternatives.foreach(alternative => alterMap(alternative) = cfg.key)) + allConfigsWithAlternatives.foreach(cfg => cfg.getAlternatives.asScala.foreach(alternative => alterMap(alternative) = cfg.key)) alterMap.toMap } @@ -794,4 +854,10 @@ object DataSourceOptionsHelper { QUERY_TYPE.key -> queryType ) ++ translateConfigurations(parameters) } + + implicit def scalaFunctionToJavaFunction[From, To](function: (From) => To): JavaFunction[From, To] = { + new JavaFunction[From, To] { + override def apply (input: From): To = function (input) + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java index b19b098443ba..95a023abb687 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java @@ -77,7 +77,7 @@ protected HoodieWriteConfig getWriteConfig(boolean populateMetaFields) { properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME); properties.setProperty(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); } - return getConfigBuilder(basePath).withProperties(properties).build(); + return getConfigBuilder(basePath, timelineServicePort).withProperties(properties).build(); } protected void assertWriteStatuses(List writeStatuses, int batches, int size, diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index 11ebeca39e96..e20f4ad48c07 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -19,12 +19,12 @@ hudi-spark-datasource org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-spark_${scala.binary.version} - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-spark_${scala.binary.version} jar @@ -48,6 +48,7 @@ -nobootcp + -target:jvm-1.8 false diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index a9d85af2ee47..9b437f5451b1 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -18,6 +18,7 @@ package org.apache.hudi import org.apache.hadoop.fs.Path + import org.apache.hudi.DataSourceReadOptions._ import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION} @@ -26,8 +27,9 @@ import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_REA import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.HoodieROTablePathFilter + import org.apache.log4j.LogManager -import org.apache.spark.sql.avro.SchemaConverters + import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -217,8 +219,7 @@ class DefaultSource extends RelationProvider // the table schema evolution. val tableSchemaResolver = new TableSchemaResolver(metaClient) try { - Some(SchemaConverters.toSqlType(tableSchemaResolver.getTableAvroSchema) - .dataType.asInstanceOf[StructType]) + Some(AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaResolver.getTableAvroSchema)) } catch { case _: Throwable => None // If there is no commit in the table, we can not get the schema diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 882636c4697b..572a97e9dbe3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -26,23 +26,27 @@ import org.apache.hudi.common.model.FileSlice import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} + import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Column, SparkSession} -import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.catalyst.{InternalRow, expressions} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} -import org.apache.spark.sql.hudi.{DataSkippingUtils, HoodieSqlUtils} +import org.apache.spark.sql.hudi.DataSkippingUtils.createColumnStatsIndexFilterExpr +import org.apache.spark.sql.hudi.HoodieSqlUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.{AnalysisException, Column, SparkSession} import org.apache.spark.unsafe.types.UTF8String + import java.util.Properties -import scala.collection.JavaConverters._ import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.util.{Failure, Success, Try} /** * A file index which support partition prune for hoodie snapshot and read-optimized query. @@ -96,8 +100,7 @@ case class HoodieFileIndex( */ lazy val schema: StructType = schemaSpec.getOrElse({ val schemaUtil = new TableSchemaResolver(metaClient) - SchemaConverters.toSqlType(schemaUtil.getTableAvroSchema) - .dataType.asInstanceOf[StructType] + AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) }) /** @@ -106,13 +109,19 @@ case class HoodieFileIndex( private lazy val _partitionSchemaFromProperties: StructType = { val tableConfig = metaClient.getTableConfig val partitionColumns = tableConfig.getPartitionFields - val nameFieldMap = schema.fields.map(filed => filed.name -> filed).toMap + val nameFieldMap = generateNameFieldMap(Right(schema)) if (partitionColumns.isPresent) { - val partitionFields = partitionColumns.get().map(column => - nameFieldMap.getOrElse(column, throw new IllegalArgumentException(s"Cannot find column: '" + + if (tableConfig.getKeyGeneratorClassName.equalsIgnoreCase(classOf[TimestampBasedKeyGenerator].getName) + || tableConfig.getKeyGeneratorClassName.equalsIgnoreCase(classOf[TimestampBasedAvroKeyGenerator].getName)) { + val partitionFields = partitionColumns.get().map(column => StructField(column, StringType)) + StructType(partitionFields) + } else { + val partitionFields = partitionColumns.get().map(column => + nameFieldMap.getOrElse(column, throw new IllegalArgumentException(s"Cannot find column: '" + s"$column' in the schema[${schema.fields.mkString(",")}]"))) - new StructType(partitionFields) + StructType(partitionFields) + } } else { // If the partition columns have not stored in hoodie.properties(the table that was // created earlier), we trait it as a non-partitioned table. logWarning("No partition columns available from hoodie.properties." + @@ -121,6 +130,25 @@ case class HoodieFileIndex( } } + /** + * This method traverses StructType recursively to build map of columnName -> StructField + * Note : If there is nesting of columns like ["a.b.c.d", "a.b.c.e"] -> final map will have keys corresponding + * only to ["a.b.c.d", "a.b.c.e"] and not for subsets like ["a.b.c", "a.b"] + * @param structField + * @return map of ( columns names -> StructField ) + */ + private def generateNameFieldMap(structField: Either[StructField, StructType]) : Map[String, StructField] = { + structField match { + case Right(field) => field.fields.map(f => generateNameFieldMap(Left(f))).flatten.toMap + case Left(field) => field.dataType match { + case struct: StructType => generateNameFieldMap(Right(struct)).map { + case (key: String, sf: StructField) => (field.name + "." + key, sf) + } + case _ => Map(field.name -> field) + } + } + } + private lazy val engineContext = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) private lazy val configProperties = { @@ -129,7 +157,7 @@ case class HoodieFileIndex( // To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users // would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing. - properties.put(HoodieMetadataConfig.ENABLE, + properties.setProperty(HoodieMetadataConfig.ENABLE.key(), sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(), HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString)) properties.putAll(options.asJava) @@ -159,41 +187,92 @@ case class HoodieFileIndex( spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean } - private def filterFilesByDataSkippingIndex(dataFilters: Seq[Expression]): Set[String] = { - var allFiles: Set[String] = Set.empty - var candidateFiles: Set[String] = Set.empty - val indexPath = metaClient.getZindexPath + /** + * Computes pruned list of candidate base-files' names based on provided list of {@link dataFilters} + * conditions, by leveraging custom Column Statistics index (col-stats-index) bearing "min", "max", + * "num_nulls" statistics for all clustered columns. + * + * NOTE: This method has to return complete set of candidate files, since only provided candidates will + * ultimately be scanned as part of query execution. Hence, this method has to maintain the + * invariant of conservatively including every base-file's name, that is NOT referenced in its index. + * + * @param queryFilters list of original data filters passed down from querying engine + * @return list of pruned (data-skipped) candidate base-files' names + */ + private def lookupCandidateFilesInColStatsIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try { + val indexPath = metaClient.getColumnStatsIndexPath val fs = metaClient.getFs - if (fs.exists(new Path(indexPath)) && dataFilters.nonEmpty) { - // try to load latest index table from index path - val candidateIndexTables = fs.listStatus(new Path(indexPath)).filter(_.isDirectory) - .map(_.getPath.getName).filter(f => completedCommits.contains(f)).sortBy(x => x) - if (candidateIndexTables.nonEmpty) { - val dataFrameOpt = try { - Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString)) - } catch { - case _: Throwable => - logError("missing index skip data-skipping") - None - } - if (dataFrameOpt.isDefined) { - val indexSchema = dataFrameOpt.get.schema - val indexFiles = DataSkippingUtils.getIndexFiles(spark.sparkContext.hadoopConfiguration, new Path(indexPath, candidateIndexTables.last).toString) - val indexFilter = dataFilters.map(DataSkippingUtils.createZindexFilter(_, indexSchema)).reduce(And) - logInfo(s"index filter condition: $indexFilter") - dataFrameOpt.get.persist() - if (indexFiles.size <= 4) { - allFiles = DataSkippingUtils.readParquetFile(spark, indexFiles) - } else { - allFiles = dataFrameOpt.get.select("file").collect().map(_.getString(0)).toSet - } - candidateFiles = dataFrameOpt.get.filter(new Column(indexFilter)).select("file").collect().map(_.getString(0)).toSet - dataFrameOpt.get.unpersist() - } - } + if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || queryFilters.isEmpty) { + // scalastyle:off return + return Success(Option.empty) + // scalastyle:on return + } + + // Collect all index tables present in `.zindex` folder + val candidateIndexTables = + fs.listStatus(new Path(indexPath)) + .filter(_.isDirectory) + .map(_.getPath.getName) + .filter(f => completedCommits.contains(f)) + .sortBy(x => x) + + if (candidateIndexTables.isEmpty) { + // scalastyle:off return + return Success(Option.empty) + // scalastyle:on return } - allFiles -- candidateFiles + + val dataFrameOpt = try { + Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString)) + } catch { + case t: Throwable => + logError("Failed to read col-stats index; skipping", t) + None + } + + dataFrameOpt.map(df => { + val indexSchema = df.schema + val indexFilter = + queryFilters.map(createColumnStatsIndexFilterExpr(_, indexSchema)) + .reduce(And) + + logInfo(s"Index filter condition: $indexFilter") + + df.persist() + + val allIndexedFileNames = + df.select("file") + .collect() + .map(_.getString(0)) + .toSet + + val prunedCandidateFileNames = + df.where(new Column(indexFilter)) + .select("file") + .collect() + .map(_.getString(0)) + .toSet + + df.unpersist() + + // NOTE: Col-Stats Index isn't guaranteed to have complete set of statistics for every + // base-file: since it's bound to clustering, which could occur asynchronously + // at arbitrary point in time, and is not likely to be touching all of the base files. + // + // To close that gap, we manually compute the difference b/w all indexed (by col-stats-index) + // files and all outstanding base-files, and make sure that all base files not + // represented w/in the index are included in the output of this method + val notIndexedFileNames = + lookupFileNamesMissingFromIndex(allIndexedFileNames) + + prunedCandidateFileNames ++ notIndexedFileNames + }) + } + + private def lookupFileNamesMissingFromIndex(allIndexedFileNames: Set[String]) = { + val allBaseFileNames = allFiles.map(f => f.getPath.getName).toSet + allBaseFileNames -- allIndexedFileNames } /** @@ -205,21 +284,37 @@ case class HoodieFileIndex( */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - // try to load filterFiles from index - val filterFiles: Set[String] = if (enableDataSkipping()) { - filterFilesByDataSkippingIndex(dataFilters) - } else { - Set.empty - } - if (queryAsNonePartitionedTable) { // Read as Non-Partitioned table. - val candidateFiles = if (!filterFiles.isEmpty) { - allFiles.filterNot(fileStatus => filterFiles.contains(fileStatus.getPath.getName)) - } else { - allFiles + // Look up candidate files names in the col-stats index, if all of the following conditions are true + // - Data-skipping is enabled + // - Col-Stats Index is present + // - List of predicates (filters) is present + val candidateFilesNamesOpt: Option[Set[String]] = + lookupCandidateFilesInColStatsIndex(dataFilters) match { + case Success(opt) => opt + case Failure(e) => + if (e.isInstanceOf[AnalysisException]) { + logDebug("Failed to relay provided data filters to Z-index lookup", e) + } else { + logError("Failed to lookup candidate files in Z-index", e) + } + Option.empty } - logInfo(s"Total files : ${allFiles.size}," + - s" candidate files after data skipping: ${candidateFiles.size} " + - s" skipping percent ${if (allFiles.length != 0) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}") + + logDebug(s"Overlapping candidate files (from Z-index): ${candidateFilesNamesOpt.getOrElse(Set.empty)}") + + if (queryAsNonePartitionedTable) { + // Read as Non-Partitioned table + // Filter in candidate files based on the col-stats index lookup + val candidateFiles = + allFiles.filter(fileStatus => + // NOTE: This predicate is true when {@code Option} is empty + candidateFilesNamesOpt.forall(_.contains(fileStatus.getPath.getName)) + ) + + logInfo(s"Total files : ${allFiles.size}; " + + s"candidate files after data skipping: ${candidateFiles.size}; " + + s"skipping percent ${if (allFiles.nonEmpty) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}") + Seq(PartitionDirectory(InternalRow.empty, candidateFiles)) } else { // Prune the partition path by the partition filters @@ -228,25 +323,27 @@ case class HoodieFileIndex( var candidateFileSize = 0 val result = prunedPartitions.map { partition => - val baseFileStatuses = cachedAllInputFileSlices(partition).map(fileSlice => { - if (fileSlice.getBaseFile.isPresent) { - fileSlice.getBaseFile.get().getFileStatus - } else { - null - } - }).filterNot(_ == null) - val candidateFiles = if (!filterFiles.isEmpty) { - baseFileStatuses.filterNot(fileStatus => filterFiles.contains(fileStatus.getPath.getName)) - } else { - baseFileStatuses - } + val baseFileStatuses: Seq[FileStatus] = + cachedAllInputFileSlices(partition) + .map(fs => fs.getBaseFile.orElse(null)) + .filter(_ != null) + .map(_.getFileStatus) + + // Filter in candidate files based on the col-stats index lookup + val candidateFiles = + baseFileStatuses.filter(fs => + // NOTE: This predicate is true when {@code Option} is empty + candidateFilesNamesOpt.forall(_.contains(fs.getPath.getName))) + totalFileSize += baseFileStatuses.size candidateFileSize += candidateFiles.size PartitionDirectory(partition.values, candidateFiles) } - logInfo(s"Total files: ${totalFileSize}," + - s" Candidate files after data skipping : ${candidateFileSize} " + - s"skipping percent ${if (allFiles.length != 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}") + + logInfo(s"Total base files: ${totalFileSize}; " + + s"candidate files after data skipping : ${candidateFileSize}; " + + s"skipping percent ${if (allFiles.nonEmpty) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}") + result } } @@ -481,14 +578,10 @@ case class HoodieFileIndex( }.mkString("/") val pathWithPartitionName = new Path(basePath, partitionWithName) val partitionDataTypes = partitionSchema.fields.map(f => f.name -> f.dataType).toMap - val partitionValues = sparkParsePartitionUtil.parsePartition(pathWithPartitionName, + + sparkParsePartitionUtil.parsePartition(pathWithPartitionName, typeInference = false, Set(new Path(basePath)), partitionDataTypes, DateTimeUtils.getTimeZone(timeZoneId)) - - // Convert partitionValues to InternalRow - partitionValues.map(_.literals.map(_.value)) - .map(InternalRow.fromSeq) - .getOrElse(InternalRow.empty) } } PartitionRowPath(partitionRow, partitionPath) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 44d39facdea6..ffe2c9282969 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -28,13 +28,15 @@ import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.config.HoodieRealtimeConfig import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS import org.apache.spark.rdd.RDD -import org.apache.spark.sql.avro.{HoodieAvroSerializer, HooodieAvroDeserializer} +import org.apache.spark.sql.avro.{HoodieAvroSerializer, HoodieAvroDeserializer} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext} +import java.io.Closeable + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Try @@ -58,7 +60,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val mergeOnReadPartition = split.asInstanceOf[HoodieMergeOnReadPartition] - mergeOnReadPartition.split match { + val iter = mergeOnReadPartition.split match { case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty => read(dataFileOnlySplit.dataFile.get, requiredSchemaFileReader) case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => @@ -84,6 +86,12 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, s"spark partition Index: ${mergeOnReadPartition.index}" + s"merge type: ${mergeOnReadPartition.split.mergeType}") } + if (iter.isInstanceOf[Closeable]) { + // register a callback to close logScanner which will be executed on task completion. + // when tasks finished, this method will be called, and release resources. + Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.asInstanceOf[Closeable].close())) + } + iter } override protected def getPartitions: Array[Partition] = { @@ -112,16 +120,17 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def logFileIterator(split: HoodieMergeOnReadFileSplit, config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] { + new Iterator[InternalRow] with Closeable { private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) private val requiredFieldPosition = tableState.requiredStructSchema .map(f => tableAvroSchema.getField(f.name).pos()).toList private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) - private val deserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) + private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) - private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords + private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) + private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala private var recordToLoad: InternalRow = _ @@ -146,21 +155,32 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, override def next(): InternalRow = { recordToLoad } + + override def close(): Unit = { + if (logScanner != null) { + try { + logScanner.close() + } finally { + logScanner = null + } + } + } } private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit, baseFileIterator: Iterator[InternalRow], config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] { + new Iterator[InternalRow] with Closeable { private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) private val requiredFieldPosition = tableState.requiredStructSchema .map(f => tableAvroSchema.getField(f.name).pos()).toList private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) - private val deserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) + private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) - private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords + private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) + private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala private var recordToLoad: InternalRow = _ @@ -192,22 +212,33 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, override def next(): InternalRow = { recordToLoad } + + override def close(): Unit = { + if (logScanner != null) { + try { + logScanner.close() + } finally { + logScanner = null + } + } + } } private def payloadCombineFileIterator(split: HoodieMergeOnReadFileSplit, baseFileIterator: Iterator[InternalRow], config: Configuration): Iterator[InternalRow] = - new Iterator[InternalRow] { + new Iterator[InternalRow] with Closeable { private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) private val requiredFieldPosition = tableState.requiredStructSchema .map(f => tableAvroSchema.getField(f.name).pos()).toList private val serializer = HoodieAvroSerializer(tableState.tableStructSchema, tableAvroSchema, false) - private val requiredDeserializer = HooodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) + private val requiredDeserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) - private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords + private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config) + private val logRecords = logScanner.getRecords private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala private val keyToSkip = mutable.Set.empty[String] private val recordKeyPosition = if (recordKeyFieldOpt.isEmpty) HOODIE_RECORD_KEY_COL_POS else tableState.tableStructSchema.fieldIndex(recordKeyFieldOpt.get) @@ -276,6 +307,16 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, override def next(): InternalRow = recordToLoad + override def close(): Unit = { + if (logScanner != null) { + try { + logScanner.close() + } finally { + logScanner = null + } + } + } + private def createRowWithRequiredSchema(row: InternalRow): InternalRow = { val rowToReturn = new SpecificInternalRow(tableState.requiredStructSchema) val posIterator = requiredFieldPosition.iterator diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index ddbd7fc06a95..d35cd49a482f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -23,12 +23,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.DataSourceOptionsHelper.{allAlternatives, translateConfigurations} +import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} +import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, HoodieTimelineTimeZone, WriteOperationType} import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils} @@ -48,13 +48,12 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession} -import org.apache.spark.{SPARK_VERSION, SparkContext} -import java.util +import org.apache.spark.SparkContext + import java.util.Properties import scala.collection.JavaConversions._ import scala.collection.mutable -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ListBuffer object HoodieSparkSqlWriter { @@ -141,9 +140,10 @@ object HoodieSparkSqlWriter { .setPartitionFields(partitionColumns) .setPopulateMetaFields(populateMetaFields) .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD)) - .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME)) + .setKeyGeneratorClassProp(HoodieWriterUtils.getOriginKeyGenerator(parameters)) .setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING)) .setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING)) + .setCommitTimezone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE))) .initTable(sparkContext.hadoopConfiguration, path) tableConfig = tableMetaClient.getTableConfig } @@ -285,7 +285,7 @@ object HoodieSparkSqlWriter { } def generateSchemaWithoutPartitionColumns(partitionParam: String, schema: Schema): Schema = { - val fieldsToRemove = new util.ArrayList[String]() + val fieldsToRemove = new java.util.ArrayList[String]() partitionParam.split(",").map(partitionField => partitionField.trim) .filter(s => !s.isEmpty).map(field => fieldsToRemove.add(field)) HoodieAvroUtils.removeFields(schema, fieldsToRemove) @@ -394,6 +394,7 @@ object HoodieSparkSqlWriter { .setKeyGeneratorClassProp(keyGenProp) .setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING)) .setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING)) + .setCommitTimezone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE))) .initTable(sparkContext.hadoopConfiguration, path) } @@ -463,13 +464,13 @@ object HoodieSparkSqlWriter { } else { HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(df) } - if (SPARK_VERSION.startsWith("2.")) { + if (HoodieSparkUtils.isSpark2) { hoodieDF.write.format("org.apache.hudi.internal") .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) .options(params) .mode(SaveMode.Append) .save() - } else if (SPARK_VERSION.startsWith("3.")) { + } else if(HoodieSparkUtils.isSpark3) { hoodieDF.write.format("org.apache.hudi.spark3.internal") .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) .option(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key, hoodieDF.schema.toDDL) @@ -625,7 +626,7 @@ object HoodieSparkSqlWriter { kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX.key))) val commitSuccess = client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses, - common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))), + common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))), tableInstantInfo.commitActionType, writeResult.getPartitionToReplaceFileIds) @@ -639,7 +640,7 @@ object HoodieSparkSqlWriter { val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration()) val compactionInstant: common.util.Option[java.lang.String] = if (asyncCompactionEnabled) { - client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + client.scheduleCompaction(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap)))) } else { common.util.Option.empty() } @@ -649,7 +650,7 @@ object HoodieSparkSqlWriter { val asyncClusteringEnabled = isAsyncClusteringEnabled(client, parameters) val clusteringInstant: common.util.Option[java.lang.String] = if (asyncClusteringEnabled) { - client.scheduleClustering(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + client.scheduleClustering(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap)))) } else { common.util.Option.empty() } @@ -713,26 +714,10 @@ object HoodieSparkSqlWriter { } } - private def validateTableConfig(spark: SparkSession, params: Map[String, String], - tableConfig: HoodieTableConfig): Unit = { - val resolver = spark.sessionState.conf.resolver - val diffConfigs = StringBuilder.newBuilder - params.foreach { case (key, value) => - val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key) - if (null != existingValue && !resolver(existingValue, value)) { - diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n") - } - } - if (diffConfigs.nonEmpty) { - diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n") - throw new HoodieException(diffConfigs.toString.trim) - } - } - private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String], tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = { - val mergedParams = mutable.Map.empty ++ - DataSourceWriteOptions.translateSqlOptions(HoodieWriterUtils.parametersWithWriteDefaults(optParams)) + val translatedOptions = DataSourceWriteOptions.translateSqlOptions(optParams) + val mergedParams = mutable.Map.empty ++ HoodieWriterUtils.parametersWithWriteDefaults(translatedOptions) if (!mergedParams.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) && mergedParams.contains(KEYGENERATOR_CLASS_NAME.key)) { mergedParams(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = mergedParams(KEYGENERATOR_CLASS_NAME.key) @@ -745,16 +730,4 @@ object HoodieSparkSqlWriter { val params = mergedParams.toMap (params, HoodieWriterUtils.convertMapToHoodieConfig(params)) } - - private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieTableConfig, key: String): String = { - if (null == tableConfig) { - null - } else { - if (allAlternatives.contains(key)) { - tableConfig.getString(allAlternatives(key)) - } else { - tableConfig.getString(key) - } - } - } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala index 6e736d225a52..8d8ebfa7e54f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala @@ -16,18 +16,16 @@ */ package org.apache.hudi -import java.lang -import java.util.function.Function - import org.apache.hudi.async.{AsyncClusteringService, AsyncCompactService, SparkStreamingAsyncClusteringService, SparkStreamingAsyncCompactService} import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.model.HoodieRecordPayload -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.table.marker.MarkerType import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} -import org.apache.hudi.common.util.CompactionUtils -import org.apache.hudi.common.util.ClusteringUtils +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.util.{ClusteringUtils, CompactionUtils} +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieCorruptedDataException import org.apache.log4j.LogManager import org.apache.spark.api.java.JavaSparkContext @@ -35,8 +33,10 @@ import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} -import scala.util.{Failure, Success, Try} +import java.lang +import java.util.function.Function import scala.collection.JavaConversions._ +import scala.util.{Failure, Success, Try} class HoodieStreamingSink(sqlContext: SQLContext, options: Map[String, String], @@ -71,25 +71,29 @@ class HoodieStreamingSink(sqlContext: SQLContext, private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized { - if (isAsyncCompactorServiceShutdownAbnormally) { + if (isAsyncCompactorServiceShutdownAbnormally) { throw new IllegalStateException("Async Compactor shutdown unexpectedly") } - if (isAsyncClusteringServiceShutdownAbnormally) { + if (isAsyncClusteringServiceShutdownAbnormally) { log.error("Async clustering service shutdown unexpectedly") throw new IllegalStateException("Async clustering service shutdown unexpectedly") } + // Override to use direct markers. In Structured streaming, timeline server is closed after + // first micro-batch and subsequent micro-batches do not have timeline server running. + // Thus, we can't use timeline-server-based markers. + val updatedOptions = options.updated(HoodieWriteConfig.MARKERS_TYPE.key(), MarkerType.DIRECT.name()) retry(retryCnt, retryIntervalMs)( Try( HoodieSparkSqlWriter.write( - sqlContext, mode, options, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering)) + sqlContext, mode, updatedOptions, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering)) ) match { case Success((true, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) => log.info(s"Micro batch id=$batchId succeeded" + (commitOps.isPresent match { - case true => s" for commit=${commitOps.get()}" - case _ => s" with no new commits" - })) + case true => s" for commit=${commitOps.get()}" + case _ => s" with no new commits" + })) writeClient = Some(client) hoodieTableConfig = Some(tableConfig) if (compactionInstantOps.isPresent) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 0e3ede1fe3eb..282de54f6525 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -17,15 +17,19 @@ package org.apache.hudi +import java.util.Properties + +import org.apache.hudi.DataSourceOptionsHelper.allAlternatives import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE -import org.apache.hudi.common.config.{HoodieConfig, TypedProperties} +import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties} +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.exception.HoodieException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hudi.command.SqlKeyGenerator -import java.util.Properties import scala.collection.JavaConversions.mapAsJavaMap -import scala.collection.JavaConverters.{mapAsScalaMapConverter, _} -import scala.collection.JavaConverters.mapAsScalaMapConverter -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory +import scala.collection.JavaConverters._ /** * WriterUtils to assist in write path in Datasource and tests. @@ -43,41 +47,44 @@ object HoodieWriterUtils { * @return */ def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = { - Map(OPERATION.key -> OPERATION.defaultValue, - TABLE_TYPE.key -> TABLE_TYPE.defaultValue, - PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue, - PAYLOAD_CLASS_NAME.key -> PAYLOAD_CLASS_NAME.defaultValue, - RECORDKEY_FIELD.key -> RECORDKEY_FIELD.defaultValue, - PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue, - KEYGENERATOR_CLASS_NAME.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, - ENABLE.key -> ENABLE.defaultValue.toString, - COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue, - INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue, - STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue, - STREAMING_RETRY_INTERVAL_MS.key -> STREAMING_RETRY_INTERVAL_MS.defaultValue, - STREAMING_IGNORE_FAILED_BATCH.key -> STREAMING_IGNORE_FAILED_BATCH.defaultValue, - META_SYNC_CLIENT_TOOL_CLASS_NAME.key -> META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue, - HIVE_SYNC_ENABLED.key -> HIVE_SYNC_ENABLED.defaultValue, - META_SYNC_ENABLED.key -> META_SYNC_ENABLED.defaultValue, - HIVE_DATABASE.key -> HIVE_DATABASE.defaultValue, - HIVE_TABLE.key -> HIVE_TABLE.defaultValue, - HIVE_BASE_FILE_FORMAT.key -> HIVE_BASE_FILE_FORMAT.defaultValue, - HIVE_USER.key -> HIVE_USER.defaultValue, - HIVE_PASS.key -> HIVE_PASS.defaultValue, - HIVE_URL.key -> HIVE_URL.defaultValue, - HIVE_PARTITION_FIELDS.key -> HIVE_PARTITION_FIELDS.defaultValue, - HIVE_PARTITION_EXTRACTOR_CLASS.key -> HIVE_PARTITION_EXTRACTOR_CLASS.defaultValue, - HIVE_STYLE_PARTITIONING.key -> HIVE_STYLE_PARTITIONING.defaultValue, - HIVE_USE_JDBC.key -> HIVE_USE_JDBC.defaultValue, - HIVE_CREATE_MANAGED_TABLE.key() -> HIVE_CREATE_MANAGED_TABLE.defaultValue.toString, - HIVE_SYNC_AS_DATA_SOURCE_TABLE.key() -> HIVE_SYNC_AS_DATA_SOURCE_TABLE.defaultValue(), - ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue, - INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue, - ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue, - ENABLE_ROW_WRITER.key -> ENABLE_ROW_WRITER.defaultValue, - RECONCILE_SCHEMA.key -> RECONCILE_SCHEMA.defaultValue.toString, - DROP_PARTITION_COLUMNS.key -> DROP_PARTITION_COLUMNS.defaultValue - ) ++ DataSourceOptionsHelper.translateConfigurations(parameters) + val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala + val props = new Properties() + props.putAll(parameters) + val hoodieConfig: HoodieConfig = new HoodieConfig(props) + hoodieConfig.setDefaultValue(OPERATION) + hoodieConfig.setDefaultValue(TABLE_TYPE) + hoodieConfig.setDefaultValue(PRECOMBINE_FIELD) + hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME) + hoodieConfig.setDefaultValue(RECORDKEY_FIELD) + hoodieConfig.setDefaultValue(KEYGENERATOR_CLASS_NAME) + hoodieConfig.setDefaultValue(ENABLE) + hoodieConfig.setDefaultValue(COMMIT_METADATA_KEYPREFIX) + hoodieConfig.setDefaultValue(INSERT_DROP_DUPS) + hoodieConfig.setDefaultValue(STREAMING_RETRY_CNT) + hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS) + hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH) + hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME) + hoodieConfig.setDefaultValue(HIVE_SYNC_ENABLED) + hoodieConfig.setDefaultValue(META_SYNC_ENABLED) + hoodieConfig.setDefaultValue(HIVE_DATABASE) + hoodieConfig.setDefaultValue(HIVE_TABLE) + hoodieConfig.setDefaultValue(HIVE_BASE_FILE_FORMAT) + hoodieConfig.setDefaultValue(HIVE_USER) + hoodieConfig.setDefaultValue(HIVE_PASS) + hoodieConfig.setDefaultValue(HIVE_URL) + hoodieConfig.setDefaultValue(HIVE_PARTITION_FIELDS) + hoodieConfig.setDefaultValue(HIVE_PARTITION_EXTRACTOR_CLASS) + hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING) + hoodieConfig.setDefaultValue(HIVE_USE_JDBC) + hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE) + hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE) + hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) + hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE) + hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE) + hoodieConfig.setDefaultValue(ENABLE_ROW_WRITER) + hoodieConfig.setDefaultValue(RECONCILE_SCHEMA) + hoodieConfig.setDefaultValue(DROP_PARTITION_COLUMNS) + Map() ++ hoodieConfig.getProps.asScala ++ globalProps ++ DataSourceOptionsHelper.translateConfigurations(parameters) } def toProperties(params: Map[String, String]): TypedProperties = { @@ -102,4 +109,87 @@ object HoodieWriterUtils { properties.putAll(mapAsJavaMap(parameters)) new HoodieConfig(properties) } + + def getOriginKeyGenerator(parameters: Map[String, String]): String = { + val kg = parameters.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null) + if (classOf[SqlKeyGenerator].getCanonicalName == kg) { + parameters.getOrElse(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null) + } else { + kg + } + } + + /** + * Detects conflicts between new parameters and existing table configurations + */ + def validateTableConfig(spark: SparkSession, params: Map[String, String], + tableConfig: HoodieConfig): Unit = { + val resolver = spark.sessionState.conf.resolver + val diffConfigs = StringBuilder.newBuilder + params.foreach { case (key, value) => + val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key) + if (null != existingValue && !resolver(existingValue, value)) { + diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n") + } + } + + if (null != tableConfig) { + val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null) + val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS) + if (null != datasourceRecordKey && null != tableConfigRecordKey + && datasourceRecordKey != tableConfigRecordKey) { + diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n") + } + + val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null) + val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD) + if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey + && datasourcePreCombineKey != tableConfigPreCombineKey) { + diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n") + } + + val datasourceKeyGen = getOriginKeyGenerator(params) + val tableConfigKeyGen = tableConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME) + if (null != datasourceKeyGen && null != tableConfigKeyGen + && datasourceKeyGen != tableConfigKeyGen) { + diffConfigs.append(s"KeyGenerator:\t$datasourceKeyGen\t$tableConfigKeyGen\n") + } + } + + if (diffConfigs.nonEmpty) { + diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n") + throw new HoodieException(diffConfigs.toString.trim) + } + } + + private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieConfig, key: String): String = { + if (null == tableConfig) { + null + } else { + if (allAlternatives.contains(key)) { + tableConfig.getString(allAlternatives(key)) + } else { + tableConfig.getString(key) + } + } + } + + val sparkDatasourceConfigsToTableConfigsMap = Map( + TABLE_NAME -> HoodieTableConfig.NAME, + TABLE_TYPE -> HoodieTableConfig.TYPE, + PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELD, + PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS, + RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS, + PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME + ) + def mappingSparkDatasourceConfigsToTableConfigs(options: Map[String, String]): Map[String, String] = { + val includingTableConfigs = scala.collection.mutable.Map() ++ options + sparkDatasourceConfigsToTableConfigsMap.foreach(kv => { + if (options.containsKey(kv._1.key)) { + includingTableConfigs(kv._2.key) = options(kv._1.key) + includingTableConfigs.remove(kv._1.key) + } + }) + includingTableConfigs.toMap + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala index 958a15eeb7ad..19071080312b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -17,8 +17,9 @@ package org.apache.hudi -import java.util.stream.Collectors +import org.apache.avro.Schema +import java.util.stream.Collectors import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata, HoodieTableType} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} @@ -89,8 +90,13 @@ class IncrementalRelation(val sqlContext: SQLContext, } else { schemaResolver.getTableAvroSchemaWithoutMetadataFields() } - val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema) - StructType(skeletonSchema.fields ++ dataSchema.fields) + if (tableSchema.getType == Schema.Type.NULL) { + // if there is only one commit in the table and is an empty commit without schema, return empty RDD here + StructType(Nil) + } else { + val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema) + StructType(skeletonSchema.fields ++ dataSchema.fields) + } } private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS.key, @@ -99,86 +105,90 @@ class IncrementalRelation(val sqlContext: SQLContext, override def schema: StructType = usedSchema override def buildScan(): RDD[Row] = { - val regularFileIdToFullPath = mutable.HashMap[String, String]() - var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]() - - // create Replaced file group - val replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline - val replacedFile = replacedTimeline.getInstants.collect(Collectors.toList[HoodieInstant]).flatMap { instant => - val replaceMetadata = HoodieReplaceCommitMetadata. - fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata]) - replaceMetadata.getPartitionToReplaceFileIds.entrySet().flatMap { entry => - entry.getValue.map { e => - val fullPath = FSUtils.getPartitionPath(basePath, entry.getKey).toString - (e, fullPath) + if (usedSchema == StructType(Nil)) { + // if first commit in a table is an empty commit without schema, return empty RDD here + sqlContext.sparkContext.emptyRDD[Row] + } else { + val regularFileIdToFullPath = mutable.HashMap[String, String]() + var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]() + + // create Replaced file group + val replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline + val replacedFile = replacedTimeline.getInstants.collect(Collectors.toList[HoodieInstant]).flatMap { instant => + val replaceMetadata = HoodieReplaceCommitMetadata. + fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata]) + replaceMetadata.getPartitionToReplaceFileIds.entrySet().flatMap { entry => + entry.getValue.map { e => + val fullPath = FSUtils.getPartitionPath(basePath, entry.getKey).toString + (e, fullPath) + } + } + }.toMap + + for (commit <- commitsToReturn) { + val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit) + .get, classOf[HoodieCommitMetadata]) + + if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) { + metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) => + replacedFile.contains(k) && v.startsWith(replacedFile(k)) + } + } else { + regularFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) => + replacedFile.contains(k) && v.startsWith(replacedFile(k)) + } } } - }.toMap - for (commit <- commitsToReturn) { - val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit) - .get, classOf[HoodieCommitMetadata]) + if (metaBootstrapFileIdToFullPath.nonEmpty) { + // filer out meta bootstrap files that have had more commits since metadata bootstrap + metaBootstrapFileIdToFullPath = metaBootstrapFileIdToFullPath + .filterNot(fileIdFullPath => regularFileIdToFullPath.contains(fileIdFullPath._1)) + } - if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) { - metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) => - replacedFile.contains(k) && v.startsWith(replacedFile(k)) - } - } else { - regularFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) => - replacedFile.contains(k) && v.startsWith(replacedFile(k)) + val pathGlobPattern = optParams.getOrElse( + DataSourceReadOptions.INCR_PATH_GLOB.key, + DataSourceReadOptions.INCR_PATH_GLOB.defaultValue) + val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = { + if (!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)) { + val globMatcher = new GlobPattern("*" + pathGlobPattern) + (regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values, + metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values) + } else { + (regularFileIdToFullPath.values, metaBootstrapFileIdToFullPath.values) } } - } - - if (metaBootstrapFileIdToFullPath.nonEmpty) { - // filer out meta bootstrap files that have had more commits since metadata bootstrap - metaBootstrapFileIdToFullPath = metaBootstrapFileIdToFullPath - .filterNot(fileIdFullPath => regularFileIdToFullPath.contains(fileIdFullPath._1)) - } - - val pathGlobPattern = optParams.getOrElse( - DataSourceReadOptions.INCR_PATH_GLOB.key, - DataSourceReadOptions.INCR_PATH_GLOB.defaultValue) - val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = { - if(!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)) { - val globMatcher = new GlobPattern("*" + pathGlobPattern) - (regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values, - metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values) + // unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view + // will filter out all the files incorrectly. + sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class") + val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) + if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) { + sqlContext.sparkContext.emptyRDD[Row] } else { - (regularFileIdToFullPath.values, metaBootstrapFileIdToFullPath.values) - } - } - // unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view - // will filter out all the files incorrectly. - sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class") - val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) - if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) { - sqlContext.sparkContext.emptyRDD[Row] - } else { - log.info("Additional Filters to be applied to incremental source are :" + filters) + log.info("Additional Filters to be applied to incremental source are :" + filters) - var df: DataFrame = sqlContext.createDataFrame(sqlContext.sparkContext.emptyRDD[Row], usedSchema) + var df: DataFrame = sqlContext.createDataFrame(sqlContext.sparkContext.emptyRDD[Row], usedSchema) - if (metaBootstrapFileIdToFullPath.nonEmpty) { - df = sqlContext.sparkSession.read - .format("hudi") - .schema(usedSchema) - .option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(",")) - .load() - } + if (metaBootstrapFileIdToFullPath.nonEmpty) { + df = sqlContext.sparkSession.read + .format("hudi") + .schema(usedSchema) + .option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(",")) + .load() + } - if (regularFileIdToFullPath.nonEmpty) - { - df = df.union(sqlContext.read.options(sOpts) - .schema(usedSchema) - .parquet(filteredRegularFullPaths.toList: _*) - .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, - commitsToReturn.head.getTimestamp)) - .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, - commitsToReturn.last.getTimestamp))) - } + if (regularFileIdToFullPath.nonEmpty) { + df = df.union(sqlContext.read.options(sOpts) + .schema(usedSchema) + .parquet(filteredRegularFullPaths.toList: _*) + .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + commitsToReturn.head.getTimestamp)) + .filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, + commitsToReturn.last.getTimestamp))) + } - filters.foldLeft(df)((e, f) => e.filter(f)).rdd + filters.foldLeft(df)((e, f) => e.filter(f)).rdd + } } } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 1d140304cb4c..a1d857c948c1 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -151,8 +151,9 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, // Load files from the global paths if it has defined to be compatible with the original mode val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get) val fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline.getCommitsTimeline - .filterCompletedInstants, inMemoryFileIndex.allFiles().toArray) + // file-slice after pending compaction-requested instant-time is also considered valid + metaClient.getCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants, + inMemoryFileIndex.allFiles().toArray) val partitionPaths = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala new file mode 100644 index 000000000000..1678dc05da4f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala @@ -0,0 +1,48 @@ +/* + * 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 org.apache.spark.sql.avro + +import org.apache.avro.Schema + +import org.apache.hudi.HoodieSparkUtils + +import org.apache.spark.sql.types.DataType + +/** + * This is to be compatible with the type returned by Spark 3.1 + * and other spark versions for AvroDeserializer + */ +case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { + + private val avroDeserializer = if (HoodieSparkUtils.isSpark3_2) { + // SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments. + // So use the reflection to get AvroDeserializer instance. + val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType], classOf[String]) + constructor.newInstance(rootAvroType, rootCatalystType, "EXCEPTION") + } else { + val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType]) + constructor.newInstance(rootAvroType, rootCatalystType) + } + + def deserializeData(data: Any): Any = { + avroDeserializer.deserialize(data) match { + case Some(r) => r // As of spark 3.1, this will return data wrapped with Option, so we fetch the data. + case o => o // for other spark version, return the data directly. + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala new file mode 100644 index 000000000000..21a60087f0f9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -0,0 +1,308 @@ +/* + * 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 org.apache.spark.sql.catalyst.catalog + +import org.apache.hudi.HoodieWriterUtils._ +import org.apache.hudi.common.config.DFSPropertiesConfiguration +import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.util.ValidationUtils +import org.apache.hudi.keygen.ComplexKeyGenerator +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.avro.SchemaConverters +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} +import org.apache.spark.sql.hudi.HoodieSqlUtils._ +import org.apache.spark.sql.types.{StructField, StructType} + +import java.util.{Locale, Properties} + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * A wrapper of hoodie CatalogTable instance and hoodie Table. + */ +class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) extends Logging { + + assert(table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi", "It's not a Hudi table") + + private val hadoopConf = spark.sessionState.newHadoopConf + + /** + * database.table in catalog + */ + val catalogTableName = table.qualifiedName + + /** + * properties defined in catalog. + */ + val catalogProperties: Map[String, String] = table.storage.properties ++ table.properties + + /** + * hoodie table's location. + * if create managed hoodie table, use `catalog.defaultTablePath`. + */ + val tableLocation: String = HoodieSqlUtils.getTableLocation(table, spark) + + /** + * A flag to whether the hoodie table exists. + */ + val hoodieTableExists: Boolean = tableExistsInPath(tableLocation, hadoopConf) + + /** + * Meta Client. + */ + lazy val metaClient: HoodieTableMetaClient = HoodieTableMetaClient.builder() + .setBasePath(tableLocation) + .setConf(hadoopConf) + .build() + + /** + * Hoodie Table Config + */ + lazy val tableConfig: HoodieTableConfig = metaClient.getTableConfig + + /** + * the name of table + */ + lazy val tableName: String = tableConfig.getTableName + + /** + * The name of type of table + */ + lazy val tableType: HoodieTableType = tableConfig.getTableType + + /** + * The type of table + */ + lazy val tableTypeName: String = tableType.name() + + /** + * Recored Field List(Primary Key List) + */ + lazy val primaryKeys: Array[String] = tableConfig.getRecordKeyFields.orElse(Array.empty) + + /** + * PreCombine Field + */ + lazy val preCombineKey: Option[String] = Option(tableConfig.getPreCombineField) + + /** + * Paritition Fields + */ + lazy val partitionFields: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty) + + /** + * The schema of table. + * Make StructField nullable. + */ + lazy val tableSchema: StructType = { + val originSchema = getTableSqlSchema(metaClient, includeMetadataFields = true).getOrElse(table.schema) + StructType(originSchema.map(_.copy(nullable = true))) + } + + /** + * The schema without hoodie meta fields + */ + lazy val tableSchemaWithoutMetaFields: StructType = HoodieSqlUtils.removeMetaFields(tableSchema) + + /** + * The schema of data fields + */ + lazy val dataSchema: StructType = { + StructType(tableSchema.filterNot(f => partitionFields.contains(f.name))) + } + + /** + * The schema of data fields not including hoodie meta fields + */ + lazy val dataSchemaWithoutMetaFields: StructType = HoodieSqlUtils.removeMetaFields(dataSchema) + + /** + * The schema of partition fields + */ + lazy val partitionSchema: StructType = StructType(tableSchema.filter(f => partitionFields.contains(f.name))) + + /** + * All the partition paths + */ + def getAllPartitionPaths: Seq[String] = HoodieSqlUtils.getAllPartitionPaths(spark, table) + + /** + * Check if table is a partitioned table + */ + def isPartitionedTable: Boolean = table.partitionColumnNames.nonEmpty + + /** + * init hoodie table for create table (as select) + */ + def initHoodieTable(): Unit = { + logInfo(s"Init hoodie.properties for ${table.identifier.unquotedString}") + val (finalSchema, tableConfigs) = parseSchemaAndConfigs() + + // Save all the table config to the hoodie.properties. + val properties = new Properties() + properties.putAll(tableConfigs.asJava) + + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(properties) + .setTableName(table.identifier.table) + .setTableCreateSchema(SchemaConverters.toAvroType(finalSchema).toString()) + .setPartitionFields(table.partitionColumnNames.mkString(",")) + .initTable(hadoopConf, tableLocation) + } + + /** + * @return schema, table parameters in which all parameters aren't sql-styled. + */ + private def parseSchemaAndConfigs(): (StructType, Map[String, String]) = { + val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap + val globalTableConfigs = mappingSparkDatasourceConfigsToTableConfigs(globalProps) + val globalSqlOptions = HoodieOptionConfig.mappingTableConfigToSqlOption(globalTableConfigs) + + val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(globalSqlOptions ++ catalogProperties) + + // get final schema and parameters + val (finalSchema, tableConfigs) = (table.tableType, hoodieTableExists) match { + case (CatalogTableType.EXTERNAL, true) => + val existingTableConfig = tableConfig.getProps.asScala.toMap + val currentTableConfig = globalTableConfigs ++ existingTableConfig + val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(catalogProperties) + validateTableConfig(spark, catalogTableProps, convertMapToHoodieConfig(existingTableConfig)) + + val options = extraTableConfig(spark, hoodieTableExists, currentTableConfig) ++ + HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) ++ currentTableConfig + + ValidationUtils.checkArgument(tableSchema.nonEmpty || table.schema.nonEmpty, + s"Missing schema for Create Table: $catalogTableName") + val schema = if (tableSchema.nonEmpty) { + tableSchema + } else { + addMetaFields(table.schema) + } + + (schema, options) + + case (_, false) => + ValidationUtils.checkArgument(table.schema.nonEmpty, + s"Missing schema for Create Table: $catalogTableName") + val schema = table.schema + val options = extraTableConfig(spark, isTableExists = false, globalTableConfigs) ++ + HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) + (addMetaFields(schema), options) + + case (CatalogTableType.MANAGED, true) => + throw new AnalysisException(s"Can not create the managed table('$catalogTableName')" + + s". The associated location('$tableLocation') already exists.") + } + HoodieOptionConfig.validateTable(spark, finalSchema, + HoodieOptionConfig.mappingTableConfigToSqlOption(tableConfigs)) + + val resolver = spark.sessionState.conf.resolver + val dataSchema = finalSchema.filterNot { f => + table.partitionColumnNames.exists(resolver(_, f.name)) + } + verifyDataSchema(table.identifier, table.tableType, dataSchema) + + (finalSchema, tableConfigs) + } + + private def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean, + originTableConfig: Map[String, String] = Map.empty): Map[String, String] = { + val extraConfig = mutable.Map.empty[String, String] + if (isTableExists) { + val allPartitionPaths = getAllPartitionPaths + if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) { + extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = + originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) + } else { + extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = + String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table)) + } + if (originTableConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)) { + extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = + originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) + } else { + extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = + String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table)) + } + } else { + extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true" + extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue() + } + + if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) { + extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = + HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator( + originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) + } else { + extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[ComplexKeyGenerator].getCanonicalName + } + extraConfig.toMap + } + + // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema + private def verifyDataSchema(tableIdentifier: TableIdentifier, tableType: CatalogTableType, + dataSchema: Seq[StructField]): Unit = { + if (tableType != CatalogTableType.VIEW) { + val invalidChars = Seq(",", ":", ";") + def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f => + f.dataType match { + case st: StructType => verifyNestedColumnNames(st) + case _ if invalidChars.exists(f.name.contains) => + val invalidCharsString = invalidChars.map(c => s"'$c'").mkString(", ") + val errMsg = "Cannot create a table having a nested column whose name contains " + + s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableIdentifier; " + + s"Column: ${f.name}" + throw new AnalysisException(errMsg) + case _ => + } + } + + dataSchema.foreach { f => + f.dataType match { + // Checks top-level column names + case _ if f.name.contains(",") => + throw new AnalysisException("Cannot create a table having a column whose name " + + s"contains commas in Hive metastore. Table: $tableIdentifier; Column: ${f.name}") + // Checks nested column names + case st: StructType => + verifyNestedColumnNames(st) + case _ => + } + } + } + } +} + +object HoodieCatalogTable { + + def apply(sparkSession: SparkSession, tableIdentifier: TableIdentifier): HoodieCatalogTable = { + val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier) + HoodieCatalogTable(sparkSession, catalogTable) + } + + def apply(sparkSession: SparkSession, catalogTable: CatalogTable): HoodieCatalogTable = { + new HoodieCatalogTable(sparkSession, catalogTable) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Compaction.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Compaction.scala index 8d4ef7a016d8..f445e7c8a0da 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Compaction.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Compaction.scala @@ -22,17 +22,37 @@ import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.Compactio case class CompactionTable(table: LogicalPlan, operation: CompactionOperation, instantTimestamp: Option[Long]) extends Command { override def children: Seq[LogicalPlan] = Seq(table) + + def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionTable = { + copy(table = newChildren.head) + } } case class CompactionPath(path: String, operation: CompactionOperation, instantTimestamp: Option[Long]) - extends Command + extends Command { + override def children: Seq[LogicalPlan] = Seq.empty + + def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionPath = { + this + } +} case class CompactionShowOnTable(table: LogicalPlan, limit: Int = 20) extends Command { override def children: Seq[LogicalPlan] = Seq(table) + + def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionShowOnTable = { + copy(table = newChildren.head) + } } -case class CompactionShowOnPath(path: String, limit: Int = 20) extends Command +case class CompactionShowOnPath(path: String, limit: Int = 20) extends Command { + override def children: Seq[LogicalPlan] = Seq.empty + + def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionShowOnPath = { + this + } +} object CompactionOperation extends Enumeration { type CompactionOperation = Value diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HooodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala similarity index 57% rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HooodieAvroDeserializer.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala index ba911a7b3075..bde1ba29e8b6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/avro/HooodieAvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/catalyst/trees/HoodieLeafLike.scala @@ -15,22 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql.avro - -import org.apache.avro.Schema -import org.apache.spark.sql.types.DataType +package org.apache.spark.sql.catalyst.trees /** - * This is to be compatible with the type returned by Spark 3.1 - * and other spark versions for AvroDeserializer + * Similar to `LeafLike` in Spark3.2. */ -case class HooodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) - extends AvroDeserializer(rootAvroType, rootCatalystType) { +trait HoodieLeafLike[T <: TreeNode[T]] { self: TreeNode[T] => + + override final def children: Seq[T] = Nil + + override final def mapChildren(f: T => T): T = this.asInstanceOf[T] - def deserializeData(data: Any): Any = { - super.deserialize(data) match { - case Some(r) => r // spark 3.1 return type is Option, we fetch the data. - case o => o // for other spark version, return the data directly. - } - } + final def withNewChildrenInternal(newChildren: IndexedSeq[T]): T = this.asInstanceOf[T] } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala index 45a7aec142d5..e5d6f525bac4 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala @@ -19,9 +19,11 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor} +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, StartsWith} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -29,148 +31,230 @@ import org.apache.spark.sql.functions.col import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters._ -object DataSkippingUtils { +object DataSkippingUtils extends Logging { /** - * create z_index filter and push those filters to index table to filter all candidate scan files. - * @param condition origin filter from query. - * @param indexSchema schema from index table. - * @return filters for index table. - */ - def createZindexFilter(condition: Expression, indexSchema: StructType): Expression = { - def buildExpressionInternal(colName: Seq[String], statisticValue: String): Expression = { - val appendColName = UnresolvedAttribute(colName).name + statisticValue - col(appendColName).expr + * Translates provided {@link filterExpr} into corresponding filter-expression for column-stats index index table + * to filter out candidate files that would hold records matching the original filter + * + * @param sourceFilterExpr source table's query's filter expression + * @param indexSchema index table schema + * @return filter for column-stats index's table + */ + def createColumnStatsIndexFilterExpr(sourceFilterExpr: Expression, indexSchema: StructType): Expression = { + // Try to transform original Source Table's filter expression into + // Column-Stats Index filter expression + tryComposeIndexFilterExpr(sourceFilterExpr, indexSchema) match { + case Some(e) => e + // NOTE: In case we can't transform source filter expression, we fallback + // to {@code TrueLiteral}, to essentially avoid pruning any indexed files from scanning + case None => TrueLiteral } + } - def reWriteCondition(colName: Seq[String], conditionExpress: Expression): Expression = { - val appendColName = UnresolvedAttribute(colName).name + "_minValue" - if (indexSchema.exists(p => p.name == appendColName)) { - conditionExpress - } else { - Literal.TrueLiteral - } - } + private def tryComposeIndexFilterExpr(sourceExpr: Expression, indexSchema: StructType): Option[Expression] = { + def minValue(colName: String) = col(getMinColumnNameFor(colName)).expr + def maxValue(colName: String) = col(getMaxColumnNameFor(colName)).expr + def numNulls(colName: String) = col(getNumNullsColumnNameFor(colName)).expr + + def colContainsValuesEqualToLiteral(colName: String, value: Literal): Expression = + // Only case when column C contains value V is when min(C) <= V <= max(c) + And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value)) - val minValue = (colName: Seq[String]) => buildExpressionInternal(colName, "_minValue") - val maxValue = (colName: Seq[String]) => buildExpressionInternal(colName, "_maxValue") - val num_nulls = (colName: Seq[String]) => buildExpressionInternal(colName, "_num_nulls") + def colContainsOnlyValuesEqualToLiteral(colName: String, value: Literal) = + // Only case when column C contains _only_ value V is when min(C) = V AND max(c) = V + And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value)) - condition match { - // query filter "colA = b" convert it to "colA_minValue <= b and colA_maxValue >= b" for index table + sourceExpr match { + // Filter "colA = b" + // Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup case EqualTo(attribute: AttributeReference, value: Literal) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value))) - // query filter "b = colA" convert it to "colA_minValue <= b and colA_maxValue >= b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => colContainsValuesEqualToLiteral(colName, value)) + + // Filter "b = colA" + // Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup case EqualTo(value: Literal, attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value))) - // query filter "colA = null" convert it to "colA_num_nulls = null" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => colContainsValuesEqualToLiteral(colName, value)) + + // Filter "colA != b" + // Translates to "NOT(colA_minValue = b AND colA_maxValue = b)" + // NOTE: This is NOT an inversion of `colA = b` + case Not(EqualTo(attribute: AttributeReference, value: Literal)) => + getTargetIndexedColName(attribute, indexSchema) + .map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value))) + + // Filter "b != colA" + // Translates to "NOT(colA_minValue = b AND colA_maxValue = b)" + // NOTE: This is NOT an inversion of `colA = b` + case Not(EqualTo(value: Literal, attribute: AttributeReference)) => + getTargetIndexedColName(attribute, indexSchema) + .map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value))) + + // Filter "colA = null" + // Translates to "colA_num_nulls = null" for index lookup case equalNullSafe @ EqualNullSafe(_: AttributeReference, _ @ Literal(null, _)) => - val colName = getTargetColNameParts(equalNullSafe.left) - reWriteCondition(colName, EqualTo(num_nulls(colName), equalNullSafe.right)) - // query filter "colA < b" convert it to "colA_minValue < b" for index table + getTargetIndexedColName(equalNullSafe.left, indexSchema) + .map(colName => EqualTo(numNulls(colName), equalNullSafe.right)) + + // Filter "colA < b" + // Translates to "colA_minValue < b" for index lookup case LessThan(attribute: AttributeReference, value: Literal) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName,LessThan(minValue(colName), value)) - // query filter "b < colA" convert it to "colA_maxValue > b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => LessThan(minValue(colName), value)) + + // Filter "b > colA" + // Translates to "b > colA_minValue" for index lookup + case GreaterThan(value: Literal, attribute: AttributeReference) => + getTargetIndexedColName(attribute, indexSchema) + .map(colName => LessThan(minValue(colName), value)) + + // Filter "b < colA" + // Translates to "b < colA_maxValue" for index lookup case LessThan(value: Literal, attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, GreaterThan(maxValue(colName), value)) - // query filter "colA > b" convert it to "colA_maxValue > b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => GreaterThan(maxValue(colName), value)) + + // Filter "colA > b" + // Translates to "colA_maxValue > b" for index lookup case GreaterThan(attribute: AttributeReference, value: Literal) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, GreaterThan(maxValue(colName), value)) - // query filter "b > colA" convert it to "colA_minValue < b" for index table - case GreaterThan(value: Literal, attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, LessThan(minValue(colName), value)) - // query filter "colA <= b" convert it to "colA_minValue <= b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => GreaterThan(maxValue(colName), value)) + + // Filter "colA <= b" + // Translates to "colA_minValue <= b" for index lookup case LessThanOrEqual(attribute: AttributeReference, value: Literal) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, LessThanOrEqual(minValue(colName), value)) - // query filter "b <= colA" convert it to "colA_maxValue >= b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => LessThanOrEqual(minValue(colName), value)) + + // Filter "b >= colA" + // Translates to "b >= colA_minValue" for index lookup + case GreaterThanOrEqual(value: Literal, attribute: AttributeReference) => + getTargetIndexedColName(attribute, indexSchema) + .map(colName => LessThanOrEqual(minValue(colName), value)) + + // Filter "b <= colA" + // Translates to "b <= colA_maxValue" for index lookup case LessThanOrEqual(value: Literal, attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, GreaterThanOrEqual(maxValue(colName), value)) - // query filter "colA >= b" convert it to "colA_maxValue >= b" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => GreaterThanOrEqual(maxValue(colName), value)) + + // Filter "colA >= b" + // Translates to "colA_maxValue >= b" for index lookup case GreaterThanOrEqual(attribute: AttributeReference, right: Literal) => - val colName = getTargetColNameParts(attribute) - GreaterThanOrEqual(maxValue(colName), right) - // query filter "b >= colA" convert it to "colA_minValue <= b" for index table - case GreaterThanOrEqual(value: Literal, attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, LessThanOrEqual(minValue(colName), value)) - // query filter "colA is null" convert it to "colA_num_nulls > 0" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => GreaterThanOrEqual(maxValue(colName), right)) + + // Filter "colA is null" + // Translates to "colA_num_nulls > 0" for index lookup case IsNull(attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, GreaterThan(num_nulls(colName), Literal(0))) - // query filter "colA is not null" convert it to "colA_num_nulls = 0" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => GreaterThan(numNulls(colName), Literal(0))) + + // Filter "colA is not null" + // Translates to "colA_num_nulls = 0" for index lookup case IsNotNull(attribute: AttributeReference) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, EqualTo(num_nulls(colName), Literal(0))) - // query filter "colA in (a,b)" convert it to " (colA_minValue <= a and colA_maxValue >= a) or (colA_minValue <= b and colA_maxValue >= b) " for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => EqualTo(numNulls(colName), Literal(0))) + + // Filter "colA in (a, b, ...)" + // Translates to "(colA_minValue <= a AND colA_maxValue >= a) OR (colA_minValue <= b AND colA_maxValue >= b)" for index lookup + // NOTE: This is equivalent to "colA = a OR colA = b OR ..." case In(attribute: AttributeReference, list: Seq[Literal]) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, list.map { lit => - And(LessThanOrEqual(minValue(colName), lit), GreaterThanOrEqual(maxValue(colName), lit)) - }.reduce(Or)) - // query filter "colA like xxx" convert it to " (colA_minValue <= xxx and colA_maxValue >= xxx) or (colA_min start with xxx or colA_max start with xxx) " for index table - case StartsWith(attribute, v @ Literal(_: UTF8String, _)) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, Or(And(LessThanOrEqual(minValue(colName), v), GreaterThanOrEqual(maxValue(colName), v)) , - Or(StartsWith(minValue(colName), v), StartsWith(maxValue(colName), v)))) - // query filter "colA not in (a, b)" convert it to " (not( colA_minValue = a and colA_maxValue = a)) and (not( colA_minValue = b and colA_maxValue = b)) " for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => + list.map { lit => colContainsValuesEqualToLiteral(colName, lit) }.reduce(Or) + ) + + // Filter "colA not in (a, b, ...)" + // Translates to "NOT((colA_minValue = a AND colA_maxValue = a) OR (colA_minValue = b AND colA_maxValue = b))" for index lookup + // NOTE: This is NOT an inversion of `in (a, b, ...)` expr, this is equivalent to "colA != a AND colA != b AND ..." case Not(In(attribute: AttributeReference, list: Seq[Literal])) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, list.map { lit => - Not(And(EqualTo(minValue(colName), lit), EqualTo(maxValue(colName), lit))) - }.reduce(And)) - // query filter "colA != b" convert it to "not ( colA_minValue = b and colA_maxValue = b )" for index table - case Not(EqualTo(attribute: AttributeReference, value: Literal)) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, Not(And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value)))) - // query filter "b != colA" convert it to "not ( colA_minValue = b and colA_maxValue = b )" for index table - case Not(EqualTo(value: Literal, attribute: AttributeReference)) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, Not(And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value)))) - // query filter "colA not like xxxx" convert it to "not ( colA_minValue startWith xxx and colA_maxValue startWith xxx)" for index table + getTargetIndexedColName(attribute, indexSchema) + .map(colName => + Not( + list.map { lit => colContainsOnlyValuesEqualToLiteral(colName, lit) }.reduce(Or) + ) + ) + + // Filter "colA like 'xxx%'" + // Translates to "colA_minValue <= xxx AND colA_maxValue >= xxx" for index lookup + // NOTE: That this operator only matches string prefixes, and this is + // essentially equivalent to "colA = b" expression + case StartsWith(attribute, v @ Literal(_: UTF8String, _)) => + getTargetIndexedColName(attribute, indexSchema) + .map(colName => colContainsValuesEqualToLiteral(colName, v)) + + // Filter "colA not like 'xxx%'" + // Translates to "NOT(colA_minValue like 'xxx%' AND colA_maxValue like 'xxx%')" for index lookup + // NOTE: This is NOT an inversion of "colA like xxx" case Not(StartsWith(attribute, value @ Literal(_: UTF8String, _))) => - val colName = getTargetColNameParts(attribute) - reWriteCondition(colName, Not(And(StartsWith(minValue(colName), value), StartsWith(maxValue(colName), value)))) + getTargetIndexedColName(attribute, indexSchema) + .map(colName => + Not(And(StartsWith(minValue(colName), value), StartsWith(maxValue(colName), value))) + ) + case or: Or => - val resLeft = createZindexFilter(or.left, indexSchema) - val resRight = createZindexFilter(or.right, indexSchema) - Or(resLeft, resRight) + val resLeft = createColumnStatsIndexFilterExpr(or.left, indexSchema) + val resRight = createColumnStatsIndexFilterExpr(or.right, indexSchema) + + Option(Or(resLeft, resRight)) case and: And => - val resLeft = createZindexFilter(and.left, indexSchema) - val resRight = createZindexFilter(and.right, indexSchema) - And(resLeft, resRight) + val resLeft = createColumnStatsIndexFilterExpr(and.left, indexSchema) + val resRight = createColumnStatsIndexFilterExpr(and.right, indexSchema) + + Option(And(resLeft, resRight)) + + // + // Pushing Logical NOT inside the AND/OR expressions + // NOTE: This is required to make sure we're properly handling negations in + // cases like {@code NOT(colA = 0)}, {@code NOT(colA in (a, b, ...)} + // + + case Not(And(left: Expression, right: Expression)) => + Option(createColumnStatsIndexFilterExpr(Or(Not(left), Not(right)), indexSchema)) - case expr: Expression => - Literal.TrueLiteral + case Not(Or(left: Expression, right: Expression)) => + Option(createColumnStatsIndexFilterExpr(And(Not(left), Not(right)), indexSchema)) + + case _: Expression => None } } - /** - * Extracts name from a resolved expression referring to a nested or non-nested column. - */ - def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = { + private def checkColIsIndexed(colName: String, indexSchema: StructType): Boolean = { + Set.apply( + getMinColumnNameFor(colName), + getMaxColumnNameFor(colName), + getNumNullsColumnNameFor(colName) + ) + .forall(stat => indexSchema.exists(_.name == stat)) + } + + private def getTargetIndexedColName(resolvedExpr: Expression, indexSchema: StructType): Option[String] = { + val colName = UnresolvedAttribute(getTargetColNameParts(resolvedExpr)).name + + // Verify that the column is indexed + if (checkColIsIndexed(colName, indexSchema)) { + Option.apply(colName) + } else { + None + } + } + + private def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = { resolvedTargetCol match { case attr: Attribute => Seq(attr.name) - case Alias(c, _) => getTargetColNameParts(c) - case GetStructField(c, _, Some(name)) => getTargetColNameParts(c) :+ name - case ex: ExtractValue => throw new AnalysisException(s"convert reference to name failed, Updating nested fields is only supported for StructType: ${ex}.") - case other => throw new AnalysisException(s"convert reference to name failed, Found unsupported expression ${other}") } @@ -179,7 +263,7 @@ object DataSkippingUtils { def getIndexFiles(conf: Configuration, indexPath: String): Seq[FileStatus] = { val basePath = new Path(indexPath) basePath.getFileSystem(conf) - .listStatus(basePath).filterNot(f => f.getPath.getName.endsWith(".parquet")) + .listStatus(basePath).filter(f => f.getPath.getName.endsWith(".parquet")) } /** diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 963035cb638d..bc9f14978c29 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -18,8 +18,11 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions -import org.apache.hudi.common.model.DefaultHoodieRecordPayload +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType} import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.util.ValidationUtils +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType /** @@ -39,26 +42,27 @@ object HoodieOptionConfig { val SQL_VALUE_TABLE_TYPE_MOR = "mor" - val SQL_KEY_TABLE_PRIMARY_KEY: HoodieOption[String] = buildConf() + val SQL_KEY_TABLE_PRIMARY_KEY: HoodieSQLOption[String] = buildConf() .withSqlKey("primaryKey") .withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD.key) .withTableConfigKey(HoodieTableConfig.RECORDKEY_FIELDS.key) + .defaultValue(DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue()) .build() - val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf() + val SQL_KEY_TABLE_TYPE: HoodieSQLOption[String] = buildConf() .withSqlKey("type") .withHoodieKey(DataSourceWriteOptions.TABLE_TYPE.key) .withTableConfigKey(HoodieTableConfig.TYPE.key) .defaultValue(SQL_VALUE_TABLE_TYPE_COW) .build() - val SQL_KEY_PRECOMBINE_FIELD: HoodieOption[String] = buildConf() + val SQL_KEY_PRECOMBINE_FIELD: HoodieSQLOption[String] = buildConf() .withSqlKey("preCombineField") .withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD.key) .withTableConfigKey(HoodieTableConfig.PRECOMBINE_FIELD.key) .build() - val SQL_PAYLOAD_CLASS: HoodieOption[String] = buildConf() + val SQL_PAYLOAD_CLASS: HoodieSQLOption[String] = buildConf() .withSqlKey("payloadClass") .withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key) .withTableConfigKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key) @@ -70,8 +74,8 @@ object HoodieOptionConfig { */ private lazy val keyMapping: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields - .filter(f => f.getType == classOf[HoodieOption[_]]) - .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieOption[_]]}) + .filter(f => f.getType == classOf[HoodieSQLOption[_]]) + .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) .map(option => option.sqlKeyName -> option.hoodieKeyName) .toMap } @@ -82,8 +86,8 @@ object HoodieOptionConfig { */ private lazy val keyTableConfigMapping: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields - .filter(f => f.getType == classOf[HoodieOption[_]]) - .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieOption[_]]}) + .filter(f => f.getType == classOf[HoodieSQLOption[_]]) + .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) .filter(_.tableConfigKey.isDefined) .map(option => option.sqlKeyName -> option.tableConfigKey.get) .toMap @@ -102,6 +106,8 @@ object HoodieOptionConfig { private lazy val reverseValueMapping = valueMapping.map(f => f._2 -> f._1) + def withDefaultSqlOptions(options: Map[String, String]): Map[String, String] = defaultSqlOptions ++ options + /** * Mapping the sql's short name key/value in the options to the hoodie's config key/value. * @param options @@ -119,14 +125,13 @@ object HoodieOptionConfig { * @return */ def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = { - defaultTableConfig ++ - options.map { case (k, v) => - if (keyTableConfigMapping.contains(k)) { - keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v) - } else { - k -> v - } + options.map { case (k, v) => + if (keyTableConfigMapping.contains(k)) { + keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v) + } else { + k -> v } + } } /** @@ -136,13 +141,12 @@ object HoodieOptionConfig { options.map(kv => tableConfigKeyToSqlKey.getOrElse(kv._1, kv._1) -> reverseValueMapping.getOrElse(kv._2, kv._2)) } - private lazy val defaultTableConfig: Map[String, String] = { + val defaultSqlOptions: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields - .filter(f => f.getType == classOf[HoodieOption[_]]) - .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieOption[_]]}) + .filter(f => f.getType == classOf[HoodieSQLOption[_]]) + .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) .filter(option => option.tableConfigKey.isDefined && option.defaultValue.isDefined) - .map(option => option.tableConfigKey.get -> - valueMapping.getOrElse(option.defaultValue.get.toString, option.defaultValue.get.toString)) + .map(option => option.sqlKeyName -> option.defaultValue.get.toString) .toMap } @@ -154,7 +158,7 @@ object HoodieOptionConfig { def getPrimaryColumns(options: Map[String, String]): Array[String] = { val params = mappingSqlOptionToHoodieParam(options) params.get(DataSourceWriteOptions.RECORDKEY_FIELD.key) - .map(_.split(",").filter(_.length > 0)) + .map(_.split(",").filter(_.nonEmpty)) .getOrElse(Array.empty) } @@ -171,45 +175,89 @@ object HoodieOptionConfig { def getPreCombineField(options: Map[String, String]): Option[String] = { val params = mappingSqlOptionToHoodieParam(options) - params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key) + params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty) + } + + def deleteHoodieOptions(options: Map[String, String]): Map[String, String] = { + options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => keyMapping.contains(kv._1)) + } + + // extract primaryKey, preCombineField, type options + def extractSqlOptions(options: Map[String, String]): Map[String, String] = { + val sqlOptions = mappingTableConfigToSqlOption(options) + val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) + sqlOptions.filterKeys(targetOptions.contains) + } + + // validate primaryKey, preCombineField and type options + def validateTable(spark: SparkSession, schema: StructType, sqlOptions: Map[String, String]): Unit = { + val resolver = spark.sessionState.conf.resolver + + // validate primary key + val primaryKeys = sqlOptions.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName) + .map(_.split(",").filter(_.length > 0)) + ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.") + primaryKeys.get.foreach { primaryKey => + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, primaryKey)), + s"Can't find primaryKey `$primaryKey` in ${schema.treeString}.") + } + + // validate precombine key + val precombineKey = sqlOptions.get(SQL_KEY_PRECOMBINE_FIELD.sqlKeyName) + if (precombineKey.isDefined && precombineKey.get.nonEmpty) { + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, precombineKey.get)), + s"Can't find preCombineKey `${precombineKey.get}` in ${schema.treeString}.") + } + + // validate table type + val tableType = sqlOptions.get(SQL_KEY_TABLE_TYPE.sqlKeyName) + ValidationUtils.checkArgument(tableType.nonEmpty, "No `type` is specified.") + ValidationUtils.checkArgument( + tableType.get.equalsIgnoreCase(SQL_VALUE_TABLE_TYPE_COW) || + tableType.get.equalsIgnoreCase(SQL_VALUE_TABLE_TYPE_MOR), + s"'type' must be '$SQL_VALUE_TABLE_TYPE_COW' or '$SQL_VALUE_TABLE_TYPE_MOR'") } - def buildConf[T](): HoodieOptions[T] = { - new HoodieOptions[T] + def buildConf[T](): HoodieSQLOptionBuilder[T] = { + new HoodieSQLOptionBuilder[T] } } -case class HoodieOption[T](sqlKeyName: String, hoodieKeyName: String, - defaultValue: Option[T], tableConfigKey: Option[String] = None) +case class HoodieSQLOption[T]( + sqlKeyName: String, + hoodieKeyName: String, + tableConfigKey: Option[String], + defaultValue: Option[T] +) -class HoodieOptions[T] { +class HoodieSQLOptionBuilder[T] { private var sqlKeyName: String = _ private var hoodieKeyName: String =_ private var tableConfigKey: String =_ private var defaultValue: T =_ - def withSqlKey(sqlKeyName: String): HoodieOptions[T] = { + def withSqlKey(sqlKeyName: String): HoodieSQLOptionBuilder[T] = { this.sqlKeyName = sqlKeyName this } - def withHoodieKey(hoodieKeyName: String): HoodieOptions[T] = { + def withHoodieKey(hoodieKeyName: String): HoodieSQLOptionBuilder[T] = { this.hoodieKeyName = hoodieKeyName this } - def withTableConfigKey(tableConfigKey: String): HoodieOptions[T] = { + def withTableConfigKey(tableConfigKey: String): HoodieSQLOptionBuilder[T] = { this.tableConfigKey = tableConfigKey this } - def defaultValue(defaultValue: T): HoodieOptions[T] = { + def defaultValue(defaultValue: T): HoodieSQLOptionBuilder[T] = { this.defaultValue = defaultValue this } - def build(): HoodieOption[T] = { - HoodieOption(sqlKeyName, hoodieKeyName, Option(defaultValue), Option(tableConfigKey)) + def build(): HoodieSQLOption[T] = { + HoodieSQLOption(sqlKeyName, hoodieKeyName, Option(tableConfigKey), Option(defaultValue)) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala index cf9c49ef02a9..3b6436ee2e60 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala @@ -20,19 +20,18 @@ package org.apache.spark.sql.hudi import scala.collection.JavaConverters._ import java.net.URI import java.util.{Date, Locale, Properties} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hudi.SparkAdapterSupport + +import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport} import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.common.config.DFSPropertiesConfiguration import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline -import org.apache.spark.SPARK_VERSION -import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.avro.SchemaConverters +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstantTimeGenerator} + import org.apache.spark.sql.{Column, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -45,11 +44,16 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.types.{DataType, NullType, StringType, StructField, StructType} import java.text.SimpleDateFormat + import scala.collection.immutable.Map object HoodieSqlUtils extends SparkAdapterSupport { - private val defaultDateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - private val defaultDateFormat = new SimpleDateFormat("yyyy-MM-dd") + // NOTE: {@code SimpleDataFormat} is NOT thread-safe + // TODO replace w/ DateTimeFormatter + private val defaultDateFormat = + ThreadLocal.withInitial(new java.util.function.Supplier[SimpleDateFormat] { + override def get() = new SimpleDateFormat("yyyy-MM-dd") + }) def isHoodieTable(table: CatalogTable): Boolean = { table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi" @@ -76,21 +80,21 @@ object HoodieSqlUtils extends SparkAdapterSupport { } } - def getTableSqlSchema(metaClient: HoodieTableMetaClient): Option[StructType] = { + def getTableSqlSchema(metaClient: HoodieTableMetaClient, + includeMetadataFields: Boolean = false): Option[StructType] = { val schemaResolver = new TableSchemaResolver(metaClient) - val avroSchema = try Some(schemaResolver.getTableAvroSchema(false)) + val avroSchema = try Some(schemaResolver.getTableAvroSchema(includeMetadataFields)) catch { case _: Throwable => None } - avroSchema.map(SchemaConverters.toSqlType(_).dataType - .asInstanceOf[StructType]).map(removeMetaFields) + avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType) } def getAllPartitionPaths(spark: SparkSession, table: CatalogTable): Seq[String] = { val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) val metadataConfig = { val properties = new Properties() - properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties).asJava) + properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties ++ table.properties).asJava) HoodieMetadataConfig.newBuilder.fromProperties(properties).build() } FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, getTableLocation(table, spark)).asScala @@ -273,13 +277,11 @@ object HoodieSqlUtils extends SparkAdapterSupport { */ def withSparkConf(spark: SparkSession, options: Map[String, String]) (baseConfig: Map[String, String] = Map.empty): Map[String, String] = { - baseConfig ++ // Table options has the highest priority + baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options)) .filterKeys(_.startsWith("hoodie.")) } - def isSpark3: Boolean = SPARK_VERSION.startsWith("3.") - def isEnableHive(sparkSession: SparkSession): Boolean = "hive" == sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION) @@ -292,16 +294,35 @@ object HoodieSqlUtils extends SparkAdapterSupport { * 3、yyyyMMddHHmmss */ def formatQueryInstant(queryInstant: String): String = { - if (queryInstant.length == 19) { // for yyyy-MM-dd HH:mm:ss - HoodieActiveTimeline.formatInstantTime(defaultDateTimeFormat.parse(queryInstant)) - } else if (queryInstant.length == 14) { // for yyyyMMddHHmmss - HoodieActiveTimeline.parseInstantTime(queryInstant) // validate the format + val instantLength = queryInstant.length + if (instantLength == 19 || instantLength == 23) { // for yyyy-MM-dd HH:mm:ss[.SSS] + HoodieInstantTimeGenerator.getInstantForDateString(queryInstant) + } else if (instantLength == HoodieInstantTimeGenerator.SECS_INSTANT_ID_LENGTH + || instantLength == HoodieInstantTimeGenerator.MILLIS_INSTANT_ID_LENGTH) { // for yyyyMMddHHmmss[SSS] + HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) // validate the format queryInstant - } else if (queryInstant.length == 10) { // for yyyy-MM-dd - HoodieActiveTimeline.formatInstantTime(defaultDateFormat.parse(queryInstant)) + } else if (instantLength == 10) { // for yyyy-MM-dd + HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant)) } else { throw new IllegalArgumentException(s"Unsupported query instant time format: $queryInstant," - + s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss' or 'yyyy-MM-dd' or 'yyyyMMddHHmmss'") + + s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss.SSS' or 'yyyy-MM-dd' or 'yyyyMMddHHmmssSSS'") + } + } + + def formatName(sparkSession: SparkSession, name: String): String = { + if (sparkSession.sessionState.conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) + } + + /** + * Check if this is a empty table path. + */ + def isEmptyPath(tablePath: String, conf: Configuration): Boolean = { + val basePath = new Path(tablePath) + val fs = basePath.getFileSystem(conf) + if (fs.exists(basePath)) { + fs.listStatus(basePath).isEmpty + } else { + true } } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index 87cbb8a7f030..31af71994d0b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -17,26 +17,25 @@ package org.apache.spark.sql.hudi.analysis +import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport} import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL -import org.apache.hudi.SparkAdapterSupport - -import scala.collection.JavaConverters._ import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.analysis.UnresolvedStar -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, Literal, NamedExpression} + +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, Literal, NamedExpression} import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} -import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} import org.apache.spark.sql.hudi.HoodieSqlUtils._ import org.apache.spark.sql.hudi.command._ +import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.{AnalysisException, SparkSession} + +import scala.collection.JavaConverters._ object HoodieAnalysis { def customResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] = @@ -139,7 +138,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi // We can do this because under the normal case, we should not allow to update or set // the hoodie's meta field in sql statement, it is a system field, cannot set the value // by user. - if (HoodieSqlUtils.isSpark3) { + if (HoodieSparkUtils.isSpark3) { val assignmentFieldNames = assignments.map(_.key).map { case attr: AttributeReference => attr.name @@ -180,11 +179,19 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi .map { case (targetAttr, sourceAttr) => Assignment(targetAttr, sourceAttr) } } } else { - assignments.map(assignment => { + // For Spark3.2, InsertStarAction/UpdateStarAction's assignments will contain the meta fields. + val withoutMetaAttrs = assignments.filterNot{ assignment => + if (assignment.key.isInstanceOf[Attribute]) { + HoodieSqlUtils.isMetaField(assignment.key.asInstanceOf[Attribute].name) + } else { + false + } + } + withoutMetaAttrs.map { assignment => val resolvedKey = resolveExpressionFrom(target)(assignment.key) val resolvedValue = resolveExpressionFrom(resolvedSource, Some(target))(assignment.value) Assignment(resolvedKey, resolvedValue) - }) + } } (resolvedCondition, resolvedAssignments) } @@ -202,8 +209,9 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi val targetTableId = getMergeIntoTargetTableId(mergeInto) val targetTable = sparkSession.sessionState.catalog.getTableMetadata(targetTableId) - val targetTableType = HoodieOptionConfig.getTableType(targetTable.storage.properties) - val preCombineField = HoodieOptionConfig.getPreCombineField(targetTable.storage.properties) + val tblProperties = targetTable.storage.properties ++ targetTable.properties + val targetTableType = HoodieOptionConfig.getTableType(tblProperties) + val preCombineField = HoodieOptionConfig.getPreCombineField(tblProperties) // Get the map of target attribute to value of the update assignments. val target2Values = resolvedAssignments.map { @@ -243,6 +251,12 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi case DeleteAction(condition) => val resolvedCondition = condition.map(resolveExpressionFrom(resolvedSource)(_)) DeleteAction(resolvedCondition) + case action: MergeAction => + // SPARK-34962: use UpdateStarAction as the explicit representation of * in UpdateAction. + // So match and covert this in Spark3.2 env. + val (resolvedCondition, resolvedAssignments) = + resolveConditionAssignments(action.condition, Seq.empty) + UpdateAction(resolvedCondition, resolvedAssignments) } // Resolve the notMatchedActions val resolvedNotMatchedActions = notMatchedActions.map { @@ -250,6 +264,12 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi val (resolvedCondition, resolvedAssignments) = resolveConditionAssignments(condition, assignments) InsertAction(resolvedCondition, resolvedAssignments) + case action: MergeAction => + // SPARK-34962: use InsertStarAction as the explicit representation of * in InsertAction. + // So match and covert this in Spark3.2 env. + val (resolvedCondition, resolvedAssignments) = + resolveConditionAssignments(action.condition, Seq.empty) + InsertAction(resolvedCondition, resolvedAssignments) } // Return the resolved MergeIntoTable MergeIntoTable(target, resolvedSource, resolvedMergeCondition, @@ -406,10 +426,14 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic case CreateDataSourceTableCommand(table, ignoreIfExists) if isHoodieTable(table) => CreateHoodieTableCommand(table, ignoreIfExists) + // Rewrite the DropTableCommand to DropHoodieTableCommand + case DropTableCommand(tableName, ifExists, isView, purge) + if isHoodieTable(tableName, sparkSession) => + DropHoodieTableCommand(tableName, ifExists, isView, purge) // Rewrite the AlterTableDropPartitionCommand to AlterHoodieTableDropPartitionCommand - case AlterTableDropPartitionCommand(tableName, specs, _, _, _) + case AlterTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData) if isHoodieTable(tableName, sparkSession) => - AlterHoodieTableDropPartitionCommand(tableName, specs) + AlterHoodieTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData) // Rewrite the AlterTableRenameCommand to AlterHoodieTableRenameCommand // Rewrite the AlterTableAddColumnsCommand to AlterHoodieTableAddColumnsCommand case AlterTableAddColumnsCommand(tableId, colsToAdd) @@ -423,9 +447,11 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic case AlterTableChangeColumnCommand(tableName, columnName, newColumn) if isHoodieTable(tableName, sparkSession) => AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn) - case ShowPartitionsCommand(tableName, specOpt) - if isHoodieTable(tableName, sparkSession) => - ShowHoodieTablePartitionsCommand(tableName, specOpt) + // SPARK-34238: the definition of ShowPartitionsCommand has been changed in Spark3.2. + // Match the class type instead of call the `unapply` method. + case s: ShowPartitionsCommand + if isHoodieTable(s.tableName, sparkSession) => + ShowHoodieTablePartitionsCommand(s.tableName, s.spec) // Rewrite TruncateTableCommand to TruncateHoodieTableCommand case TruncateTableCommand(tableName, partitionSpec) if isHoodieTable(tableName, sparkSession) => diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala index 4123ea9499a6..c6c08da1e69c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala @@ -18,26 +18,24 @@ package org.apache.spark.sql.hudi.command import java.nio.charset.StandardCharsets + import org.apache.avro.Schema import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType} -import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} import org.apache.hudi.common.util.{CommitUtils, Option} import org.apache.hudi.table.HoodieSparkTable - -import scala.collection.JavaConverters._ import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, HoodieWriterUtils} + import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} -import org.apache.spark.sql.hudi.HoodieSqlUtils -import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.util.SchemaUtils +import scala.collection.JavaConverters._ import scala.util.control.NonFatal /** @@ -46,36 +44,37 @@ import scala.util.control.NonFatal case class AlterHoodieTableAddColumnsCommand( tableId: TableIdentifier, colsToAdd: Seq[StructField]) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { if (colsToAdd.nonEmpty) { val resolver = sparkSession.sessionState.conf.resolver - val table = sparkSession.sessionState.catalog.getTableMetadata(tableId) + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableId) + val tableSchema = hoodieCatalogTable.tableSchema val existsColumns = - colsToAdd.map(_.name).filter(col => table.schema.fieldNames.exists(f => resolver(f, col))) + colsToAdd.map(_.name).filter(col => tableSchema.fieldNames.exists(f => resolver(f, col))) if (existsColumns.nonEmpty) { throw new AnalysisException(s"Columns: [${existsColumns.mkString(",")}] already exists in the table," + - s" table columns is: [${HoodieSqlUtils.removeMetaFields(table.schema).fieldNames.mkString(",")}]") + s" table columns is: [${hoodieCatalogTable.tableSchemaWithoutMetaFields.fieldNames.mkString(",")}]") } // Get the new schema - val newSqlSchema = StructType(table.schema.fields ++ colsToAdd) + val newSqlSchema = StructType(tableSchema.fields ++ colsToAdd) val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableId.table) val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace) // Commit with new schema to change the table schema - AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, table, sparkSession) + AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, hoodieCatalogTable, sparkSession) // Refresh the new schema to meta - val newDataSchema = StructType(table.dataSchema.fields ++ colsToAdd) - refreshSchemaInMeta(sparkSession, table, newDataSchema) + val newDataSchema = StructType(hoodieCatalogTable.dataSchema.fields ++ colsToAdd) + refreshSchemaInMeta(sparkSession, hoodieCatalogTable.table, newDataSchema) } Seq.empty[Row] } private def refreshSchemaInMeta(sparkSession: SparkSession, table: CatalogTable, - newSqlSchema: StructType): Unit = { + newSqlDataSchema: StructType): Unit = { try { sparkSession.catalog.uncacheTable(tableId.quotedString) } catch { @@ -85,12 +84,11 @@ case class AlterHoodieTableAddColumnsCommand( sparkSession.catalog.refreshTable(table.identifier.unquotedString) SchemaUtils.checkColumnNameDuplication( - newSqlSchema.map(_.name), + newSqlDataSchema.map(_.name), "in the table definition of " + table.identifier, conf.caseSensitiveAnalysis) - DDLUtils.checkDataColNames(table, colsToAdd.map(_.name)) - sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlSchema) + sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlDataSchema) } } @@ -98,20 +96,22 @@ object AlterHoodieTableAddColumnsCommand { /** * Generate an empty commit with new schema to change the table's schema. * @param schema The new schema to commit. - * @param table The hoodie table. + * @param hoodieCatalogTable The hoodie catalog table. * @param sparkSession The spark session. */ - def commitWithSchema(schema: Schema, table: CatalogTable, sparkSession: SparkSession): Unit = { - val path = getTableLocation(table, sparkSession) + def commitWithSchema(schema: Schema, hoodieCatalogTable: HoodieCatalogTable, + sparkSession: SparkSession): Unit = { val jsc = new JavaSparkContext(sparkSession.sparkContext) - val client = DataSourceUtils.createHoodieClient(jsc, schema.toString, - path, table.identifier.table, HoodieWriterUtils.parametersWithWriteDefaults(table.storage.properties).asJava) - - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build() - - val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, metaClient.getTableType) + val client = DataSourceUtils.createHoodieClient( + jsc, + schema.toString, + hoodieCatalogTable.tableLocation, + hoodieCatalogTable.tableName, + HoodieWriterUtils.parametersWithWriteDefaults(hoodieCatalogTable.catalogProperties).asJava + ) + + val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, hoodieCatalogTable.tableType) val instantTime = HoodieActiveTimeline.createNewInstantTime client.startCommitWithTime(instantTime, commitActionType) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala index d9569ceb53b4..b69c686f99bf 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala @@ -18,14 +18,16 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema + import org.apache.hudi.AvroConversionUtils import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException + import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{StructField, StructType} import scala.util.control.NonFatal @@ -34,22 +36,21 @@ import scala.util.control.NonFatal * Command for alter hudi table's column type. */ case class AlterHoodieTableChangeColumnCommand( - tableName: TableIdentifier, + tableIdentifier: TableIdentifier, columnName: String, newColumn: StructField) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val table = catalog.getTableMetadata(tableName) - val resolver = sparkSession.sessionState.conf.resolver + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) + val resolver = sparkSession.sessionState.conf.resolver if (!resolver(columnName, newColumn.name)) { throw new AnalysisException(s"Can not support change column name for hudi table currently.") } // Get the new schema - val newSqlSchema = StructType( - table.schema.fields.map { field => + val newTableSchema = StructType( + hoodieCatalogTable.tableSchema.fields.map { field => if (resolver(field.name, columnName)) { newColumn } else { @@ -57,34 +58,30 @@ case class AlterHoodieTableChangeColumnCommand( } }) val newDataSchema = StructType( - table.dataSchema.fields.map { field => + hoodieCatalogTable.dataSchema.fields.map { field => if (resolver(field.name, columnName)) { newColumn } else { field } }) - val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName.table) - val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace) + val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableIdentifier.table) + val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newTableSchema, structName, nameSpace) - val path = getTableLocation(table, sparkSession) - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder().setBasePath(path) - .setConf(hadoopConf).build() // Validate the compatibility between new schema and origin schema. - validateSchema(newSchema, metaClient) + validateSchema(newSchema, hoodieCatalogTable.metaClient) // Commit new schema to change the table schema - AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, table, sparkSession) + AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, hoodieCatalogTable, sparkSession) try { - sparkSession.catalog.uncacheTable(tableName.quotedString) + sparkSession.catalog.uncacheTable(tableIdentifier.quotedString) } catch { case NonFatal(e) => - log.warn(s"Exception when attempting to uncache table ${tableName.quotedString}", e) + log.warn(s"Exception when attempting to uncache table ${tableIdentifier.quotedString}", e) } - sparkSession.catalog.refreshTable(tableName.unquotedString) + sparkSession.catalog.refreshTable(tableIdentifier.unquotedString) // Change the schema in the meta using new data schema. - catalog.alterTableDataSchema(tableName, newDataSchema) + sparkSession.sessionState.catalog.alterTableDataSchema(tableIdentifier, newDataSchema) Seq.empty[Row] } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala index 7c4d45649587..21f16275d132 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala @@ -17,97 +17,103 @@ package org.apache.spark.sql.hudi.command -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils} import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME -import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} +import org.apache.hudi.hive.MultiPartKeysValueExtractor +import org.apache.hudi.hive.ddl.HiveSyncMode +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter} + import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.hudi.HoodieSqlUtils._ +import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} case class AlterHoodieTableDropPartitionCommand( tableIdentifier: TableIdentifier, - specs: Seq[TablePartitionSpec]) -extends RunnableCommand { + specs: Seq[TablePartitionSpec], + ifExists : Boolean, + purge : Boolean, + retainData : Boolean) +extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val table = catalog.getTableMetadata(tableIdentifier) - DDLUtils.verifyAlterTableType(catalog, table, isView = false) - - val path = getTableLocation(table, sparkSession) - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build() - val partitionColumns = metaClient.getTableConfig.getPartitionFields + val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}" + logInfo(s"start execute alter table drop partition command for $fullTableName") + + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) + + if (!hoodieCatalogTable.isPartitionedTable) { + throw new AnalysisException(s"$fullTableName is a non-partitioned table that is not allowed to drop partition") + } + + DDLUtils.verifyAlterTableType( + sparkSession.sessionState.catalog, hoodieCatalogTable.table, isView = false) + val normalizedSpecs: Seq[Map[String, String]] = specs.map { spec => normalizePartitionSpec( spec, - partitionColumns.get(), - table.identifier.quotedString, + hoodieCatalogTable.partitionFields, + hoodieCatalogTable.tableName, sparkSession.sessionState.conf.resolver) } - val parameters = buildHoodieConfig(sparkSession, path, partitionColumns.get, normalizedSpecs) - + val partitionsToDrop = getPartitionPathToDrop(hoodieCatalogTable, normalizedSpecs) + val parameters = buildHoodieConfig(sparkSession, hoodieCatalogTable, partitionsToDrop) HoodieSparkSqlWriter.write( sparkSession.sqlContext, SaveMode.Append, parameters, sparkSession.emptyDataFrame) + + // Recursively delete partition directories + if (purge) { + val engineContext = new HoodieSparkEngineContext(sparkSession.sparkContext) + val basePath = hoodieCatalogTable.tableLocation + val fullPartitionPath = FSUtils.getPartitionPath(basePath, partitionsToDrop) + logInfo("Clean partition up " + fullPartitionPath) + val fs = FSUtils.getFs(basePath, sparkSession.sparkContext.hadoopConfiguration) + FSUtils.deleteDir(engineContext, fs, fullPartitionPath, sparkSession.sparkContext.defaultParallelism) + } + + sparkSession.catalog.refreshTable(tableIdentifier.unquotedString) + logInfo(s"Finish execute alter table drop partition command for $fullTableName") Seq.empty[Row] } private def buildHoodieConfig( sparkSession: SparkSession, - path: String, - partitionColumns: Seq[String], - normalizedSpecs: Seq[Map[String, String]]): Map[String, String] = { - val table = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier) - val allPartitionPaths = getAllPartitionPaths(sparkSession, table) - val enableHiveStylePartitioning = isHiveStyledPartitioning(allPartitionPaths, table) - val enableEncodeUrl = isUrlEncodeEnabled(allPartitionPaths, table) - val partitionsToDelete = normalizedSpecs.map { spec => - partitionColumns.map{ partitionColumn => - val encodedPartitionValue = if (enableEncodeUrl) { - PartitionPathEncodeUtils.escapePathName(spec(partitionColumn)) - } else { - spec(partitionColumn) - } - if (enableHiveStylePartitioning) { - partitionColumn + "=" + encodedPartitionValue - } else { - encodedPartitionValue - } - }.mkString("/") - }.mkString(",") - - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(sparkSession.sessionState.newHadoopConf) - .build() - val tableConfig = metaClient.getTableConfig - - val optParams = withSparkConf(sparkSession, table.storage.properties) { + hoodieCatalogTable: HoodieCatalogTable, + partitionsToDrop: String): Map[String, String] = { + val partitionFields = hoodieCatalogTable.partitionFields.mkString(",") + val enableHive = isEnableHive(sparkSession) + withSparkConf(sparkSession, Map.empty) { Map( - "path" -> path, - TBL_NAME.key -> tableIdentifier.table, - TABLE_TYPE.key -> tableConfig.getTableType.name, + "path" -> hoodieCatalogTable.tableLocation, + TBL_NAME.key -> hoodieCatalogTable.tableName, + TABLE_TYPE.key -> hoodieCatalogTable.tableTypeName, OPERATION.key -> DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL, - PARTITIONS_TO_DELETE.key -> partitionsToDelete, - RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp, - PRECOMBINE_FIELD.key -> tableConfig.getPreCombineField, - PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp + PARTITIONS_TO_DELETE.key -> partitionsToDrop, + RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), + PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), + PARTITIONPATH_FIELD.key -> partitionFields, + HIVE_SYNC_ENABLED.key -> enableHive.toString, + META_SYNC_ENABLED.key -> enableHive.toString, + HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), + HIVE_USE_JDBC.key -> "false", + HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"), + HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table, + HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", + HIVE_PARTITION_FIELDS.key -> partitionFields, + HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName ) } - - val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams) - val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters) - translatedOptions } def normalizePartitionSpec[T]( @@ -139,4 +145,27 @@ extends RunnableCommand { normalizedPartSpec.toMap } + def getPartitionPathToDrop( + hoodieCatalogTable: HoodieCatalogTable, + normalizedSpecs: Seq[Map[String, String]]): String = { + val table = hoodieCatalogTable.table + val allPartitionPaths = hoodieCatalogTable.getAllPartitionPaths + val enableHiveStylePartitioning = isHiveStyledPartitioning(allPartitionPaths, table) + val enableEncodeUrl = isUrlEncodeEnabled(allPartitionPaths, table) + val partitionsToDrop = normalizedSpecs.map { spec => + hoodieCatalogTable.partitionFields.map { partitionColumn => + val encodedPartitionValue = if (enableEncodeUrl) { + PartitionPathEncodeUtils.escapePathName(spec(partitionColumn)) + } else { + spec(partitionColumn) + } + if (enableHiveStylePartitioning) { + partitionColumn + "=" + encodedPartitionValue + } else { + encodedPartitionValue + } + }.mkString("/") + }.mkString(",") + partitionsToDrop + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala index 2df9ec86998c..c7b5bdc202f6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.common.table.HoodieTableMetaClient + import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.execution.command.AlterTableRenameCommand -import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation /** * Command for alter hudi table's table name. @@ -34,18 +35,15 @@ class AlterHoodieTableRenameCommand( override def run(sparkSession: SparkSession): Seq[Row] = { if (newName != oldName) { - val catalog = sparkSession.sessionState.catalog - val table = catalog.getTableMetadata(oldName) - val path = getTableLocation(table, sparkSession) - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder().setBasePath(path) - .setConf(hadoopConf).build() + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, oldName) + // Init table with new name. HoodieTableMetaClient.withPropertyBuilder() - .fromProperties(metaClient.getTableConfig.getProps) + .fromProperties(hoodieCatalogTable.tableConfig.getProps) .setTableName(newName.table) - .initTable(hadoopConf, path) + .initTable(hadoopConf, hoodieCatalogTable.tableLocation) + // Call AlterTableRenameCommand#run to rename table in meta. super.run(sparkSession) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala index e310da673ed4..be66584778b4 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala @@ -24,12 +24,12 @@ import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeli import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{HoodieTimer, Option => HOption} import org.apache.hudi.exception.HoodieException + import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation +import org.apache.spark.sql.catalyst.plans.logical.{CompactionOperation, LogicalPlan} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE} -import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.hudi.HoodieSqlUtils import org.apache.spark.sql.types.StringType @@ -38,7 +38,7 @@ import scala.collection.JavaConverters._ case class CompactionHoodiePathCommand(path: String, operation: CompactionOperation, instantTimestamp: Option[Long] = None) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val metaClient = HoodieTableMetaClient.builder().setBasePath(path) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala index 631504d51415..27fb6e779a25 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala @@ -21,13 +21,13 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation import org.apache.spark.sql.types.StringType case class CompactionHoodieTableCommand(table: CatalogTable, operation: CompactionOperation, instantTimestamp: Option[Long]) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val basePath = getTableLocation(table, sparkSession) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala index ca7891c5d69e..44c57239703f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala @@ -22,14 +22,14 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.util.CompactionUtils import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.types.{IntegerType, StringType} import scala.collection.JavaConverters.asScalaIteratorConverter case class CompactionShowHoodiePathCommand(path: String, limit: Int) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val metaClient = HoodieTableMetaClient.builder().setBasePath(path.toString) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala index 0702e6bc2449..7502bf7623aa 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.hudi.command import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation import org.apache.spark.sql.types.{IntegerType, StringType} case class CompactionShowHoodieTableCommand(table: CatalogTable, limit: Int) - extends RunnableCommand { + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val basePath = getTableLocation(table, sparkSession) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 38c7e290a265..2790ea97c87c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -19,15 +19,16 @@ package org.apache.spark.sql.hudi.command import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path + import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.hive.util.ConfigUtils import org.apache.hudi.sql.InsertMode + import org.apache.spark.sql.{Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, HoodieCatalogTable} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.DataWritingCommand -import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation +import org.apache.spark.sql.hudi.HoodieSqlUtils import scala.collection.JavaConverters._ @@ -37,9 +38,9 @@ import scala.collection.JavaConverters._ case class CreateHoodieTableAsSelectCommand( table: CatalogTable, mode: SaveMode, - query: LogicalPlan) extends DataWritingCommand { + query: LogicalPlan) extends HoodieLeafRunnableCommand { - override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { + override def run(sparkSession: SparkSession): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) @@ -62,20 +63,26 @@ case class CreateHoodieTableAsSelectCommand( // scalastyle:on } } - val tablePath = getTableLocation(table, sparkSession) - val hadoopConf = sparkSession.sessionState.newHadoopConf() - assert(CreateHoodieTableCommand.isEmptyPath(tablePath, hadoopConf), - s"Path '$tablePath' should be empty for CTAS") // ReOrder the query which move the partition columns to the last of the project list val reOrderedQuery = reOrderPartitionColumn(query, table.partitionColumnNames) val tableWithSchema = table.copy(schema = reOrderedQuery.schema) + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableWithSchema) + val tablePath = hoodieCatalogTable.tableLocation + val hadoopConf = sparkSession.sessionState.newHadoopConf() + assert(HoodieSqlUtils.isEmptyPath(tablePath, hadoopConf), + s"Path '$tablePath' should be empty for CTAS") + // Execute the insert query try { + // init hoodie table + hoodieCatalogTable.initHoodieTable() + + val tblProperties = hoodieCatalogTable.catalogProperties val options = Map( DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString, - DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(table.storage.properties.asJava), + DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava), DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava), DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" @@ -86,9 +93,8 @@ case class CreateHoodieTableAsSelectCommand( // If write success, create the table in catalog if it has not synced to the // catalog by the meta sync. if (!sparkSession.sessionState.catalog.tableExists(tableIdentWithDB)) { - // Create the table - val createTableCommand = CreateHoodieTableCommand(tableWithSchema, mode == SaveMode.Ignore) - createTableCommand.createTableInCatalog(sparkSession, checkPathForManagedTable = false) + // create catalog table for this hoodie table + CreateHoodieTableCommand.createTableInCatalog(sparkSession, hoodieCatalogTable, mode == SaveMode.Ignore) } } else { // failed to insert data, clear table path clearTablePath(tablePath, hadoopConf) @@ -107,8 +113,6 @@ case class CreateHoodieTableAsSelectCommand( fs.delete(path, true) } - override def outputColumnNames: Seq[String] = query.output.map(_.name) - private def reOrderPartitionColumn(query: LogicalPlan, partitionColumns: Seq[String]): LogicalPlan = { if (partitionColumns.isEmpty) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 8ac63126a4b9..2608f9383601 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -17,48 +17,38 @@ package org.apache.spark.sql.hudi.command -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path + +import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} import org.apache.hudi.common.model.HoodieFileFormat -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import org.apache.hudi.common.util.ValidationUtils +import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.hadoop.HoodieParquetInputFormat import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils -import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} -import org.apache.spark.internal.Logging -import org.apache.spark.sql.avro.SchemaConverters -import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, TableAlreadyExistsException} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.hive.HiveClientUtils import org.apache.spark.sql.hive.HiveExternalCatalog._ -import org.apache.spark.sql.hudi.HoodieOptionConfig +import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath} import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.{SPARK_VERSION, SparkConf} -import java.util.{Locale, Properties} - -import org.apache.hudi.exception.HoodieException -import org.apache.hudi.keygen.ComplexKeyGenerator -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.util.control.NonFatal /** * Command for create hoodie table. */ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean) - extends RunnableCommand with SparkAdapterSupport { + extends HoodieLeafRunnableCommand with SparkAdapterSupport { override def run(sparkSession: SparkSession): Seq[Row] = { - val tableName = table.identifier.unquotedString - val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier) if (tableIsExists) { if (ignoreIfExists) { @@ -66,64 +56,54 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean return Seq.empty[Row] // scalastyle:on } else { - throw new IllegalArgumentException(s"Table $tableName already exists.") + throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.") } } - // Create table in the catalog - val createTable = createTableInCatalog(sparkSession) - // Init the hoodie.properties - initTableIfNeed(sparkSession, createTable) + + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, table) + // check if there are conflict between table configs defined in hoodie table and properties defined in catalog. + CreateHoodieTableCommand.validateTblProperties(hoodieCatalogTable) + // init hoodie table + hoodieCatalogTable.initHoodieTable() + + try { + // create catalog table for this hoodie table + CreateHoodieTableCommand.createTableInCatalog(sparkSession, hoodieCatalogTable, ignoreIfExists) + } catch { + case NonFatal(e) => + logWarning(s"Failed to create catalog table in metastore: ${e.getMessage}") + } Seq.empty[Row] } +} + +object CreateHoodieTableCommand { + + def validateTblProperties(hoodieCatalogTable: HoodieCatalogTable): Unit = { + if (hoodieCatalogTable.hoodieTableExists) { + val originTableConfig = hoodieCatalogTable.tableConfig.getProps.asScala.toMap + val tableOptions = hoodieCatalogTable.catalogProperties + + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key) + checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) + } + } def createTableInCatalog(sparkSession: SparkSession, - checkPathForManagedTable: Boolean = true): CatalogTable = { + hoodieCatalogTable: HoodieCatalogTable, ignoreIfExists: Boolean): Unit = { + val table = hoodieCatalogTable.table assert(table.tableType != CatalogTableType.VIEW) - assert(table.provider.isDefined) - val sessionState = sparkSession.sessionState - val tableName = table.identifier.unquotedString - val path = getTableLocation(table, sparkSession) - val conf = sparkSession.sessionState.newHadoopConf() - val isTableExists = tableExistsInPath(path, conf) - // Get the schema & table options - val (newSchema, tableOptions) = if (table.tableType == CatalogTableType.EXTERNAL && - isTableExists) { - // If this is an external table & the table has already exists in the location, - // load the schema from the table meta. - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - val tableSchema = getTableSqlSchema(metaClient) - - // Get options from the external table and append with the options in ddl. - val originTableConfig = HoodieOptionConfig.mappingTableConfigToSqlOption( - metaClient.getTableConfig.getProps.asScala.toMap) - val extraConfig = extraTableConfig(sparkSession, isTableExists, originTableConfig) - val options = originTableConfig ++ table.storage.properties ++ extraConfig - - val userSpecifiedSchema = table.schema - if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) { - (addMetaFields(tableSchema.get), options) - } else if (userSpecifiedSchema.nonEmpty) { - (addMetaFields(userSpecifiedSchema), options) - } else { - throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName") - } - } else { - assert(table.schema.nonEmpty, s"Missing schema for Create Table: $tableName") - // SPARK-19724: the default location of a managed table should be non-existent or empty. - if (checkPathForManagedTable && table.tableType == CatalogTableType.MANAGED - && !isEmptyPath(path, conf)) { - throw new AnalysisException(s"Can not create the managed table('$tableName')" + - s". The associated location('$path') already exists.") - } - // Add the meta fields to the schema if this is a managed table or an empty external table. - val options = table.storage.properties ++ extraTableConfig(sparkSession, false) - (addMetaFields(table.schema), options) - } - val tableType = HoodieOptionConfig.getTableType(table.storage.properties) + val catalog = sparkSession.sessionState.catalog + val path = hoodieCatalogTable.tableLocation + val tableConfig = hoodieCatalogTable.tableConfig + val properties = tableConfig.getProps.asScala.toMap + + val tableType = tableConfig.getTableType.name() val inputFormat = tableType match { case DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL => classOf[HoodieParquetInputFormat].getCanonicalName @@ -134,31 +114,40 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val outputFormat = HoodieInputFormatUtils.getOutputFormatClassName(HoodieFileFormat.PARQUET) val serdeFormat = HoodieInputFormatUtils.getSerDeClassName(HoodieFileFormat.PARQUET) - val newStorage = new CatalogStorageFormat(Some(new Path(path).toUri), - Some(inputFormat), Some(outputFormat), Some(serdeFormat), - table.storage.compressed, tableOptions + ("path" -> path)) + // only parameters irrelevant to hudi can be set to storage.properties + val storageProperties = HoodieOptionConfig.deleteHoodieOptions(properties) + val newStorage = new CatalogStorageFormat( + Some(new Path(path).toUri), + Some(inputFormat), + Some(outputFormat), + Some(serdeFormat), + table.storage.compressed, + storageProperties + ("path" -> path)) - val newDatabaseName = formatName(table.identifier.database - .getOrElse(sessionState.catalog.getCurrentDatabase)) - val newTableName = formatName(table.identifier.table) + val tablName = HoodieSqlUtils.formatName(sparkSession, table.identifier.table) + val newDatabaseName = HoodieSqlUtils.formatName(sparkSession, table.identifier.database + .getOrElse(catalog.getCurrentDatabase)) val newTableIdentifier = table.identifier - .copy(table = newTableName, database = Some(newDatabaseName)) - - val newTable = table.copy(identifier = newTableIdentifier, - schema = newSchema, storage = newStorage, createVersion = SPARK_VERSION) - // validate the table - validateTable(newTable) + .copy(table = tablName, database = Some(newDatabaseName)) + + // append pk, preCombineKey, type to the properties of table + val newTblProperties = hoodieCatalogTable.catalogProperties ++ HoodieOptionConfig.extractSqlOptions(properties) + val newTable = table.copy( + identifier = newTableIdentifier, + schema = hoodieCatalogTable.tableSchema, + storage = newStorage, + createVersion = SPARK_VERSION, + properties = newTblProperties + ) // Create table in the catalog val enableHive = isEnableHive(sparkSession) if (enableHive) { - createHiveDataSourceTable(newTable, sparkSession) + createHiveDataSourceTable(sparkSession, newTable, ignoreIfExists) } else { - sessionState.catalog.createTable(newTable, ignoreIfExists = false, - validateLocation = checkPathForManagedTable) + catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) } - newTable } /** @@ -169,9 +158,8 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean * @param table * @param sparkSession */ - private def createHiveDataSourceTable(table: CatalogTable, sparkSession: SparkSession): Unit = { - // check schema - verifyDataSchema(table.identifier, table.tableType, table.schema) + private def createHiveDataSourceTable(sparkSession: SparkSession, table: CatalogTable, + ignoreIfExists: Boolean): Unit = { val dbName = table.identifier.database.get // check database val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName) @@ -186,54 +174,16 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val dataSourceProps = tableMetaToTableProps(sparkSession.sparkContext.conf, table, table.schema) - val tableWithDataSourceProps = table.copy(properties = dataSourceProps) + val tableWithDataSourceProps = table.copy(properties = dataSourceProps ++ table.properties) val client = HiveClientUtils.newClientForMetadata(sparkSession.sparkContext.conf, sparkSession.sessionState.newHadoopConf()) // create hive table. client.createTable(tableWithDataSourceProps, ignoreIfExists) } - private def formatName(name: String): String = { - if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) - } - - // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema - private def verifyDataSchema(tableName: TableIdentifier, - tableType: CatalogTableType, - dataSchema: StructType): Unit = { - if (tableType != CatalogTableType.VIEW) { - val invalidChars = Seq(",", ":", ";") - def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f => - f.dataType match { - case st: StructType => verifyNestedColumnNames(st) - case _ if invalidChars.exists(f.name.contains) => - val invalidCharsString = invalidChars.map(c => s"'$c'").mkString(", ") - val errMsg = "Cannot create a table having a nested column whose name contains " + - s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableName; " + - s"Column: ${f.name}" - throw new AnalysisException(errMsg) - case _ => - } - } - - dataSchema.foreach { f => - f.dataType match { - // Checks top-level column names - case _ if f.name.contains(",") => - throw new AnalysisException("Cannot create a table having a column whose name " + - s"contains commas in Hive metastore. Table: $tableName; Column: ${f.name}") - // Checks nested column names - case st: StructType => - verifyNestedColumnNames(st) - case _ => - } - } - } - } // This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#tableMetaToTableProps - private def tableMetaToTableProps( sparkConf: SparkConf, - table: CatalogTable, - schema: StructType): Map[String, String] = { + private def tableMetaToTableProps(sparkConf: SparkConf, table: CatalogTable, + schema: StructType): Map[String, String] = { val partitionColumns = table.partitionColumnNames val bucketSpec = table.bucketSpec @@ -248,7 +198,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean val schemaJsonString = schema.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq - properties.put(DATASOURCE_SCHEMA_NUMPARTS, parts.size.toString) + properties.put(DATASOURCE_SCHEMA_PREFIX + "numParts", parts.size.toString) parts.zipWithIndex.foreach { case (part, index) => properties.put(s"$DATASOURCE_SCHEMA_PART_PREFIX$index", part) } @@ -280,122 +230,15 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean properties.toMap } - private def validateTable(table: CatalogTable): Unit = { - val options = table.storage.properties - // validate the pk if it exist in the table. - HoodieOptionConfig.getPrimaryColumns(options).foreach(pk => table.schema.fieldIndex(pk)) - // validate the version column if it exist in the table. - HoodieOptionConfig.getPreCombineField(options).foreach(v => table.schema.fieldIndex(v)) - // validate the partition columns - table.partitionColumnNames.foreach(p => table.schema.fieldIndex(p)) - // validate table type - options.get(HoodieOptionConfig.SQL_KEY_TABLE_TYPE.sqlKeyName).foreach { tableType => - ValidationUtils.checkArgument( - tableType.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW) || - tableType.equalsIgnoreCase(HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR), - s"'type' must be '${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_COW}' or " + - s"'${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR}'") - } - } - - def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean, - originTableConfig: Map[String, String] = Map.empty): Map[String, String] = { - val extraConfig = mutable.Map.empty[String, String] - if (isTableExists) { - val allPartitionPaths = getAllPartitionPaths(sparkSession, table) - if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) { - extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = - originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) - } else { - extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = - String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table)) - } - if (originTableConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)) { - extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = - originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) - } else { - extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = - String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table)) - } - } else { - extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true" - extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue() - } - - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(originTableConfig ++ table.storage.properties) - if (primaryColumns.isEmpty) { - extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[UuidKeyGenerator].getCanonicalName - } else if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) { - extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = - HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator( - originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) - } else { - extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[ComplexKeyGenerator].getCanonicalName - } - extraConfig.toMap - } -} - -object CreateHoodieTableCommand extends Logging { - - /** - * Init the hoodie.properties. - */ - def initTableIfNeed(sparkSession: SparkSession, table: CatalogTable): Unit = { - val location = getTableLocation(table, sparkSession) - - val conf = sparkSession.sessionState.newHadoopConf() - // Init the hoodie table - val originTableConfig = if (tableExistsInPath(location, conf)) { - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(location) - .setConf(conf) - .build() - metaClient.getTableConfig.getProps.asScala.toMap - } else { - Map.empty[String, String] - } - - val tableName = table.identifier.table - logInfo(s"Init hoodie.properties for $tableName") - val tableOptions = HoodieOptionConfig.mappingSqlOptionToTableConfig(table.storage.properties) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key) - checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) - // Save all the table config to the hoodie.properties. - val parameters = originTableConfig ++ tableOptions - val properties = new Properties() - properties.putAll(parameters.asJava) - HoodieTableMetaClient.withPropertyBuilder() - .fromProperties(properties) - .setTableName(tableName) - .setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString()) - .setPartitionFields(table.partitionColumnNames.mkString(",")) - .initTable(conf, location) - } - - def checkTableConfigEqual(originTableConfig: Map[String, String], - newTableConfig: Map[String, String], configKey: String): Unit = { + private def checkTableConfigEqual( + originTableConfig: Map[String, String], + newTableConfig: Map[String, String], + configKey: String): Unit = { if (originTableConfig.contains(configKey) && newTableConfig.contains(configKey)) { assert(originTableConfig(configKey) == newTableConfig(configKey), s"Table config: $configKey in the create table is: ${newTableConfig(configKey)}, is not the same with the value in " + s"hoodie.properties, which is: ${originTableConfig(configKey)}. Please keep the same.") } } - - /** - * Check if this is a empty table path. - */ - def isEmptyPath(tablePath: String, conf: Configuration): Boolean = { - val basePath = new Path(tablePath) - val fs = basePath.getFileSystem(conf) - if (fs.exists(basePath)) { - fs.listStatus(basePath).isEmpty - } else { - true - } - } } + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala index 987ce0e050be..a77acf066a76 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala @@ -18,18 +18,18 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _} -import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} + import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable -import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hudi.HoodieOptionConfig +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable +import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, LogicalPlan} import org.apache.spark.sql.hudi.HoodieSqlUtils._ +import org.apache.spark.sql.types.StructType -case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends RunnableCommand +case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends HoodieLeafRunnableCommand with SparkAdapterSupport { private val table = deleteTable.table @@ -56,32 +56,30 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab } private def buildHoodieConfig(sparkSession: SparkSession): Map[String, String] = { - val targetTable = sparkSession.sessionState.catalog - .getTableMetadata(tableId) - val path = getTableLocation(targetTable, sparkSession) - val conf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - val tableConfig = metaClient.getTableConfig - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableId) + val path = hoodieCatalogTable.tableLocation + val tableConfig = hoodieCatalogTable.tableConfig + val tableSchema = hoodieCatalogTable.tableSchema + val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) + val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + val primaryColumns = tableConfig.getRecordKeyFields.get() assert(primaryColumns.nonEmpty, s"There are no primary key defined in table $tableId, cannot execute delete operator") - withSparkConf(sparkSession, targetTable.storage.properties) { + withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) { Map( "path" -> path, - TBL_NAME.key -> tableId.table, + TBL_NAME.key -> tableConfig.getTableName, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, - URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, + URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala new file mode 100644 index 000000000000..aa9d9b8123d8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala @@ -0,0 +1,129 @@ +/* + * 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 org.apache.spark.sql.hudi.command + +import org.apache.hadoop.fs.Path +import org.apache.hudi.client.common.HoodieSparkEngineContext +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieTableType +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, HoodieCatalogTable} +import org.apache.spark.sql.hive.HiveClientUtils +import org.apache.spark.sql.hudi.HoodieSqlUtils.isEnableHive + +import scala.util.control.NonFatal + +case class DropHoodieTableCommand( + tableIdentifier: TableIdentifier, + ifExists: Boolean, + isView: Boolean, + purge: Boolean) +extends HoodieLeafRunnableCommand { + + val MOR_SNAPSHOT_TABLE_SUFFIX = "_rt" + val MOR_READ_OPTIMIZED_TABLE_SUFFIX = "_ro" + + override def run(sparkSession: SparkSession): Seq[Row] = { + val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}" + logInfo(s"start execute drop table command for $fullTableName") + sparkSession.catalog.refreshTable(tableIdentifier.unquotedString) + + try { + // drop catalog table for this hoodie table + dropTableInCatalog(sparkSession, tableIdentifier, ifExists, purge) + } catch { + case NonFatal(e) => + logWarning(s"Failed to drop catalog table in metastore: ${e.getMessage}") + } + + logInfo(s"Finish execute drop table command for $fullTableName") + Seq.empty[Row] + } + + def dropTableInCatalog(sparkSession: SparkSession, + tableIdentifier: TableIdentifier, + ifExists: Boolean, + purge: Boolean): Unit = { + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) + val table = hoodieCatalogTable.table + assert(table.tableType != CatalogTableType.VIEW) + + val basePath = hoodieCatalogTable.tableLocation + val catalog = sparkSession.sessionState.catalog + + // Drop table in the catalog + val enableHive = isEnableHive(sparkSession) + if (enableHive) { + dropHiveDataSourceTable(sparkSession, hoodieCatalogTable) + } else { + if (catalog.tableExists(tableIdentifier)) { + catalog.dropTable(tableIdentifier, ifExists, purge) + } + } + + // Recursively delete table directories + if (purge) { + logInfo("Clean up " + basePath) + val targetPath = new Path(basePath) + val engineContext = new HoodieSparkEngineContext(sparkSession.sparkContext) + val fs = FSUtils.getFs(basePath, sparkSession.sparkContext.hadoopConfiguration) + FSUtils.deleteDir(engineContext, fs, targetPath, sparkSession.sparkContext.defaultParallelism) + } + } + + private def dropHiveDataSourceTable( + sparkSession: SparkSession, + hoodieCatalogTable: HoodieCatalogTable): Unit = { + val table = hoodieCatalogTable.table + val dbName = table.identifier.database.get + val tableName = hoodieCatalogTable.tableName + + // check database exists + val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName) + if (!dbExists) { + throw new NoSuchDatabaseException(dbName) + } + + if (HoodieTableType.MERGE_ON_READ == hoodieCatalogTable.tableType && purge) { + val snapshotTableName = tableName + MOR_SNAPSHOT_TABLE_SUFFIX + val roTableName = tableName + MOR_READ_OPTIMIZED_TABLE_SUFFIX + + dropHiveTable(sparkSession, dbName, snapshotTableName) + dropHiveTable(sparkSession, dbName, roTableName) + } + + dropHiveTable(sparkSession, dbName, tableName, purge) + } + + private def dropHiveTable( + sparkSession: SparkSession, + dbName: String, + tableName: String, + purge: Boolean = false): Unit = { + // check table exists + if (sparkSession.sessionState.catalog.tableExists(new TableIdentifier(tableName, Option(dbName)))) { + val client = HiveClientUtils.newClientForMetadata(sparkSession.sparkContext.conf, + sparkSession.sessionState.newHadoopConf()) + + // drop hive table. + client.dropTable(dbName, tableName, ifExists, purge) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala new file mode 100644 index 000000000000..47e884e962d4 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/HoodieLeafRunnableCommand.scala @@ -0,0 +1,29 @@ +/* + * 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 org.apache.spark.sql.hudi.command + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.HoodieLeafLike +import org.apache.spark.sql.execution.command.RunnableCommand + +/** + * Similar to `LeafRunnableCommand` in Spark3.2, `HoodieLeafRunnableCommand` mixed in + * `HoodieLeafLike` can avoid subclasses of `RunnableCommand` to override + * the `withNewChildrenInternal` method repeatedly. + */ +trait HoodieLeafRunnableCommand extends RunnableCommand with HoodieLeafLike[LogicalPlan] diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala index 2b88373115b7..37d30c813588 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema import org.apache.avro.generic.{GenericRecord, IndexedRecord} + import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord} -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME @@ -30,29 +30,30 @@ import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.sql.InsertMode -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils} +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter} + import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable} import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} import java.util.Properties +import scala.collection.JavaConverters._ + /** * Command for insert into hoodie table. */ case class InsertIntoHoodieTableCommand( - logicalRelation: LogicalRelation, - query: LogicalPlan, - partition: Map[String, Option[String]], - overwrite: Boolean) - extends RunnableCommand { + logicalRelation: LogicalRelation, + query: LogicalPlan, + partition: Map[String, Option[String]], + overwrite: Boolean) + extends HoodieLeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { assert(logicalRelation.catalogTable.isDefined, "Missing catalog table") @@ -78,14 +79,18 @@ object InsertIntoHoodieTableCommand extends Logging { * @param refreshTable Whether to refresh the table after insert finished. * @param extraOptions Extra options for insert. */ - def run(sparkSession: SparkSession, table: CatalogTable, query: LogicalPlan, - insertPartitions: Map[String, Option[String]], - overwrite: Boolean, refreshTable: Boolean = true, - extraOptions: Map[String, String] = Map.empty): Boolean = { + def run(sparkSession: SparkSession, + table: CatalogTable, + query: LogicalPlan, + insertPartitions: Map[String, Option[String]], + overwrite: Boolean, + refreshTable: Boolean = true, + extraOptions: Map[String, String] = Map.empty): Boolean = { - val config = buildHoodieInsertConfig(table, sparkSession, overwrite, insertPartitions, extraOptions) + val hoodieCatalogTable = new HoodieCatalogTable(sparkSession, table) + val config = buildHoodieInsertConfig(hoodieCatalogTable, sparkSession, overwrite, insertPartitions, extraOptions) - val mode = if (overwrite && table.partitionColumnNames.isEmpty) { + val mode = if (overwrite && hoodieCatalogTable.partitionFields.isEmpty) { // insert overwrite non-partition table SaveMode.Overwrite } else { @@ -93,7 +98,7 @@ object InsertIntoHoodieTableCommand extends Logging { SaveMode.Append } val conf = sparkSession.sessionState.conf - val alignedQuery = alignOutputFields(query, table, insertPartitions, conf) + val alignedQuery = alignOutputFields(query, hoodieCatalogTable, insertPartitions, conf) // If we create dataframe using the Dataset.ofRows(sparkSession, alignedQuery), // The nullable attribute of fields will lost. // In order to pass the nullable attribute to the inputDF, we specify the schema @@ -115,18 +120,18 @@ object InsertIntoHoodieTableCommand extends Logging { /** * Aligned the type and name of query's output fields with the result table's fields. * @param query The insert query which to aligned. - * @param table The result table. + * @param hoodieCatalogTable The result hoodie catalog table. * @param insertPartitions The insert partition map. * @param conf The SQLConf. * @return */ private def alignOutputFields( query: LogicalPlan, - table: CatalogTable, + hoodieCatalogTable: HoodieCatalogTable, insertPartitions: Map[String, Option[String]], conf: SQLConf): LogicalPlan = { - val targetPartitionSchema = table.partitionSchema + val targetPartitionSchema = hoodieCatalogTable.partitionSchema val staticPartitionValues = insertPartitions.filter(p => p._2.isDefined).mapValues(_.get) assert(staticPartitionValues.isEmpty || @@ -134,20 +139,22 @@ object InsertIntoHoodieTableCommand extends Logging { s"Required partition columns is: ${targetPartitionSchema.json}, Current static partitions " + s"is: ${staticPartitionValues.mkString("," + "")}") - assert(staticPartitionValues.size + query.output.size == table.schema.size, - s"Required select columns count: ${removeMetaFields(table.schema).size}, " + + val queryOutputWithoutMetaFields = removeMetaFields(query.output) + assert(staticPartitionValues.size + queryOutputWithoutMetaFields.size + == hoodieCatalogTable.tableSchemaWithoutMetaFields.size, + s"Required select columns count: ${hoodieCatalogTable.tableSchemaWithoutMetaFields.size}, " + s"Current select columns(including static partition column) count: " + - s"${staticPartitionValues.size + removeMetaFields(query.output).size},columns: " + - s"(${(removeMetaFields(query.output).map(_.name) ++ staticPartitionValues.keys).mkString(",")})") - val queryDataFields = if (staticPartitionValues.isEmpty) { // insert dynamic partition - query.output.dropRight(targetPartitionSchema.fields.length) + s"${staticPartitionValues.size + queryOutputWithoutMetaFields.size},columns: " + + s"(${(queryOutputWithoutMetaFields.map(_.name) ++ staticPartitionValues.keys).mkString(",")})") + + val queryDataFieldsWithoutMetaFields = if (staticPartitionValues.isEmpty) { // insert dynamic partition + queryOutputWithoutMetaFields.dropRight(targetPartitionSchema.fields.length) } else { // insert static partition - query.output + queryOutputWithoutMetaFields } - val targetDataSchema = table.dataSchema // Align for the data fields of the query - val dataProjects = queryDataFields.zip(targetDataSchema.fields).map { - case (dataAttr, targetField) => + val dataProjectsWithputMetaFields = queryDataFieldsWithoutMetaFields.zip( + hoodieCatalogTable.dataSchemaWithoutMetaFields.fields).map { case (dataAttr, targetField) => val castAttr = castIfNeeded(dataAttr.withNullability(targetField.nullable), targetField.dataType, conf) Alias(castAttr, targetField.name)() @@ -156,9 +163,9 @@ object InsertIntoHoodieTableCommand extends Logging { val partitionProjects = if (staticPartitionValues.isEmpty) { // insert dynamic partitions // The partition attributes is followed the data attributes in the query // So we init the partitionAttrPosition with the data schema size. - var partitionAttrPosition = targetDataSchema.size + var partitionAttrPosition = hoodieCatalogTable.dataSchemaWithoutMetaFields.size targetPartitionSchema.fields.map(f => { - val partitionAttr = query.output(partitionAttrPosition) + val partitionAttr = queryOutputWithoutMetaFields(partitionAttrPosition) partitionAttrPosition = partitionAttrPosition + 1 val castAttr = castIfNeeded(partitionAttr.withNullability(f.nullable), f.dataType, conf) Alias(castAttr, f.name)() @@ -171,9 +178,7 @@ object InsertIntoHoodieTableCommand extends Logging { Alias(castAttr, f.name)() }) } - // Remove the hoodie meta fields from the projects as we do not need these to write - val withoutMetaFieldDataProjects = dataProjects.filter(c => !HoodieSqlUtils.isMetaField(c.name)) - val alignedProjects = withoutMetaFieldDataProjects ++ partitionProjects + val alignedProjects = dataProjectsWithputMetaFields ++ partitionProjects Project(alignedProjects, query) } @@ -182,95 +187,65 @@ object InsertIntoHoodieTableCommand extends Logging { * @return */ private def buildHoodieInsertConfig( - table: CatalogTable, + hoodieCatalogTable: HoodieCatalogTable, sparkSession: SparkSession, isOverwrite: Boolean, insertPartitions: Map[String, Option[String]] = Map.empty, extraOptions: Map[String, String]): Map[String, String] = { if (insertPartitions.nonEmpty && - (insertPartitions.keys.toSet != table.partitionColumnNames.toSet)) { + (insertPartitions.keys.toSet != hoodieCatalogTable.partitionFields.toSet)) { throw new IllegalArgumentException(s"Insert partition fields" + s"[${insertPartitions.keys.mkString(" " )}]" + - s" not equal to the defined partition in table[${table.partitionColumnNames.mkString(",")}]") + s" not equal to the defined partition in table[${hoodieCatalogTable.partitionFields.mkString(",")}]") } - val options = table.storage.properties ++ extraOptions - val parameters = withSparkConf(sparkSession, options)() - - val tableType = parameters.getOrElse(TABLE_TYPE.key, TABLE_TYPE.defaultValue) - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(options) - val partitionFields = table.partitionColumnNames.mkString(",") + val path = hoodieCatalogTable.tableLocation + val tableType = hoodieCatalogTable.tableTypeName + val tableConfig = hoodieCatalogTable.tableConfig + val tableSchema = hoodieCatalogTable.tableSchema - val path = getTableLocation(table, sparkSession) - val conf = sparkSession.sessionState.newHadoopConf() - val isTableExists = tableExistsInPath(path, conf) - val tableConfig = if (isTableExists) { - HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - .getTableConfig - } else { - null - } - val hiveStylePartitioningEnable = if (null == tableConfig || null == tableConfig.getHiveStylePartitioningEnable) { - "true" - } else { - tableConfig.getHiveStylePartitioningEnable - } - val urlEncodePartitioning = if (null == tableConfig || null == tableConfig.getUrlEncodePartitoning) { - "false" - } else { - tableConfig.getUrlEncodePartitoning - } - val keyGeneratorClassName = if (null == tableConfig || null == tableConfig.getKeyGeneratorClassName) { - if (primaryColumns.nonEmpty) { - classOf[ComplexKeyGenerator].getCanonicalName - } else { - classOf[UuidKeyGenerator].getCanonicalName - } - } else { - tableConfig.getKeyGeneratorClassName - } + val options = hoodieCatalogTable.catalogProperties ++ tableConfig.getProps.asScala.toMap ++ extraOptions + val parameters = withSparkConf(sparkSession, options)() - val tableSchema = table.schema + val preCombineColumn = hoodieCatalogTable.preCombineKey.getOrElse("") + val partitionFields = hoodieCatalogTable.partitionFields.mkString(",") - val dropDuplicate = sparkSession.conf - .getOption(INSERT_DROP_DUPS.key) - .getOrElse(INSERT_DROP_DUPS.defaultValue) - .toBoolean + val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true") + val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitioning).getOrElse("false") + val keyGeneratorClassName = Option(tableConfig.getKeyGeneratorClassName) + .getOrElse(classOf[ComplexKeyGenerator].getCanonicalName) val enableBulkInsert = parameters.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key, DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean - val isPartitionedTable = table.partitionColumnNames.nonEmpty - val isPrimaryKeyTable = primaryColumns.nonEmpty + val dropDuplicate = sparkSession.conf + .getOption(INSERT_DROP_DUPS.key).getOrElse(INSERT_DROP_DUPS.defaultValue).toBoolean + val insertMode = InsertMode.of(parameters.getOrElse(DataSourceWriteOptions.SQL_INSERT_MODE.key, DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue())) val isNonStrictMode = insertMode == InsertMode.NON_STRICT - + val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty + val hasPrecombineColumn = preCombineColumn.nonEmpty val operation = - (isPrimaryKeyTable, enableBulkInsert, isOverwrite, dropDuplicate) match { - case (true, true, _, _) if !isNonStrictMode => + (enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match { + case (true, _, _, false, _) => throw new IllegalArgumentException(s"Table with primaryKey can not use bulk insert in ${insertMode.value()} mode.") - case (_, true, true, _) if isPartitionedTable => + case (true, true, _, _, true) => throw new IllegalArgumentException(s"Insert Overwrite Partition can not use bulk insert.") - case (_, true, _, true) => + case (true, _, true, _, _) => throw new IllegalArgumentException(s"Bulk insert cannot support drop duplication." + s" Please disable $INSERT_DROP_DUPS and try again.") // if enableBulkInsert is true, use bulk insert for the insert overwrite non-partitioned table. - case (_, true, true, _) if !isPartitionedTable => BULK_INSERT_OPERATION_OPT_VAL - // insert overwrite partition - case (_, _, true, _) if isPartitionedTable => INSERT_OVERWRITE_OPERATION_OPT_VAL + case (true, true, _, _, false) => BULK_INSERT_OPERATION_OPT_VAL // insert overwrite table - case (_, _, true, _) if !isPartitionedTable => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL - // if it is pk table and the dropDuplicate has disable, use the upsert operation for strict and upsert mode. - case (true, false, false, false) if !isNonStrictMode => UPSERT_OPERATION_OPT_VAL - // if enableBulkInsert is true and the table is non-primaryKeyed, use the bulk insert operation - case (false, true, _, _) => BULK_INSERT_OPERATION_OPT_VAL + case (false, true, _, _, false) => INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL + // insert overwrite partition + case (_, true, _, _, true) => INSERT_OVERWRITE_OPERATION_OPT_VAL + // disable dropDuplicate, and provide preCombineKey, use the upsert operation for strict and upsert mode. + case (false, false, false, false, _) if hasPrecombineColumn => UPSERT_OPERATION_OPT_VAL // if table is pk table and has enableBulkInsert use bulk insert for non-strict mode. - case (true, true, _, _) if isNonStrictMode => BULK_INSERT_OPERATION_OPT_VAL + case (true, _, _, true, _) => BULK_INSERT_OPERATION_OPT_VAL // for the rest case, use the insert operation - case (_, _, _, _) => INSERT_OPERATION_OPT_VAL + case _ => INSERT_OPERATION_OPT_VAL } val payloadClassName = if (operation == UPSERT_OPERATION_OPT_VAL && @@ -279,7 +254,7 @@ object InsertIntoHoodieTableCommand extends Logging { // on reading. classOf[ValidateDuplicateKeyPayload].getCanonicalName } else { - classOf[DefaultHoodieRecordPayload].getCanonicalName + classOf[OverwriteWithLatestAvroPayload].getCanonicalName } logInfo(s"insert statement use write operation type: $operation, payloadClass: $payloadClassName") @@ -288,28 +263,29 @@ object InsertIntoHoodieTableCommand extends Logging { Map( "path" -> path, TABLE_TYPE.key -> tableType, - TBL_NAME.key -> table.identifier.table, - PRECOMBINE_FIELD.key -> tableSchema.fields.last.name, + TBL_NAME.key -> hoodieCatalogTable.tableName, + PRECOMBINE_FIELD.key -> preCombineColumn, OPERATION.key -> operation, HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable, URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning, - KEYGENERATOR_CLASS_NAME.key -> keyGeneratorClassName, - RECORDKEY_FIELD.key -> primaryColumns.mkString(","), + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> keyGeneratorClassName, + RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), PARTITIONPATH_FIELD.key -> partitionFields, PAYLOAD_CLASS_NAME.key -> payloadClassName, ENABLE_ROW_WRITER.key -> enableBulkInsert.toString, - HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> isPrimaryKeyTable.toString, + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", - HIVE_DATABASE.key -> table.identifier.database.getOrElse("default"), - HIVE_TABLE.key -> table.identifier.table, + HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"), + HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HIVE_PARTITION_FIELDS.key -> partitionFields, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) } } @@ -327,7 +303,7 @@ class ValidateDuplicateKeyPayload(record: GenericRecord, orderingVal: Comparable } override def combineAndGetUpdateValue(currentValue: IndexedRecord, - schema: Schema, properties: Properties): HOption[IndexedRecord] = { + schema: Schema, properties: Properties): HOption[IndexedRecord] = { val key = currentValue.asInstanceOf[GenericRecord].get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString throw new HoodieDuplicateKeyException(key) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 5ec15ce4d84f..2d36c6c31fb3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -19,25 +19,28 @@ package org.apache.spark.sql.hudi.command import org.apache.avro.Schema import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils, SparkAdapterSupport} + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BoundReference, Cast, EqualTo, Expression, Literal} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.hudi.HoodieSqlUtils._ import org.apache.spark.sql.hudi.command.payload.ExpressionPayload import org.apache.spark.sql.hudi.command.payload.ExpressionPayload._ -import org.apache.spark.sql.hudi.{HoodieOptionConfig, SerDeUtils} +import org.apache.spark.sql.hudi.SerDeUtils import org.apache.spark.sql.types.{BooleanType, StructType} + import java.util.Base64 + /** * The Command for hoodie MergeIntoTable. * The match on condition must contain the row key fields currently, so that we can use Hoodie @@ -56,7 +59,7 @@ import java.util.Base64 * ExpressionPayload#getInsertValue. * */ -case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends RunnableCommand +case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends HoodieLeafRunnableCommand with SparkAdapterSupport { private var sparkSession: SparkSession = _ @@ -77,11 +80,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab private lazy val targetTableSchemaWithoutMetaFields = removeMetaFields(mergeInto.targetTable.schema).fields - private lazy val targetTable = - sparkSession.sessionState.catalog.getTableMetadata(targetTableIdentify) + private lazy val hoodieCatalogTable = HoodieCatalogTable(sparkSession, targetTableIdentify) - private lazy val targetTableType = - HoodieOptionConfig.getTableType(targetTable.storage.properties) + private lazy val targetTableType = hoodieCatalogTable.tableTypeName /** * @@ -124,7 +125,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab assert(updateActions.size <= 1, s"Only support one updateAction currently, current update action count is: ${updateActions.size}") val updateAction = updateActions.headOption - HoodieOptionConfig.getPreCombineField(targetTable.storage.properties).map(preCombineField => { + hoodieCatalogTable.preCombineKey.map(preCombineField => { val sourcePreCombineField = updateAction.map(u => u.assignments.filter { case Assignment(key: AttributeReference, _) => key.name.equalsIgnoreCase(preCombineField) @@ -147,9 +148,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab this.sparkSession = sparkSession // Create the write parameters - val parameters = buildMergeIntoConfig(mergeInto) - - val sourceDF = buildSourceDF(sparkSession) + val parameters = buildMergeIntoConfig(hoodieCatalogTable) if (mergeInto.matchedActions.nonEmpty) { // Do the upsert executeUpsert(sourceDF, parameters) @@ -178,7 +177,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab * row key and pre-combine field. * */ - private def buildSourceDF(sparkSession: SparkSession): DataFrame = { + private lazy val sourceDF: DataFrame = { var sourceDF = Dataset.ofRows(sparkSession, mergeInto.sourceTable) targetKey2SourceExpression.foreach { case (targetColumn, sourceExpression) @@ -203,7 +202,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab sourceExpression match { case attr: AttributeReference if sourceColumnName.find(resolver(_, attr.name)).get.equals(targetColumnName) => true - case Cast(attr: AttributeReference, _, _) if sourceColumnName.find(resolver(_, attr.name)).get.equals(targetColumnName) => true + // SPARK-35857: the definition of Cast has been changed in Spark3.2. + // Match the class type instead of call the `unapply` method. + case cast: Cast => + cast.child match { + case attr: AttributeReference if sourceColumnName.find(resolver(_, attr.name)).get.equals(targetColumnName) => true + case _ => false + } case _=> false } } @@ -242,8 +247,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab // Append the table schema to the parameters. In the case of merge into, the schema of sourceDF // may be different from the target table, because the are transform logical in the update or // insert actions. + val operation = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) { + INSERT_OPERATION_OPT_VAL + } else { + UPSERT_OPERATION_OPT_VAL + } var writeParams = parameters + - (OPERATION.key -> UPSERT_OPERATION_OPT_VAL) + + (OPERATION.key -> operation) + (HoodieWriteConfig.WRITE_SCHEMA.key -> getTableSchema.toString) + (DataSourceWriteOptions.TABLE_TYPE.key -> targetTableType) @@ -422,52 +432,43 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab /** * Create the config for hoodie writer. - * @param mergeInto - * @return */ - private def buildMergeIntoConfig(mergeInto: MergeIntoTable): Map[String, String] = { + private def buildMergeIntoConfig(hoodieCatalogTable: HoodieCatalogTable): Map[String, String] = { val targetTableDb = targetTableIdentify.database.getOrElse("default") val targetTableName = targetTableIdentify.identifier - val path = getTableLocation(targetTable, sparkSession) - val conf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - val tableConfig = metaClient.getTableConfig - val options = targetTable.storage.properties - val definedPk = HoodieOptionConfig.getPrimaryColumns(options) - // TODO Currently the mergeEqualConditionKeys must be the same the primary key. - if (targetKey2SourceExpression.keySet != definedPk.toSet) { - throw new IllegalArgumentException(s"Merge Key[${targetKey2SourceExpression.keySet.mkString(",")}] is not" + - s" Equal to the defined primary key[${definedPk.mkString(",")}] in table $targetTableName") - } + val path = hoodieCatalogTable.tableLocation + val tableConfig = hoodieCatalogTable.tableConfig + val tableSchema = hoodieCatalogTable.tableSchema + val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) + val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name))) + // Enable the hive sync by default if spark have enable the hive metastore. val enableHive = isEnableHive(sparkSession) - withSparkConf(sparkSession, options) { + withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) { Map( "path" -> path, - RECORDKEY_FIELD.key -> targetKey2SourceExpression.keySet.mkString(","), - PRECOMBINE_FIELD.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field + RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp, + PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), TBL_NAME.key -> targetTableName, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, - URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, + URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", HIVE_DATABASE.key -> targetTableDb, HIVE_TABLE.key -> targetTableName, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", - HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), + HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala index 1c1f4b73d0da..6a3eff8b4c35 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala @@ -17,52 +17,41 @@ package org.apache.spark.sql.hudi.command -import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.PartitionPathEncodeUtils + import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.PartitioningUtils -import org.apache.spark.sql.hudi.HoodieSqlUtils._ import org.apache.spark.sql.types.StringType /** * Command for show hudi table's partitions. */ case class ShowHoodieTablePartitionsCommand( - tableName: TableIdentifier, + tableIdentifier: TableIdentifier, specOpt: Option[TablePartitionSpec]) -extends RunnableCommand { +extends HoodieLeafRunnableCommand { override val output: Seq[Attribute] = { AttributeReference("partition", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val resolver = sparkSession.sessionState.conf.resolver - val catalogTable = catalog.getTableMetadata(tableName) - val tablePath = getTableLocation(catalogTable, sparkSession) - - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder().setBasePath(tablePath) - .setConf(hadoopConf).build() - val schemaOpt = getTableSqlSchema(metaClient) - val partitionColumnNamesOpt = metaClient.getTableConfig.getPartitionFields - if (partitionColumnNamesOpt.isPresent && partitionColumnNamesOpt.get.nonEmpty - && schemaOpt.isDefined && schemaOpt.nonEmpty) { + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) - val partitionColumnNames = partitionColumnNamesOpt.get - val schema = schemaOpt.get - val allPartitionPaths: Seq[String] = getAllPartitionPaths(sparkSession, catalogTable) + val schemaOpt = hoodieCatalogTable.tableSchema + val partitionColumnNamesOpt = hoodieCatalogTable.tableConfig.getPartitionFields + if (partitionColumnNamesOpt.isPresent && partitionColumnNamesOpt.get.nonEmpty && schemaOpt.nonEmpty) { if (specOpt.isEmpty) { - allPartitionPaths.map(Row(_)) + hoodieCatalogTable.getAllPartitionPaths.map(Row(_)) } else { val spec = specOpt.get - allPartitionPaths.filter { partitionPath => + hoodieCatalogTable.getAllPartitionPaths.filter { partitionPath => val part = PartitioningUtils.parsePathFragment(partitionPath) spec.forall { case (col, value) => PartitionPathEncodeUtils.escapePartitionValue(value) == part.getOrElse(col, null) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala index 2cc3ecee80b8..e069df97aff5 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hudi.command -import java.sql.Timestamp import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS} import org.apache.avro.generic.GenericRecord @@ -97,7 +96,7 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props) val timeMs = if (rowType) { // In RowType, the partitionPathValue is the time format string, convert to millis SqlKeyGenerator.sqlTimestampFormat.parseMillis(_partitionValue) } else { - Timestamp.valueOf(_partitionValue).getTime + MILLISECONDS.convert(_partitionValue.toLong, MICROSECONDS) } val timestampFormat = PartitionPathEncodeUtils.escapePathName( SqlKeyGenerator.timestampTimeFormat.print(timeMs)) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala index 339f4b52cba0..12ec22499db7 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala @@ -18,46 +18,50 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.common.table.HoodieTableMetaClient + import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.execution.command.TruncateTableCommand -import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation + +import scala.util.control.NonFatal /** * Command for truncate hudi table. */ class TruncateHoodieTableCommand( - tableName: TableIdentifier, + tableIdentifier: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) - extends TruncateTableCommand(tableName, partitionSpec) { + extends TruncateTableCommand(tableIdentifier, partitionSpec) { override def run(sparkSession: SparkSession): Seq[Row] = { - val table = sparkSession.sessionState.catalog.getTableMetadata(tableName) - val path = getTableLocation(table, sparkSession) - val hadoopConf = sparkSession.sessionState.newHadoopConf() - // If we have not specified the partition, truncate will delete all the - // data in the table path include the hoodi.properties. In this case we - // should reInit the table. - val needReInitTable = partitionSpec.isEmpty - - val tableProperties = if (needReInitTable) { - // Create MetaClient - val metaClient = HoodieTableMetaClient.builder().setBasePath(path) - .setConf(hadoopConf).build() - Some(metaClient.getTableConfig.getProps) - } else { - None + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) + val properties = hoodieCatalogTable.tableConfig.getProps + + try { + // Delete all data in the table directory + super.run(sparkSession) + } catch { + // TruncateTableCommand will delete the related directories first, and then refresh the table. + // It will fail when refresh table, because the hudi meta directory(.hoodie) has been deleted at the first step. + // So here ignore this failure, and refresh table later. + case NonFatal(_) => } - // Delete all data in the table directory - super.run(sparkSession) - if (tableProperties.isDefined) { + // If we have not specified the partition, truncate will delete all the data in the table path + // include the hoodi.properties. In this case we should reInit the table. + if (partitionSpec.isEmpty) { + val hadoopConf = sparkSession.sessionState.newHadoopConf() // ReInit hoodie.properties HoodieTableMetaClient.withPropertyBuilder() - .fromProperties(tableProperties.get) - .initTable(hadoopConf, path) + .fromProperties(properties) + .initTable(hadoopConf, hoodieCatalogTable.tableLocation) } + + // After deleting the data, refresh the table to make sure we don't keep around a stale + // file relation in the metastore cache and cached table data in the cache manager. + sparkSession.catalog.refreshTable(hoodieCatalogTable.table.identifier.quotedString) Seq.empty[Row] } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala index b1c8a04429e2..0ff7ffb45203 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/UpdateHoodieTableCommand.scala @@ -18,24 +18,24 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.common.model.HoodieRecord -import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.ddl.HiveSyncMode -import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} + import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, UpdateTable} -import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hudi.HoodieOptionConfig +import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan, UpdateTable} import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StructField, StructType} import scala.collection.JavaConverters._ -case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCommand +case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends HoodieLeafRunnableCommand with SparkAdapterSupport { private val table = updateTable.table @@ -43,16 +43,17 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo override def run(sparkSession: SparkSession): Seq[Row] = { logInfo(s"start execute update command for $tableId") - def cast(exp:Expression, field: StructField): Expression = { - castIfNeeded(exp, field.dataType, sparkSession.sqlContext.conf) - } + val sqlConf = sparkSession.sessionState.conf val name2UpdateValue = updateTable.assignments.map { case Assignment(attr: AttributeReference, value) => attr.name -> value }.toMap val updateExpressions = table.output - .map(attr => name2UpdateValue.getOrElse(attr.name, attr)) + .map(attr => { + val UpdateValueOption = name2UpdateValue.find(f => sparkSession.sessionState.conf.resolver(f._1, attr.name)) + if(UpdateValueOption.isEmpty) attr else UpdateValueOption.get._2 + }) .filter { // filter the meta columns case attr: AttributeReference => !HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet.contains(attr.name) @@ -61,9 +62,9 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo val projects = updateExpressions.zip(removeMetaFields(table.schema).fields).map { case (attr: AttributeReference, field) => - Column(cast(attr, field)) + Column(cast(attr, field, sqlConf)) case (exp, field) => - Column(Alias(cast(exp, field), field.name)()) + Column(Alias(cast(exp, field, sqlConf), field.name)()) } var df = Dataset.ofRows(sparkSession, table) @@ -83,42 +84,42 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo } private def buildHoodieConfig(sparkSession: SparkSession): Map[String, String] = { - val targetTable = sparkSession.sessionState.catalog - .getTableMetadata(tableId) - val path = getTableLocation(targetTable, sparkSession) - val conf = sparkSession.sessionState.newHadoopConf() - val metaClient = HoodieTableMetaClient.builder() - .setBasePath(path) - .setConf(conf) - .build() - val tableConfig = metaClient.getTableConfig - val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableId) + val catalogProperties = hoodieCatalogTable.catalogProperties + val tableConfig = hoodieCatalogTable.tableConfig - assert(primaryColumns.nonEmpty, + val preCombineColumn = Option(tableConfig.getPreCombineField).getOrElse("") + assert(hoodieCatalogTable.primaryKeys.nonEmpty, s"There are no primary key in table $tableId, cannot execute update operator") val enableHive = isEnableHive(sparkSession) - withSparkConf(sparkSession, targetTable.storage.properties) { + + withSparkConf(sparkSession, catalogProperties) { Map( - "path" -> path, - RECORDKEY_FIELD.key -> primaryColumns.mkString(","), - PRECOMBINE_FIELD.key -> primaryColumns.head, //set the default preCombine field. - TBL_NAME.key -> tableId.table, + "path" -> hoodieCatalogTable.tableLocation, + RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), + PRECOMBINE_FIELD.key -> preCombineColumn, + TBL_NAME.key -> hoodieCatalogTable.tableName, HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable, - URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning, - KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName, - OPERATION.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, - PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), + URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, + KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, + OPERATION.key -> UPSERT_OPERATION_OPT_VAL, + PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, META_SYNC_ENABLED.key -> enableHive.toString, HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_USE_JDBC.key -> "false", HIVE_DATABASE.key -> tableId.database.getOrElse("default"), HIVE_TABLE.key -> tableId.table, - HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), + HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", - SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) } } + + def cast(exp:Expression, field: StructField, sqlConf: SQLConf): Expression = { + castIfNeeded(exp, field.dataType, sqlConf) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index b025cf3efa44..e660fe870b80 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -19,10 +19,13 @@ package org.apache.spark.sql.hudi.command.payload import java.util.{Base64, Properties} import java.util.concurrent.Callable -import scala.collection.JavaConverters._ + import com.google.common.cache.CacheBuilder + import org.apache.avro.Schema import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} + +import org.apache.hudi.AvroConversionUtils import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro @@ -31,12 +34,14 @@ import org.apache.hudi.common.util.{ValidationUtils, Option => HOption} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.io.HoodieWriteHandle import org.apache.hudi.sql.IExpressionEvaluator + import org.apache.spark.sql.avro.{AvroSerializer, HoodieAvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.hudi.SerDeUtils import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.getEvaluator import org.apache.spark.sql.types.{StructField, StructType} +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer /** @@ -309,7 +314,7 @@ object ExpressionPayload { SchemaConverters.toAvroType(conditionType), false) val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer) - val assignSqlType = SchemaConverters.toSqlType(writeSchema).dataType.asInstanceOf[StructType] + val assignSqlType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema) val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false) val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer) conditionEvaluator -> assignmentEvaluator diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala index 3fb48f430221..749761443547 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala @@ -19,17 +19,19 @@ package org.apache.spark.sql.hudi.command.payload import org.apache.avro.generic.IndexedRecord import org.apache.avro.Schema -import org.apache.spark.sql.avro.{HooodieAvroDeserializer, SchemaConverters} + +import org.apache.hudi.AvroConversionUtils + +import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.types._ /** * A sql typed record which will convert the avro field to sql typed value. */ class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord { - private lazy val sqlType = SchemaConverters.toSqlType(getSchema).dataType.asInstanceOf[StructType] - private lazy val avroDeserializer = HooodieAvroDeserializer(record.getSchema, sqlType) + private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema) + private lazy val avroDeserializer = HoodieAvroDeserializer(record.getSchema, sqlType) private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow] override def put(i: Int, v: Any): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala index a60a63b7a7d7..ffe9b6498402 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala @@ -22,16 +22,17 @@ import java.nio.charset.StandardCharsets import java.util.Date import org.apache.hadoop.fs.Path -import org.apache.hudi.{DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport} + +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport} import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{FileIOUtils, TablePathUtils} + import org.apache.spark.sql.hudi.streaming.HoodieStreamSource.VERSION import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, Source} @@ -118,8 +119,7 @@ class HoodieStreamSource( override def schema: StructType = { schemaOption.getOrElse { val schemaUtil = new TableSchemaResolver(metaClient) - SchemaConverters.toSqlType(schemaUtil.getTableAvroSchema) - .dataType.asInstanceOf[StructType] + AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) } } @@ -179,10 +179,10 @@ class HoodieStreamSource( startOffset match { case INIT_OFFSET => startOffset.commitTime case HoodieSourceOffset(commitTime) => - val time = HoodieActiveTimeline.parseInstantTime(commitTime).getTime + val time = HoodieActiveTimeline.parseDateFromInstantTime(commitTime).getTime // As we consume the data between (start, end], start is not included, // so we +1s to the start commit time here. - HoodieActiveTimeline.formatInstantTime(new Date(time + 1000)) + HoodieActiveTimeline.formatDate(new Date(time + 1000)) case _=> throw new IllegalStateException("UnKnow offset type.") } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index 0557d70405c4..b6e595c40a8d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -26,6 +26,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.hive.NonPartitionedExtractor; +import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -206,7 +207,7 @@ public void run() throws Exception { .option(DataSourceWriteOptions.OPERATION().key(), "delete") .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "_row_key") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp") .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName() : SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor @@ -270,7 +271,9 @@ private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr"); + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } return writer; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java index cf86ba7ba395..8302ece4b9ae 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java @@ -25,6 +25,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.hive.NonPartitionedExtractor; +import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -140,7 +141,9 @@ private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr"); + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } return writer; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index c047ef19677c..f1e6b45b292b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -28,6 +28,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.MultiPartKeysValueExtractor; +import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; @@ -391,7 +392,9 @@ private DataStreamWriter updateHiveSyncConfig(DataStreamWriter writer) DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr"); + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } return writer; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index 6353aa216512..ae89e8af7547 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -40,10 +40,16 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.DecimalType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; import org.mockito.Captor; @@ -52,7 +58,12 @@ import java.math.BigDecimal; import java.time.LocalDate; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import static org.apache.hudi.DataSourceUtils.mayBeOverwriteParquetWriteLegacyFormatProp; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; import static org.apache.hudi.hive.ddl.HiveSyncMode.HMS; import static org.hamcrest.CoreMatchers.containsString; @@ -274,4 +285,33 @@ public boolean arePartitionRecordsSorted() { return false; } } + + @ParameterizedTest + @CsvSource({"true, false", "true, true", "false, true", "false, false"}) + public void testAutoModifyParquetWriteLegacyFormatParameter(boolean smallDecimal, boolean defaultWriteValue) { + // create test StructType + List structFields = new ArrayList<>(); + if (smallDecimal) { + structFields.add(StructField.apply("d1", DecimalType$.MODULE$.apply(10, 2), false, Metadata.empty())); + } else { + structFields.add(StructField.apply("d1", DecimalType$.MODULE$.apply(38, 10), false, Metadata.empty())); + } + StructType structType = StructType$.MODULE$.apply(structFields); + // create write options + Map options = new HashMap<>(); + options.put("hoodie.parquet.writeLegacyFormat.enabled", String.valueOf(defaultWriteValue)); + + // start test + mayBeOverwriteParquetWriteLegacyFormatProp(options, structType); + + // check result + boolean res = Boolean.parseBoolean(options.get("hoodie.parquet.writeLegacyFormat.enabled")); + if (smallDecimal) { + // should auto modify "hoodie.parquet.writeLegacyFormat.enabled" = "true". + assertEquals(true, res); + } else { + // should not modify the value of "hoodie.parquet.writeLegacyFormat.enabled". + assertEquals(defaultWriteValue, res); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf b/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf new file mode 100644 index 000000000000..c883b5bbe83d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/external-config/hudi-defaults.conf @@ -0,0 +1,23 @@ +# +# 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. +# + +# Default system properties included when running Hudi jobs. +# This is useful for setting default environmental settings. + +# Example: +hoodie.datasource.write.table.type MERGE_ON_READ +hoodie.datasource.write.hive_style_partitioning false diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table-merged.json b/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table-merged.json new file mode 100644 index 000000000000..5c876126ae1d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table-merged.json @@ -0,0 +1,8 @@ +{"c1_maxValue":1000,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 993sdc","c2_minValue":" 1000sdc","c2_num_nulls":0,"c3_maxValue":999.348,"c3_minValue":5.102,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"} +{"c1_maxValue":1000,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 996sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":999.779,"c3_minValue":2.992,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":998,"c1_minValue":2,"c1_num_nulls":0,"c2_maxValue":" 998sdc","c2_minValue":" 104sdc","c2_num_nulls":0,"c3_maxValue":997.905,"c3_minValue":0.876,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"Ag==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"} +{"c1_maxValue":997,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 102sdc","c2_num_nulls":0,"c3_maxValue":990.531,"c3_minValue":2.336,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":994,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":997.496,"c3_minValue":7.742,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"} +{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 999sdc","c2_minValue":" 100sdc","c2_num_nulls":0,"c3_maxValue":980.676,"c3_minValue":0.120,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 10sdc","c2_num_nulls":0,"c3_maxValue":993.940,"c3_minValue":4.598,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"} +{"c1_maxValue":998,"c1_minValue":6,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":999.282,"c3_minValue":1.217,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table.json b/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table.json new file mode 100644 index 000000000000..45cb9aaf88c2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/index/zorder/z-index-table.json @@ -0,0 +1,4 @@ +{"c1_maxValue":1000,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 996sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":999.779,"c3_minValue":2.992,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":997,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 102sdc","c2_num_nulls":0,"c3_maxValue":990.531,"c3_minValue":2.336,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 999sdc","c2_minValue":" 100sdc","c2_num_nulls":0,"c3_maxValue":980.676,"c3_minValue":0.120,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} +{"c1_maxValue":998,"c1_minValue":6,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":999.282,"c3_minValue":1.217,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql index 280fde59ff30..135c83b4b975 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql @@ -18,7 +18,7 @@ set hoodie.delete.shuffle.parallelism = 1; # CTAS -create table h0 using hudi options(type = '${tableType}') +create table h0 using hudi options(type = '${tableType}', primaryKey = 'id') as select 1 as id, 'a1' as name, 10 as price; +----------+ | ok | @@ -30,7 +30,7 @@ select id, name, price from h0; +-----------+ create table h0_p using hudi partitioned by(dt) -options(type = '${tableType}') +options(type = '${tableType}', primaryKey = 'id') as select cast('2021-05-07 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as price; +----------+ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala index 3a4494c144ad..9b1b88d34ce1 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala @@ -19,6 +19,7 @@ package org.apache.hudi import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpressions import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpression + import org.apache.spark.sql.sources.{And, EqualNullSafe, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith} import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType} import org.junit.jupiter.api.Assertions.assertEquals @@ -68,22 +69,36 @@ class TestConvertFilterToCatalystExpression { } private def checkConvertFilter(filter: Filter, expectExpression: String): Unit = { + // [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute, + // AttributeReference and Alias don't quote qualified names properly + val removeQuotesIfNeed = if (expectExpression != null && HoodieSparkUtils.isSpark3_2) { + expectExpression.replace("`", "") + } else { + expectExpression + } val exp = convertToCatalystExpression(filter, tableSchema) - if (expectExpression == null) { + if (removeQuotesIfNeed == null) { assertEquals(exp.isEmpty, true) } else { assertEquals(exp.isDefined, true) - assertEquals(expectExpression, exp.get.sql) + assertEquals(removeQuotesIfNeed, exp.get.sql) } } private def checkConvertFilters(filters: Array[Filter], expectExpression: String): Unit = { + // [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute, + // AttributeReference and Alias don't quote qualified names properly + val removeQuotesIfNeed = if (expectExpression != null && HoodieSparkUtils.isSpark3_2) { + expectExpression.replace("`", "") + } else { + expectExpression + } val exp = convertToCatalystExpressions(filters, tableSchema) - if (expectExpression == null) { + if (removeQuotesIfNeed == null) { assertEquals(exp.isEmpty, true) } else { assertEquals(exp.isDefined, true) - assertEquals(expectExpression, exp.get.sql) + assertEquals(removeQuotesIfNeed, exp.get.sql) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala new file mode 100644 index 000000000000..9e3572b56161 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala @@ -0,0 +1,365 @@ +/* + * 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 org.apache.hudi + +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper +import org.apache.hudi.testutils.HoodieClientTestBase +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Expression, Not} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.hudi.DataSkippingUtils +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType, VarcharType} +import org.apache.spark.sql.{Column, SparkSession} +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments.arguments +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import scala.collection.JavaConverters._ + +// NOTE: Only A, B columns are indexed +case class IndexRow( + file: String, + A_minValue: Long, + A_maxValue: Long, + A_num_nulls: Long, + B_minValue: String = null, + B_maxValue: String = null, + B_num_nulls: Long = -1 +) + +class TestDataSkippingUtils extends HoodieClientTestBase { + + var spark: SparkSession = _ + + @BeforeEach + override def setUp(): Unit = { + initSparkContexts() + spark = sqlContext.sparkSession + } + + val indexedCols = Seq("A", "B") + val sourceTableSchema = + StructType( + Seq( + StructField("A", LongType), + StructField("B", StringType), + StructField("C", VarcharType(32)) + ) + ) + + val indexSchema = + ColumnStatsIndexHelper.composeIndexSchema( + sourceTableSchema.fields.toSeq + .filter(f => indexedCols.contains(f.name)) + .asJava + ) + + @ParameterizedTest + @MethodSource(Array("testBaseLookupFilterExpressionsSource", "testAdvancedLookupFilterExpressionsSource")) + def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = { + val resolvedExpr: Expression = resolveFilterExpr(sourceExpr, sourceTableSchema) + + val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) + + val spark2 = spark + import spark2.implicits._ + + val indexDf = spark.createDataset(input) + + val rows = indexDf.where(new Column(lookupFilter)) + .select("file") + .collect() + .map(_.getString(0)) + .toSeq + + assertEquals(output, rows) + } + + @ParameterizedTest + @MethodSource(Array("testStringsLookupFilterExpressionsSource")) + def testStringsLookupFilterExpressions(sourceExpr: Expression, input: Seq[IndexRow], output: Seq[String]): Unit = { + val resolvedExpr = resolveFilterExpr(sourceExpr, sourceTableSchema) + val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) + + val spark2 = spark + import spark2.implicits._ + + val indexDf = spark.createDataset(input) + + val rows = indexDf.where(new Column(lookupFilter)) + .select("file") + .collect() + .map(_.getString(0)) + .toSeq + + assertEquals(output, rows) + } + + private def resolveFilterExpr(exprString: String, tableSchema: StructType): Expression = { + val expr = spark.sessionState.sqlParser.parseExpression(exprString) + resolveFilterExpr(expr, tableSchema) + } + + private def resolveFilterExpr(expr: Expression, tableSchema: StructType): Expression = { + val schemaFields = tableSchema.fields + val resolvedExpr = spark.sessionState.analyzer.ResolveReferences( + Filter(expr, LocalRelation(schemaFields.head, schemaFields.drop(1): _*)) + ) + .asInstanceOf[Filter].condition + + checkForUnresolvedRefs(resolvedExpr) + } + + def checkForUnresolvedRefs(resolvedExpr: Expression): Expression = + resolvedExpr match { + case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute") + case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e)) + } +} + +object TestDataSkippingUtils { + def testStringsLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + arguments( + col("B").startsWith("abc").expr, + Seq( + IndexRow("file_1", 0, 0, 0, "aba", "adf", 1), // may contain strings starting w/ "abc" + IndexRow("file_2", 0, 0, 0, "adf", "azy", 0), + IndexRow("file_3", 0, 0, 0, "aaa", "aba", 0) + ), + Seq("file_1")), + arguments( + Not(col("B").startsWith("abc").expr), + Seq( + IndexRow("file_1", 0, 0, 0, "aba", "adf", 1), // may contain strings starting w/ "abc" + IndexRow("file_2", 0, 0, 0, "adf", "azy", 0), + IndexRow("file_3", 0, 0, 0, "aaa", "aba", 0), + IndexRow("file_4", 0, 0, 0, "abc123", "abc345", 0) // all strings start w/ "abc" + ), + Seq("file_1", "file_2", "file_3")) + ) + } + + def testBaseLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + // TODO cases + // A = null + arguments( + "A = 0", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0) + ), + Seq("file_2")), + arguments( + "0 = A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0) + ), + Seq("file_2")), + arguments( + "A != 0", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", 0, 0, 0) // Contains only 0s + ), + Seq("file_1", "file_2")), + arguments( + "0 != A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", 0, 0, 0) // Contains only 0s + ), + Seq("file_1", "file_2")), + arguments( + "A < 0", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_2", "file_3")), + arguments( + "0 > A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_2", "file_3")), + arguments( + "A > 0", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_1", "file_2")), + arguments( + "0 < A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_1", "file_2")), + arguments( + "A <= -1", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_2", "file_3")), + arguments( + "-1 >= A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_2", "file_3")), + arguments( + "A >= 1", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_1", "file_2")), + arguments( + "1 <= A", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_1", "file_2")), + arguments( + "A is null", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 1) + ), + Seq("file_2")), + arguments( + "A is not null", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 1) + ), + Seq("file_1")), + arguments( + "A in (0, 1)", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0) + ), + Seq("file_1", "file_2")), + arguments( + "A not in (0, 1)", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0), // only contains 0 + IndexRow("file_5", 1, 1, 0) // only contains 1 + ), + Seq("file_1", "file_2", "file_3")) + ) + } + + def testAdvancedLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + arguments( + // Filter out all rows that contain either A = 0 OR A = 1 + "A != 0 AND A != 1", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0), // only contains 0 + IndexRow("file_5", 1, 1, 0) // only contains 1 + ), + Seq("file_1", "file_2", "file_3")), + arguments( + // This is an equivalent to the above expression + "NOT(A = 0 OR A = 1)", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0), // only contains 0 + IndexRow("file_5", 1, 1, 0) // only contains 1 + ), + Seq("file_1", "file_2", "file_3")), + + arguments( + // Filter out all rows that contain A = 0 AND B = 'abc' + "A != 0 OR B != 'abc'", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0, "abc", "abc", 0), // only contains A = 0, B = 'abc' + IndexRow("file_5", 0, 0, 0, "abc", "abc", 0) // only contains A = 0, B = 'abc' + ), + Seq("file_1", "file_2", "file_3")), + arguments( + // This is an equivalent to the above expression + "NOT(A = 0 AND B = 'abc')", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0, "abc", "abc", 0), // only contains A = 0, B = 'abc' + IndexRow("file_5", 0, 0, 0, "abc", "abc", 0) // only contains A = 0, B = 'abc' + ), + Seq("file_1", "file_2", "file_3")), + + arguments( + // Queries contains expression involving non-indexed column C + "A = 0 AND B = 'abc' AND C = '...'", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0, "aaa", "xyz", 0) // might contain A = 0 AND B = 'abc' + ), + Seq("file_4")), + + arguments( + // Queries contains expression involving non-indexed column C + "A = 0 OR B = 'abc' OR C = '...'", + Seq( + IndexRow("file_1", 1, 2, 0), + IndexRow("file_2", -1, 1, 0), + IndexRow("file_3", -2, -1, 0), + IndexRow("file_4", 0, 0, 0, "aaa", "xyz", 0) // might contain B = 'abc' + ), + Seq("file_1", "file_2", "file_3", "file_4")) + ) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala new file mode 100644 index 000000000000..d5c3bfa01fc2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala @@ -0,0 +1,51 @@ +/* + * 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 org.apache.hudi + +import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, MultiPartKeysValueExtractor} +import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test + +class TestDataSourceOptions { + @Test def inferDataSourceOptions(): Unit = { + val inputOptions1 = Map( + TABLE_NAME.key -> "hudi_table", + PARTITIONPATH_FIELD.key -> "year,month" + ) + val modifiedOptions1 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions1) + assertEquals(classOf[ComplexKeyGenerator].getName, modifiedOptions1(KEYGENERATOR_CLASS_NAME.key)) + assertEquals("hudi_table", modifiedOptions1(HIVE_TABLE.key)) + assertEquals("year,month", modifiedOptions1(HIVE_PARTITION_FIELDS.key)) + assertEquals(classOf[MultiPartKeysValueExtractor].getName, + modifiedOptions1(HIVE_PARTITION_EXTRACTOR_CLASS.key)) + + val inputOptions2 = Map( + TABLE_NAME.key -> "hudi_table", + PARTITIONPATH_FIELD.key -> "year", + HIVE_STYLE_PARTITIONING.key -> "true" + ) + val modifiedOptions2 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions2) + assertEquals(classOf[SimpleKeyGenerator].getName, modifiedOptions2(KEYGENERATOR_CLASS_NAME.key)) + assertEquals("hudi_table", modifiedOptions2(HIVE_TABLE.key)) + assertEquals("year", modifiedOptions2(HIVE_PARTITION_FIELDS.key)) + assertEquals(classOf[HiveStylePartitionValueExtractor].getName, + modifiedOptions2(HIVE_PARTITION_EXTRACTOR_CLASS.key)) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala deleted file mode 100644 index 2caf4cc20eaa..000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestGenericRecordAndRowConsistency.scala +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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 org.apache.hudi - -import java.sql.{Date, Timestamp} - -import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.spark.sql.{DataFrame, SparkSession} - -import org.junit.jupiter.api.{BeforeEach, Test} - -class TestGenericRecordAndRowConsistency extends HoodieClientTestBase { - - var spark: SparkSession = _ - val commonOpts = Map( - HoodieWriteConfig.TBL_NAME.key -> "hoodie_type_consistency_tbl", - "hoodie.insert.shuffle.parallelism" -> "1", - "hoodie.upsert.shuffle.parallelism" -> "1", - DataSourceWriteOptions.TABLE_TYPE.key -> "COPY_ON_WRITE", - DataSourceWriteOptions.RECORDKEY_FIELD.key -> "str,eventTime", - DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "typeId", - DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "typeId", - DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.ComplexKeyGenerator" - ) - - /** - * Setup method running before each test. - */ - @BeforeEach override def setUp() { - setTableName("hoodie_type_consistency_tbl") - initPath() - initSparkContexts() - spark = sqlContext.sparkSession - } - - @Test - def testTimestampTypeConsistency(): Unit = { - val _spark = spark - import _spark.implicits._ - - val df = Seq( - (1, Timestamp.valueOf("2014-01-01 23:00:01"), "abc"), - (1, Timestamp.valueOf("2014-11-30 12:40:32"), "abc"), - (2, Timestamp.valueOf("2016-12-29 09:54:00"), "def"), - (2, Timestamp.valueOf("2016-05-09 10:12:43"), "def") - ).toDF("typeId","eventTime", "str") - - testConsistencyBetweenGenericRecordAndRow(df) - } - - @Test - def testDateTypeConsistency(): Unit = { - val _spark = spark - import _spark.implicits._ - - val df = Seq( - (1, Date.valueOf("2014-01-01"), "abc"), - (1, Date.valueOf("2014-11-30"), "abc"), - (2, Date.valueOf("2016-12-29"), "def"), - (2, Date.valueOf("2016-05-09"), "def") - ).toDF("typeId","eventTime", "str") - - testConsistencyBetweenGenericRecordAndRow(df) - } - - private def testConsistencyBetweenGenericRecordAndRow(df: DataFrame): Unit = { - val _spark = spark - import _spark.implicits._ - - // upsert operation generate recordKey by GenericRecord - val tempRecordPath = basePath + "/record_tbl/" - df.write.format("hudi") - .options(commonOpts) - .option(DataSourceWriteOptions.OPERATION.key, "upsert") - .mode(org.apache.spark.sql.SaveMode.Overwrite) - .save(tempRecordPath) - - val data1 = spark.read.format("hudi") - .load(tempRecordPath) - .select("_hoodie_record_key") - .map(_.toString()).collect().sorted - - // bulk_insert operation generate recordKey by Row - val tempRowPath = basePath + "/row_tbl/" - df.write.format("hudi") - .options(commonOpts) - .option(DataSourceWriteOptions.OPERATION.key, "bulk_insert") - .mode(org.apache.spark.sql.SaveMode.Overwrite) - .save(tempRowPath) - - val data2 = spark.read.format("hudi") - .load(tempRowPath) - .select("_hoodie_record_key") - .map(_.toString()).collect().sorted - - assert(data1 sameElements data2) - } - -} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 7c58cc07ea38..62f98cf53617 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -18,7 +18,6 @@ package org.apache.hudi import java.util.Properties - import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.table.HoodieTableMetaClient @@ -31,6 +30,7 @@ import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.{Config, TimestampType} import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal} import org.apache.spark.sql.execution.datasources.PartitionDirectory import org.apache.spark.sql.types.StringType @@ -38,7 +38,7 @@ import org.apache.spark.sql.{SaveMode, SparkSession} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -59,7 +59,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase { DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL ) - @BeforeEach override def setUp() { + @BeforeEach + override def setUp() { setTableName("hoodie_test") initPath() initSparkContexts() @@ -252,6 +253,29 @@ class TestHoodieFileIndex extends HoodieClientTestBase { assertEquals(5, readDF2.filter("dt = '2021/03/01' and hh ='10'").count()) } + @ParameterizedTest + @CsvSource(Array("true,a.b.c","false,a.b.c","true,c","false,c")) + def testQueryPartitionPathsForNestedPartition(useMetaFileList:Boolean, partitionBy:String): Unit = { + val inputDF = spark.range(100) + .withColumn("c",lit("c")) + .withColumn("b",struct("c")) + .withColumn("a",struct("b")) + inputDF.write.format("hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "id") + .option(PARTITIONPATH_FIELD.key, partitionBy) + .option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList) + .mode(SaveMode.Overwrite) + .save(basePath) + metaClient = HoodieTableMetaClient.reload(metaClient) + val fileIndex = HoodieFileIndex(spark, metaClient, None, + queryOpts ++ Map(HoodieMetadataConfig.ENABLE.key -> useMetaFileList.toString)) + // test if table is partitioned on nested columns, getAllQueryPartitionPaths does not break + assert(fileIndex.getAllQueryPartitionPaths.get(0).partitionPath.equals("c")) + } + private def attribute(partition: String): AttributeReference = { AttributeReference(partition, StringType, true)() } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala similarity index 85% rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index 96fb18db3672..35f4a61fe1bb 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -19,14 +19,15 @@ package org.apache.hudi import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path -import org.apache.hudi.DataSourceWriteOptions.{INSERT_DROP_DUPS, INSERT_OPERATION_OPT_VAL, KEYGENERATOR_CLASS_NAME, MOR_TABLE_TYPE_OPT_VAL, OPERATION, PAYLOAD_CLASS_NAME, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE} +import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.common.config.HoodieConfig -import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload, HoodieTableType, WriteOperationType} +import org.apache.hudi.common.model._ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig} -import org.apache.hudi.exception.HoodieException +import org.apache.hudi.exception.{ExceptionUtil, HoodieException} import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode import org.apache.hudi.functional.TestBootstrap import org.apache.hudi.hive.HiveSyncConfig @@ -35,28 +36,30 @@ import org.apache.hudi.testutils.DataSourceTestUtils import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.functions.{expr, lit} +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension +import org.apache.spark.sql.hudi.command.SqlKeyGenerator import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail} -import org.junit.jupiter.api.{AfterEach, Assertions, BeforeEach, Test} +import org.junit.jupiter.api.function.Executable +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, EnumSource, ValueSource} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{spy, times, verify} +import org.scalatest.Assertions.assertThrows import org.scalatest.Matchers.{assertResult, be, convertToAnyShouldWrapper, intercept} +import java.io.IOException import java.time.Instant -import java.util import java.util.{Collections, Date, UUID} - import scala.collection.JavaConversions._ import scala.collection.JavaConverters -import scala.util.control.NonFatal /** * Test suite for SparkSqlWriter class. */ -class HoodieSparkSqlWriterSuite { +class TestHoodieSparkSqlWriter { var spark: SparkSession = _ var sqlContext: SQLContext = _ var sc: SparkContext = _ @@ -71,7 +74,7 @@ class HoodieSparkSqlWriterSuite { * Setup method running before each test. */ @BeforeEach - def setUp() { + def setUp(): Unit = { initSparkContext() tempPath = java.nio.file.Files.createTempDirectory("hoodie_test_path") tempBootStrapPath = java.nio.file.Files.createTempDirectory("hoodie_test_bootstrap") @@ -96,6 +99,7 @@ class HoodieSparkSqlWriterSuite { spark = SparkSession.builder() .appName(hoodieFooTableName) .master("local[2]") + .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .getOrCreate() sc = spark.sparkContext @@ -147,7 +151,7 @@ class HoodieSparkSqlWriterSuite { * @param inputList list of Row * @return list of Seq */ - def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] = + def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] = JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq /** @@ -251,12 +255,14 @@ class HoodieSparkSqlWriterSuite { "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4") val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableModifier, dataFrame2)) - assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist")) + assert(tableAlreadyExistException.getMessage.contains("Config conflict")) + assert(tableAlreadyExistException.getMessage.contains(s"${HoodieWriteConfig.TBL_NAME.key}:\thoodie_bar_tbl\thoodie_foo_tbl")) //on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception val deleteTableModifier = barTableModifier ++ Map(OPERATION.key -> "delete") val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableModifier, dataFrame2)) - assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist")) + assert(tableAlreadyExistException.getMessage.contains("Config conflict")) + assert(tableAlreadyExistException.getMessage.contains(s"${HoodieWriteConfig.TBL_NAME.key}:\thoodie_bar_tbl\thoodie_foo_tbl")) } /** @@ -267,7 +273,7 @@ class HoodieSparkSqlWriterSuite { @ParameterizedTest @EnumSource(value = classOf[BulkInsertSortMode]) def testBulkInsertForSortMode(sortMode: BulkInsertSortMode): Unit = { - testBulkInsertWithSortMode(sortMode, true) + testBulkInsertWithSortMode(sortMode, populateMetaFields = true) } /** @@ -287,26 +293,26 @@ class HoodieSparkSqlWriterSuite { */ @Test def testDisableAndEnableMetaFields(): Unit = { - try { - testBulkInsertWithSortMode(BulkInsertSortMode.NONE, false) - //create a new table - val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") - .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) - .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") - .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name()) + testBulkInsertWithSortMode(BulkInsertSortMode.NONE, populateMetaFields = false) + //create a new table + val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") + .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name()) + .updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), "true") - // generate the inserts - val schema = DataSourceTestUtils.getStructTypeExampleSchema - val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) - val inserts = DataSourceTestUtils.generateRandomRows(1000) - val df = spark.createDataFrame(sc.parallelize(inserts), structType) - try { - // write to Hudi - HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) - Assertions.fail("Should have thrown exception") - } catch { - case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back")) - } + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val inserts = DataSourceTestUtils.generateRandomRows(1000) + val df = spark.createDataFrame(sc.parallelize(inserts), structType) + try { + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) + fail("Should have thrown exception") + } catch { + case e: HoodieException => assertTrue(e.getMessage.startsWith("Config conflict")) + case e: Exception => fail(e); } } @@ -342,7 +348,6 @@ class HoodieSparkSqlWriterSuite { @Test def testInsertDatasetWithoutPrecombineField(): Unit = { - //create a new table val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false") @@ -371,6 +376,28 @@ class HoodieSparkSqlWriterSuite { assert(df.except(trimmedDf).count() == 0) } + /** + * Test case for insert dataset without partitioning field + */ + @Test + def testInsertDatasetWithoutPartitionField(): Unit = { + val tableOpts = + commonTableModifier + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val records = DataSourceTestUtils.generateRandomRows(1) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + + // try write to Hudi + assertThrows[IOException] { + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, tableOpts - DataSourceWriteOptions.PARTITIONPATH_FIELD.key, df) + } + } + /** * Test case for bulk insert dataset with datasource impl multiple rounds. */ @@ -440,7 +467,7 @@ class HoodieSparkSqlWriterSuite { val records = DataSourceTestUtils.generateRandomRows(100) val recordsSeq = convertRowListToSeq(records) val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) - initializeMetaClientForBootstrap(fooTableParams, tableType, false) + initializeMetaClientForBootstrap(fooTableParams, tableType, addBootstrapPath = false) val client = spy(DataSourceUtils.createHoodieClient( new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName, mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) @@ -497,7 +524,7 @@ class HoodieSparkSqlWriterSuite { DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName) val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) - initializeMetaClientForBootstrap(fooTableParams, tableType, true) + initializeMetaClientForBootstrap(fooTableParams, tableType, addBootstrapPath = true) val client = spy(DataSourceUtils.createHoodieClient( new JavaSparkContext(sc), @@ -527,7 +554,8 @@ class HoodieSparkSqlWriterSuite { .setTableType(tableType) .setTableName(hoodieFooTableName) .setRecordKeyFields(fooTableParams(DataSourceWriteOptions.RECORDKEY_FIELD.key)) - .setBaseFileFormat(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name()) + .setBaseFileFormat(fooTableParams.getOrElse(HoodieWriteConfig.BASE_FILE_FORMAT.key, + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name)) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName(fooTableParams(PAYLOAD_CLASS_NAME.key)) .setPreCombineField(fooTableParams(PRECOMBINE_FIELD.key)) @@ -681,51 +709,49 @@ class HoodieSparkSqlWriterSuite { DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test") - try { - val df = spark.range(0, 1000).toDF("keyid") - .withColumn("col3", expr("keyid")) - .withColumn("age", lit(1)) - .withColumn("p", lit(2)) - - df.write.format("hudi") - .options(options) - .option(DataSourceWriteOptions.OPERATION.key, "insert") - .option("hoodie.insert.shuffle.parallelism", "4") - .mode(SaveMode.Overwrite).save(tempBasePath) - - df.write.format("hudi") - .options(options) - .option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table") - .option("hoodie.insert.shuffle.parallelism", "4") - .mode(SaveMode.Append).save(tempBasePath) - - val currentCommits = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0)) - val incrementalKeyIdNum = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000") - .option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommits(0)) - .load(tempBasePath).select("keyid").orderBy("keyid").count - assert(incrementalKeyIdNum == 1000) - - df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath) - spark.emptyDataFrame.write.format("hudi") - .options(options) - .option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath) - .option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName) - .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4") - .mode(SaveMode.Overwrite).save(tempBasePath) - df.write.format("hudi").options(options) - .option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table") - .option("hoodie.insert.shuffle.parallelism", "4").mode(SaveMode.Append).save(tempBasePath) - val currentCommitsBootstrap = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0)) - val incrementalKeyIdNumBootstrap = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000") - .option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommitsBootstrap(0)) - .load(tempBasePath).select("keyid").orderBy("keyid").count - assert(incrementalKeyIdNumBootstrap == 1000) - } + val df = spark.range(0, 1000).toDF("keyid") + .withColumn("col3", expr("keyid")) + .withColumn("age", lit(1)) + .withColumn("p", lit(2)) + + df.write.format("hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, "insert") + .option("hoodie.insert.shuffle.parallelism", "4") + .mode(SaveMode.Overwrite).save(tempBasePath) + + df.write.format("hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table") + .option("hoodie.insert.shuffle.parallelism", "4") + .mode(SaveMode.Append).save(tempBasePath) + + val currentCommits = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0)) + val incrementalKeyIdNum = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000") + .option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommits(0)) + .load(tempBasePath).select("keyid").orderBy("keyid").count + assert(incrementalKeyIdNum == 1000) + + df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath) + spark.emptyDataFrame.write.format("hudi") + .options(options) + .option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath) + .option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) + .option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4") + .mode(SaveMode.Overwrite).save(tempBasePath) + df.write.format("hudi").options(options) + .option(DataSourceWriteOptions.OPERATION.key, "insert_overwrite_table") + .option("hoodie.insert.shuffle.parallelism", "4").mode(SaveMode.Append).save(tempBasePath) + val currentCommitsBootstrap = spark.read.format("hudi").load(tempBasePath).select("_hoodie_commit_time").take(1).map(_.getString(0)) + val incrementalKeyIdNumBootstrap = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "0000") + .option(DataSourceReadOptions.END_INSTANTTIME.key, currentCommitsBootstrap(0)) + .load(tempBasePath).select("keyid").orderBy("keyid").count + assert(incrementalKeyIdNumBootstrap == 1000) } } @@ -848,7 +874,7 @@ class HoodieSparkSqlWriterSuite { .setConf(spark.sparkContext.hadoopConfiguration) .setBasePath(tablePath1).build().getTableConfig assert(tableConfig1.getHiveStylePartitioningEnable == "true") - assert(tableConfig1.getUrlEncodePartitoning == "false") + assert(tableConfig1.getUrlEncodePartitioning == "false") assert(tableConfig1.getKeyGeneratorClassName == classOf[ComplexKeyGenerator].getName) df.write.format("hudi") .options(options) @@ -869,23 +895,20 @@ class HoodieSparkSqlWriterSuite { .setConf(spark.sparkContext.hadoopConfiguration) .setBasePath(tablePath2).build().getTableConfig assert(tableConfig2.getHiveStylePartitioningEnable == "false") - assert(tableConfig2.getUrlEncodePartitoning == "true") + assert(tableConfig2.getUrlEncodePartitioning == "true") assert(tableConfig2.getKeyGeneratorClassName == classOf[SimpleKeyGenerator].getName) val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") // raise exception when use params which is not same with HoodieTableConfig - try { + val configConflictException = intercept[HoodieException] { df2.write.format("hudi") .options(options) .option(HoodieWriteConfig.TBL_NAME.key, tableName2) .option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName) .mode(SaveMode.Append).save(tablePath2) - } catch { - case NonFatal(e) => - assert(e.getMessage.contains("Config conflict")) - assert(e.getMessage.contains( - s"${HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key}\t${classOf[ComplexKeyGenerator].getName}\t${classOf[SimpleKeyGenerator].getName}")) } + assert(configConflictException.getMessage.contains("Config conflict")) + assert(configConflictException.getMessage.contains(s"KeyGenerator:\t${classOf[ComplexKeyGenerator].getName}\t${classOf[SimpleKeyGenerator].getName}")) // do not need to specify hiveStylePartitioning/urlEncodePartitioning/KeyGenerator params df2.write.format("hudi") @@ -894,6 +917,24 @@ class HoodieSparkSqlWriterSuite { .mode(SaveMode.Append).save(tablePath2) val data = spark.read.format("hudi").load(tablePath2 + "/*") assert(data.count() == 2) - assert(data.select("_hoodie_partition_path").map(_.getString(0)).distinct.collect.head == "dt=2021-10-16") + assert(data.select("_hoodie_partition_path").map(_.getString(0)).distinct.collect.head == "2021-10-16") + } + + @Test + def testGetOriginKeyGenerator(): Unit = { + // for dataframe write + val m1 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[ComplexKeyGenerator].getName + ) + val kg1 = HoodieWriterUtils.getOriginKeyGenerator(m1) + assertTrue(kg1 == classOf[ComplexKeyGenerator].getName) + + // for sql write + val m2 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getName, + SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> classOf[SimpleKeyGenerator].getName + ) + val kg2 = HoodieWriterUtils.getOriginKeyGenerator(m2) + assertTrue(kg2 == classOf[SimpleKeyGenerator].getName) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala index 1b756b5e2365..ad974286ac5a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala @@ -19,20 +19,17 @@ package org.apache.hudi import org.apache.avro.generic.GenericRecord - -import java.io.File -import java.nio.file.Paths import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.testutils.DataSourceTestUtils -import org.apache.spark.sql.avro.IncompatibleSchemaException -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SparkSession} -import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull, assertTrue, fail} +import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.io.TempDir -import java.util +import java.io.File +import java.nio.file.Paths import scala.collection.JavaConverters class TestHoodieSparkUtils { @@ -235,6 +232,6 @@ class TestHoodieSparkUtils { spark.stop() } - def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] = + def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] = JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 663493438a9e..a42a1ced1da9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -723,4 +723,29 @@ class TestCOWDataSource extends HoodieClientTestBase { val result = spark.sql("select * from tmptable limit 1").collect()(0) result.schema.contains(new StructField("partition", StringType, true)) } + + @Test + def testWriteSmallPrecisionDecimalTable(): Unit = { + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + .withColumn("shortDecimal", lit(new java.math.BigDecimal(s"2090.0000"))) // create decimalType(8, 4) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + // update the value of shortDecimal + val inputDF2 = inputDF1.withColumn("shortDecimal", lit(new java.math.BigDecimal(s"3090.0000"))) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val readResult = spark.read.format("hudi").load(basePath) + assert(readResult.count() == 5) + // compare the test result + assertEquals(inputDF2.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","), + readResult.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(",")) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala index 9524adbd5efd..bf616e2cb314 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala @@ -21,17 +21,21 @@ package org.apache.hudi.functional import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.Config +import org.apache.hudi.keygen.{ComplexKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Tag import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import scala.collection.JavaConversions._ @@ -39,7 +43,7 @@ import scala.collection.JavaConversions._ @Tag("functional") class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { - val commonOpts = Map( + var commonOpts = Map( "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4", "hoodie.bulkinsert.shuffle.parallelism" -> "2", @@ -54,14 +58,26 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val updatedVerificationVal: String = "driver_update" @ParameterizedTest - @ValueSource(booleans = Array(true, false)) - def testCopyOnWriteStorage(isMetadataEnabled: Boolean): Unit = { + @CsvSource(Array("true,org.apache.hudi.keygen.SimpleKeyGenerator", "true,org.apache.hudi.keygen.ComplexKeyGenerator", + "true,org.apache.hudi.keygen.TimestampBasedKeyGenerator", "false,org.apache.hudi.keygen.SimpleKeyGenerator", + "false,org.apache.hudi.keygen.ComplexKeyGenerator", "false,org.apache.hudi.keygen.TimestampBasedKeyGenerator")) + def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String): Unit = { + commonOpts += DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass + if (classOf[ComplexKeyGenerator].getName.equals(keyGenClass)) { + commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key, pii_col" + } + if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key" + commonOpts += DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "current_ts" + commonOpts += Config.TIMESTAMP_TYPE_FIELD_PROP -> "EPOCHMILLISECONDS" + commonOpts += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd" + } val dataGen = new HoodieTestDataGenerator() val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration) // Insert Operation - val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList - val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) - inputDF1.write.format("org.apache.hudi") + val records0 = recordsToStrings(dataGen.generateInserts("000", 100)).toList + val inputDF0 = spark.read.json(spark.sparkContext.parallelize(records0, 2)) + inputDF0.write.format("org.apache.hudi") .options(commonOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) @@ -77,9 +93,18 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { .load(basePath) assertEquals(100, snapshotDF1.count()) - // Upsert based on the written table with Hudi metadata columns - val verificationRowKey = snapshotDF1.limit(1).select("_row_key").first.getString(0) - val updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + val records1 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + val verificationRowKey = inputDF1.limit(1).select("_row_key").first.getString(0) + var updateDf: DataFrame = null + if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + // update current_ts to be same as original record so that partition path does not change with timestamp based key gen + val orignalRow = inputDF1.filter(col("_row_key") === verificationRowKey).collectAsList().get(0) + updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + .withColumn("current_ts", lit(orignalRow.getAs("current_ts"))) + } else { + updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + } updateDf.write.format("org.apache.hudi") .options(commonOpts) @@ -95,8 +120,26 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { assertEquals(updatedVerificationVal, snapshotDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) // Upsert Operation without Hudi metadata columns - val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList - val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2 , 2)) + val records2 = recordsToStrings(dataGen.generateUpdates("002", 100)).toList + var inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + + if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + // incase of Timestamp based key gen, current_ts should not be updated. but dataGen.generateUpdates() would have updated + // the value of current_ts. So, we need to revert it back to original value. + // here is what we are going to do. Copy values to temp columns, join with original df and update the current_ts + // and drop the temp columns. + + val inputDF2WithTempCols = inputDF2.withColumn("current_ts_temp", col("current_ts")) + .withColumn("_row_key_temp", col("_row_key")) + val originalRowCurrentTsDf = inputDF0.select("_row_key", "current_ts") + // join with original df + val joinedDf = inputDF2WithTempCols.drop("_row_key", "current_ts").join(originalRowCurrentTsDf, (inputDF2WithTempCols("_row_key_temp") === originalRowCurrentTsDf("_row_key"))) + // copy values from temp back to original cols and drop temp cols + inputDF2 = joinedDf.withColumn("current_ts_temp", col("current_ts")) + .drop("current_ts", "_row_key_temp").withColumn("current_ts", col("current_ts_temp")) + .drop("current_ts_temp") + } + val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() inputDF2.write.format("org.apache.hudi") @@ -131,12 +174,12 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val emptyIncDF = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, "000") - .option(DataSourceReadOptions.END_INSTANTTIME.key, "001") + .option(DataSourceReadOptions.END_INSTANTTIME.key, "002") .load(basePath) assertEquals(0, emptyIncDF.count()) // Upsert an empty dataFrame - val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList + val emptyRecords = recordsToStrings(dataGen.generateUpdates("003", 0)).toList val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1)) emptyDF.write.format("org.apache.hudi") .options(commonOpts) @@ -170,4 +213,78 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { .load(basePath) assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled } + + @ParameterizedTest + @ValueSource(strings = Array("insert_overwrite", "delete_partition")) + def testArchivalWithReplaceCommitActions(writeOperation: String): Unit = { + + val dataGen = new HoodieTestDataGenerator() + // use this to generate records only for certain partitions. + val dataGenPartition1 = new HoodieTestDataGenerator(Array[String](HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)) + val dataGenPartition2 = new HoodieTestDataGenerator(Array[String](HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) + + // do one bulk insert to all partitions + val records = recordsToStrings(dataGen.generateInserts("%05d".format(1), 100)).toList + val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) + val partition1RecordCount = inputDF.filter(row => row.getAs("partition_path") + .equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).count() + inputDF.write.format("hudi") + .options(commonOpts) + .option("hoodie.keep.min.commits", "2") + .option("hoodie.keep.max.commits", "3") + .option("hoodie.cleaner.commits.retained", "1") + .option("hoodie.metadata.enable", "false") + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + assertRecordCount(basePath, 100) + + // issue delete partition to partition1 + writeRecords(2, dataGenPartition1, writeOperation, basePath) + + val expectedRecCount = if (writeOperation.equals(DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)) { + 200 - partition1RecordCount + } else { + 100 - partition1RecordCount + } + assertRecordCount(basePath, expectedRecCount) + + // add more data to partition2. + for (i <- 3 to 7) { + writeRecords(i, dataGenPartition2, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL, basePath) + } + + assertRecordCount(basePath, expectedRecCount + 500) + val metaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true).build() + val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray + .map(instant => instant.asInstanceOf[HoodieInstant].getAction) + // assert replace commit is archived and not part of active timeline. + assertFalse(commits.contains(HoodieTimeline.REPLACE_COMMIT_ACTION)) + // assert that archival timeline has replace commit actions. + val archivedTimeline = metaClient.getArchivedTimeline(); + assertTrue(archivedTimeline.getInstants.toArray.map(instant => instant.asInstanceOf[HoodieInstant].getAction) + .filter(action => action.equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).size > 0) + } + + def writeRecords(commitTime: Int, dataGen: HoodieTestDataGenerator, writeOperation: String, basePath: String): Unit = { + val records = recordsToStrings(dataGen.generateInserts("%05d".format(commitTime), 100)).toList + val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) + inputDF.write.format("hudi") + .options(commonOpts) + .option("hoodie.keep.min.commits", "2") + .option("hoodie.keep.max.commits", "3") + .option("hoodie.cleaner.commits.retained", "1") + .option("hoodie.metadata.enable", "false") + .option(DataSourceWriteOptions.OPERATION.key, writeOperation) + .mode(SaveMode.Append) + .save(basePath) + } + + def assertRecordCount(basePath: String, expectedRecordCount: Long): Unit = { + val snapshotDF = spark.read.format("org.apache.hudi") + .load(basePath + "/*/*/*/*") + assertEquals(expectedRecordCount, snapshotDF.count()) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala new file mode 100644 index 000000000000..e79067041fb6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -0,0 +1,294 @@ +/* + * 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 org.apache.hudi.functional + +import org.apache.hadoop.fs.{LocatedFileStatus, Path} +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper +import org.apache.hudi.testutils.HoodieClientTestBase +import org.apache.spark.sql.functions.typedLit +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test} + +import scala.collection.JavaConversions._ + +class TestColumnStatsIndex extends HoodieClientTestBase { + var spark: SparkSession = _ + + val sourceTableSchema = + new StructType() + .add("c1", IntegerType) + .add("c2", StringType) + .add("c3", DecimalType(9,3)) + .add("c4", TimestampType) + .add("c5", ShortType) + .add("c6", DateType) + .add("c7", BinaryType) + .add("c8", ByteType) + + @BeforeEach + override def setUp() { + initPath() + initSparkContexts() + initFileSystem() + spark = sqlContext.sparkSession + } + + @AfterEach + override def tearDown() = { + cleanupFileSystem() + cleanupSparkContexts() + } + + @Test + @Disabled + def testColumnStatsTableComposition(): Unit = { + val inputDf = + // NOTE: Schema here is provided for validation that the input date is in the appropriate format + spark.read + .schema(sourceTableSchema) + .parquet( + getClass.getClassLoader.getResource("index/zorder/input-table").toString + ) + + val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8") + val zorderedColsSchemaFields = inputDf.schema.fields.filter(f => zorderedCols.contains(f.name)).toSeq + + // {@link TimestampType} is not supported, and will throw -- hence skipping "c4" + val newZIndexTableDf = + ColumnStatsIndexHelper.buildColumnStatsTableFor( + inputDf.sparkSession, + inputDf.inputFiles.toSeq, + zorderedColsSchemaFields + ) + + val indexSchema = + ColumnStatsIndexHelper.composeIndexSchema( + sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq + ) + + // Collect Z-index stats manually (reading individual Parquet files) + val manualZIndexTableDf = + buildColumnStatsTableManually( + getClass.getClassLoader.getResource("index/zorder/input-table").toString, + zorderedCols, + indexSchema + ) + + // NOTE: Z-index is built against stats collected w/in Parquet footers, which will be + // represented w/ corresponding Parquet schema (INT, INT64, INT96, etc). + // + // When stats are collected manually, produced Z-index table is inherently coerced into the + // schema of the original source Parquet base-file and therefore we have to similarly coerce newly + // built Z-index table (built off Parquet footers) into the canonical index schema (built off the + // original source file schema) + assertEquals(asJson(sort(manualZIndexTableDf)), asJson(sort(newZIndexTableDf))) + + // Match against expected Z-index table + val expectedZIndexTableDf = + spark.read + .schema(indexSchema) + .json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString) + + assertEquals(asJson(sort(expectedZIndexTableDf)), asJson(sort(newZIndexTableDf))) + } + + @Test + @Disabled + def testColumnStatsTableMerge(): Unit = { + val testZIndexPath = new Path(basePath, "zindex") + + val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8") + val indexSchema = + ColumnStatsIndexHelper.composeIndexSchema( + sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq + ) + + // + // Bootstrap Z-index table + // + + val firstCommitInstance = "0" + val firstInputDf = + spark.read.parquet( + getClass.getClassLoader.getResource("index/zorder/input-table").toString + ) + + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + firstInputDf.sparkSession, + sourceTableSchema, + firstInputDf.inputFiles.toSeq, + zorderedCols.toSeq, + testZIndexPath.toString, + firstCommitInstance, + Seq() + ) + + // NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able + // to reliably retrieve it + val initialZIndexTable = + spark.read + .parquet(new Path(testZIndexPath, firstCommitInstance).toString) + + val expectedInitialZIndexTableDf = + spark.read + .schema(indexSchema) + .json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString) + + assertEquals(asJson(sort(expectedInitialZIndexTableDf)), asJson(sort(initialZIndexTable))) + + val secondCommitInstance = "1" + val secondInputDf = + spark.read + .schema(sourceTableSchema) + .parquet( + getClass.getClassLoader.getResource("index/zorder/another-input-table").toString + ) + + // + // Update Z-index table + // + + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + secondInputDf.sparkSession, + sourceTableSchema, + secondInputDf.inputFiles.toSeq, + zorderedCols.toSeq, + testZIndexPath.toString, + secondCommitInstance, + Seq(firstCommitInstance) + ) + + // NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able + // to reliably retrieve it + val mergedZIndexTable = + spark.read + .parquet(new Path(testZIndexPath, secondCommitInstance).toString) + + val expectedMergedZIndexTableDf = + spark.read + .schema(indexSchema) + .json(getClass.getClassLoader.getResource("index/zorder/z-index-table-merged.json").toString) + + assertEquals(asJson(sort(expectedMergedZIndexTableDf)), asJson(sort(mergedZIndexTable))) + } + + @Test + @Disabled + def testColumnStatsTablesGarbageCollection(): Unit = { + val testZIndexPath = new Path(System.getProperty("java.io.tmpdir"), "zindex") + val fs = testZIndexPath.getFileSystem(spark.sparkContext.hadoopConfiguration) + + val inputDf = + spark.read.parquet( + getClass.getClassLoader.getResource("index/zorder/input-table").toString + ) + + // Try to save statistics + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + inputDf.sparkSession, + sourceTableSchema, + inputDf.inputFiles.toSeq, + Seq("c1","c2","c3","c5","c6","c7","c8"), + testZIndexPath.toString, + "2", + Seq("0", "1") + ) + + // Save again + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + inputDf.sparkSession, + sourceTableSchema, + inputDf.inputFiles.toSeq, + Seq("c1","c2","c3","c5","c6","c7","c8"), + testZIndexPath.toString, + "3", + Seq("0", "1", "2") + ) + + // Test old index table being cleaned up + ColumnStatsIndexHelper.updateColumnStatsIndexFor( + inputDf.sparkSession, + sourceTableSchema, + inputDf.inputFiles.toSeq, + Seq("c1","c2","c3","c5","c6","c7","c8"), + testZIndexPath.toString, + "4", + Seq("0", "1", "3") + ) + + assertEquals(!fs.exists(new Path(testZIndexPath, "2")), true) + assertEquals(!fs.exists(new Path(testZIndexPath, "3")), true) + assertEquals(fs.exists(new Path(testZIndexPath, "4")), true) + } + + private def buildColumnStatsTableManually(tablePath: String, zorderedCols: Seq[String], indexSchema: StructType) = { + val files = { + val it = fs.listFiles(new Path(tablePath), true) + var seq = Seq[LocatedFileStatus]() + while (it.hasNext) { + seq = seq :+ it.next() + } + seq + } + + spark.createDataFrame( + files.flatMap(file => { + val df = spark.read.schema(sourceTableSchema).parquet(file.getPath.toString) + val exprs: Seq[String] = + s"'${typedLit(file.getPath.getName)}' AS file" +: + df.columns + .filter(col => zorderedCols.contains(col)) + .flatMap(col => { + val minColName = s"${col}_minValue" + val maxColName = s"${col}_maxValue" + Seq( + s"min($col) AS $minColName", + s"max($col) AS $maxColName", + s"sum(cast(isnull($col) AS long)) AS ${col}_num_nulls" + ) + }) + + df.selectExpr(exprs: _*) + .collect() + }), + indexSchema + ) + } + + private def asJson(df: DataFrame) = + df.toJSON + .select("value") + .collect() + .toSeq + .map(_.getString(0)) + .mkString("\n") + + + private def sort(df: DataFrame): DataFrame = { + // Since upon parsing JSON, Spark re-order columns in lexicographical order + // of their names, we have to shuffle new Z-index table columns order to match + // Rows are sorted by filename as well to avoid + val sortedCols = df.columns.sorted + df.select(sortedCols.head, sortedCols.tail: _*) + .sort("file") + } + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala index d6ae80d09af5..eeed5fe75b84 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala @@ -17,7 +17,7 @@ package org.apache.hudi.functional -import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector import org.apache.hudi.common.fs.FSUtils @@ -31,9 +31,9 @@ import org.apache.spark.sql.{SaveMode, SparkSession} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.io.TempDir import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} - import java.time.Instant import java.util.Collections + import scala.collection.JavaConverters._ class TestDataSourceForBootstrap { @@ -102,7 +102,12 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit( + DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, + extraOpts = Map(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator") + ) + // check marked directory clean up + assert(!fs.exists(new Path(basePath, ".hoodie/.temp/00000000000001"))) // Read bootstrapped table and verify count var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") @@ -158,6 +163,9 @@ class TestDataSourceForBootstrap { Some("datestr"), Map(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true")) + // check marked directory clean up + assert(!fs.exists(new Path(basePath, ".hoodie/.temp/00000000000001"))) + // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") assertEquals(numRecords, hoodieROViewDF1.count()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 5719ad7c1d69..9996f23c728c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -715,6 +715,28 @@ class TestMORDataSource extends HoodieClientTestBase { assertEquals(true, fs.listStatus(tempPath).isEmpty) } + @Test + def testClusteringOnNullableColumn(): Unit = { + val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + .withColumn("cluster_id", when(expr("end_lon < 0.2 "), lit(null).cast("string")) + .otherwise(col("_row_key"))) + .withColumn("struct_cluster_col", when(expr("end_lon < 0.1"), lit(null)) + .otherwise(struct(col("cluster_id"), col("_row_key")))) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + // option for clustering + .option("hoodie.clustering.inline", "true") + .option("hoodie.clustering.inline.max.commits", "1") + .option("hoodie.clustering.plan.strategy.target.file.max.bytes", "1073741824") + .option("hoodie.clustering.plan.strategy.small.file.limit", "629145600") + .option("hoodie.clustering.plan.strategy.sort.columns", "struct_cluster_col") + .mode(SaveMode.Overwrite) + .save(basePath) + } + @Test def testHoodieIsDeletedMOR(): Unit = { val numRecords = 100 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala new file mode 100644 index 000000000000..57ebd038f2ae --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala @@ -0,0 +1,153 @@ +/* + * 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 org.apache.hudi.functional + +import scala.collection.JavaConversions._ + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.config.{HoodieIndexConfig, HoodieLayoutConfig, HoodieWriteConfig} +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} +import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings +import org.apache.hudi.index.HoodieIndex.IndexType +import org.apache.hudi.keygen.constant.KeyGeneratorOptions +import org.apache.hudi.testutils.HoodieClientTestBase + +import org.apache.spark.sql._ +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} + +import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner +import org.apache.hudi.table.storage.HoodieStorageLayout + +/** + * + */ +class TestDataSourceForBucketIndex extends HoodieClientTestBase { + + var spark: SparkSession = null + val commonOpts = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + "hoodie.bulkinsert.shuffle.parallelism" -> "2", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + HoodieIndexConfig.INDEX_TYPE.key -> IndexType.BUCKET.name, + HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key -> "8", + KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key -> "_row_key", + HoodieLayoutConfig.LAYOUT_TYPE.key -> HoodieStorageLayout.LayoutType.BUCKET.name, + HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME.key -> classOf[SparkBucketIndexPartitioner[_]].getName + ) + + @BeforeEach override def setUp(): Unit = { + initPath() + initSparkContexts() + spark = sqlContext.sparkSession + initTestDataGenerator() + initFileSystem() + } + + @AfterEach override def tearDown(): Unit = { + cleanupSparkContexts() + cleanupTestDataGenerator() + cleanupFileSystem() + } + + @Test def testDoubleInsert(): Unit = { + val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) + val records2 = recordsToStrings(dataGen.generateInserts("002", 100)).toList + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "false") + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + assertEquals(200, hudiSnapshotDF1.count()) + } + + @Test def testCountWithBucketIndex(): Unit = { + // First Operation: + // Producing parquet files to three default partitions. + // SNAPSHOT view on MOR table with parquet files only. + val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) + val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated + + // Second Operation: + // Upsert the update to the default partitions with duplicate records. Produced a log file for each parquet. + // SNAPSHOT view should read the log files only with the latest commit time. + val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .mode(SaveMode.Append) + .save(basePath) + val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated + val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString + val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString + assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1) + assertTrue(commit2Time > commit1Time) + assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count()) + + val partitionPaths = new Array[String](1) + partitionPaths.update(0, "2020/01/10") + val newDataGen = new HoodieTestDataGenerator(partitionPaths) + val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).toList + val inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2)) + inputDF4.write.format("org.apache.hudi") + .options(commonOpts) + .mode(SaveMode.Append) + .save(basePath) + val hudiSnapshotDF4 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + // 200, because we insert 100 records to a new partition + assertEquals(200, hudiSnapshotDF4.count()) + assertEquals(100, + hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count()) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestOptimizeTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSpaceCurveLayoutOptimization.scala similarity index 59% rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestOptimizeTable.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSpaceCurveLayoutOptimization.scala index 06ac600b0346..e453953ff11e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestOptimizeTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSpaceCurveLayoutOptimization.scala @@ -18,26 +18,38 @@ package org.apache.hudi.functional -import java.sql.{Date, Timestamp} - -import org.apache.hadoop.fs.Path -import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} -import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings +import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.spark.ZCurveOptimizeHelper +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions} import org.apache.spark.sql._ import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.params.provider.Arguments.arguments +import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import java.sql.{Date, Timestamp} import scala.collection.JavaConversions._ import scala.util.Random -class TestOptimizeTable extends HoodieClientTestBase { - var spark: SparkSession = null +@Tag("functional") +class TestSpaceCurveLayoutOptimization extends HoodieClientTestBase { + var spark: SparkSession = _ + + val sourceTableSchema = + new StructType() + .add("c1", IntegerType) + .add("c2", StringType) + .add("c3", DecimalType(9,3)) + .add("c4", TimestampType) + .add("c5", ShortType) + .add("c6", DateType) + .add("c7", BinaryType) + .add("c8", ByteType) val commonOpts = Map( "hoodie.insert.shuffle.parallelism" -> "4", @@ -49,7 +61,8 @@ class TestOptimizeTable extends HoodieClientTestBase { HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" ) - @BeforeEach override def setUp() { + @BeforeEach + override def setUp() { initPath() initSparkContexts() spark = sqlContext.sparkSession @@ -57,19 +70,22 @@ class TestOptimizeTable extends HoodieClientTestBase { initFileSystem() } - @AfterEach override def tearDown() = { + @AfterEach + override def tearDown() = { cleanupSparkContexts() cleanupTestDataGenerator() cleanupFileSystem() } @ParameterizedTest - @ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ")) - def testOptimizewithClustering(tableType: String): Unit = { + @MethodSource(Array("testLayoutOptimizationParameters")) + def testLayoutOptimizationFunctional(tableType: String): Unit = { + val targetRecordsCount = 10000 // Bulk Insert Operation - val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList - val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) - inputDF1.write.format("org.apache.hudi") + val records = recordsToStrings(dataGen.generateInserts("001", targetRecordsCount)).toList + val writeDf: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records, 2)) + + writeDf.write.format("org.apache.hudi") .options(commonOpts) .option("hoodie.compact.inline", "false") .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) @@ -81,57 +97,55 @@ class TestOptimizeTable extends HoodieClientTestBase { .option("hoodie.clustering.plan.strategy.target.file.max.bytes", "1073741824") .option("hoodie.clustering.plan.strategy.small.file.limit", "629145600") .option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString) - .option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 *1024 * 1024L)) + .option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L)) .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_ENABLE.key, "true") .option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat, begin_lon") .mode(SaveMode.Overwrite) .save(basePath) - assertEquals(1000, spark.read.format("hudi").load(basePath).count()) - assertEquals(1000, - spark.read.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true").format("hudi").load(basePath).count()) + val hudiMetaClient = HoodieTableMetaClient.builder + .setConf(hadoopConf) + .setBasePath(basePath) + .setLoadActiveTimelineOnLoad(true) + .build + + val lastCommit = hudiMetaClient.getActiveTimeline.getAllCommitsTimeline.lastInstant().get() + + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, lastCommit.getAction) + assertEquals(HoodieInstant.State.COMPLETED, lastCommit.getState) + + val readDf = + spark.read + .format("hudi") + .load(basePath) + + val readDfSkip = + spark.read + .option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true") + .format("hudi") + .load(basePath) + + assertEquals(targetRecordsCount, readDf.count()) + assertEquals(targetRecordsCount, readDfSkip.count()) + + readDf.createOrReplaceTempView("hudi_snapshot_raw") + readDfSkip.createOrReplaceTempView("hudi_snapshot_skipping") + + def select(tableName: String) = + spark.sql(s"SELECT * FROM $tableName WHERE begin_lat >= 0.49 AND begin_lat < 0.51 AND begin_lon >= 0.49 AND begin_lon < 0.51") + + assertRowsMatch( + select("hudi_snapshot_raw"), + select("hudi_snapshot_skipping") + ) } - @Test - def testCollectMinMaxStatistics(): Unit = { - val testPath = new Path(System.getProperty("java.io.tmpdir"), "minMax") - val statisticPath = new Path(System.getProperty("java.io.tmpdir"), "stat") - val fs = testPath.getFileSystem(spark.sparkContext.hadoopConfiguration) - try { - val complexDataFrame = createComplexDataFrame(spark) - complexDataFrame.repartition(3).write.mode("overwrite").save(testPath.toString) - val df = spark.read.load(testPath.toString) - // do not support TimeStampType, so if we collect statistics for c4, should throw exception - val colDf = ZCurveOptimizeHelper.getMinMaxValue(df, "c1,c2,c3,c5,c6,c7,c8") - colDf.cache() - assertEquals(colDf.count(), 3) - assertEquals(colDf.take(1)(0).length, 22) - colDf.unpersist() - // try to save statistics - ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "2", Seq("0", "1")) - // save again - ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "3", Seq("0", "1", "2")) - // test old index table clean - ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "4", Seq("0", "1", "3")) - assertEquals(!fs.exists(new Path(statisticPath, "2")), true) - assertEquals(fs.exists(new Path(statisticPath, "3")), true) - } finally { - if (fs.exists(testPath)) fs.delete(testPath) - if (fs.exists(statisticPath)) fs.delete(statisticPath) - } + private def assertRowsMatch(one: DataFrame, other: DataFrame) = { + val rows = one.count() + assert(rows == other.count() && one.intersect(other).count() == rows) } def createComplexDataFrame(spark: SparkSession): DataFrame = { - val schema = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - .add("c3", DecimalType(9,3)) - .add("c4", TimestampType) - .add("c5", ShortType) - .add("c6", DateType) - .add("c7", BinaryType) - .add("c8", ByteType) - val rdd = spark.sparkContext.parallelize(0 to 1000, 1).map { item => val c1 = Integer.valueOf(item) val c2 = s" ${item}sdc" @@ -144,6 +158,18 @@ class TestOptimizeTable extends HoodieClientTestBase { RowFactory.create(c1, c2, c3, c4, c5, c6, c7, c8) } - spark.createDataFrame(rdd, schema) + spark.createDataFrame(rdd, sourceTableSchema) } } + +object TestSpaceCurveLayoutOptimization { + def testLayoutOptimizationParameters(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + arguments("COPY_ON_WRITE", "hilbert"), + arguments("COPY_ON_WRITE", "z-order"), + arguments("MERGE_ON_READ", "hilbert"), + arguments("MERGE_ON_READ", "z-order") + ) + } +} + diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 68b630be5d5e..85c64f826541 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -59,9 +59,11 @@ class TestStructuredStreaming extends HoodieClientTestBase { spark = sqlContext.sparkSession initTestDataGenerator() initFileSystem() + initTimelineService() } @AfterEach override def tearDown() = { + cleanupTimelineService() cleanupSparkContexts() cleanupTestDataGenerator() cleanupFileSystem() diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala index c4af71768b16..55f90f0ddef4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala @@ -218,13 +218,13 @@ class TestTimeTravelQuery extends HoodieClientTestBase { } private def defaultDateTimeFormat(queryInstant: String): String = { - val date = HoodieActiveTimeline.parseInstantTime(queryInstant) - val format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val date = HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) + val format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS") format.format(date) } private def defaultDateFormat(queryInstant: String): String = { - val date = HoodieActiveTimeline.parseInstantTime(queryInstant) + val date = HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) val format = new SimpleDateFormat("yyyy-MM-dd") format.format(date) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchMark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchMark.scala new file mode 100644 index 000000000000..0b0599fb2027 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchMark.scala @@ -0,0 +1,127 @@ +/* + * 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 org.apache.spark.sql.execution.benchmark + +import org.apache.hadoop.fs.Path +import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper +import org.apache.hudi.sort.SpaceCurveSortingHelper +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.types.{IntegerType, StructField} +import org.junit.jupiter.api.{Disabled, Tag, Test} + +import scala.util.Random +import scala.collection.JavaConversions._ + +@Tag("functional") +object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase { + + def evalSkippingPercent(tableName: String, co1: String, co2: String, value1: Int, value2: Int): Unit= { + val sourceTableDF = spark.sql(s"select * from ${tableName}") + + val orderedColsTypes = Seq(StructField(co1, IntegerType), StructField(co2, IntegerType)) + val colStatsIndexTable = ColumnStatsIndexHelper + .buildColumnStatsTableFor(spark, sourceTableDF.inputFiles.toSeq, orderedColsTypes) + .collect() + .map(f => (f.getInt(1), f.getInt(2), f.getInt(4), f.getInt(5))) + + var hits = 0 + for (fileStatRow <- colStatsIndexTable) { + if ((fileStatRow._1 <= value1 && fileStatRow._2 >= value1) || (fileStatRow._3 <= value2 && fileStatRow._4 >= value2)) { + hits = hits + 1 + } + } + + val p = hits / colStatsIndexTable.size.toDouble + println(s"For table ${tableName} with query filter: ${co1} = ${value1} or ${co2} = ${value2} we can achieve skipping percent ${1.0 - p} (w/ total files ${colStatsIndexTable.size})") + } + + /* + for table table_z_sort_byMap with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.8 + for table table_z_sort_bySample with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.77 + for table table_hilbert_sort_byMap with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.855 + for table table_hilbert_sort_bySample with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.83 + */ + @Test + @Disabled + def runNormalTableSkippingBenchMark(): Unit = { + withTempDir { f => + withTempTable("table_z_sort_byMap", "table_z_sort_bySample", "table_hilbert_sort_byMap", "table_hilbert_sort_bySample") { + prepareInterTypeTable(new Path(f.getAbsolutePath), 1000000) + // choose median value as filter condition. + // the median value of c1_int is 500000 + // the median value of c2_int is 500000 + evalSkippingPercent("table_z_sort_byMap", "c1_int", "c2_int", 500000, 500000) + evalSkippingPercent("table_z_sort_bySample", "c1_int", "c2_int", 500000, 500000) + evalSkippingPercent("table_hilbert_sort_byMap", "c1_int", "c2_int", 500000, 500000) + evalSkippingPercent("table_hilbert_sort_bySample", "c1_int", "c2_int", 500000, 500000) + } + } + } + + /* + for table table_z_sort_byMap_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.0 + for table table_z_sort_bySample_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.78 + for table table_hilbert_sort_byMap_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.05500000000000005 + for table table_hilbert_sort_bySample_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.84 + */ + @Test + @Disabled + def runSkewTableSkippingBenchMark(): Unit = { + withTempDir { f => + withTempTable("table_z_sort_byMap_skew", "table_z_sort_bySample_skew", "table_hilbert_sort_byMap_skew", "table_hilbert_sort_bySample_skew") { + // prepare skewed table. + prepareInterTypeTable(new Path(f.getAbsolutePath), 1000000, 10000, 1000000, true) + // choose median value as filter condition. + // the median value of c1_int is 5000 + // the median value of c2_int is 500000 + evalSkippingPercent("table_z_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000) + evalSkippingPercent("table_z_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000) + evalSkippingPercent("table_hilbert_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000) + evalSkippingPercent("table_hilbert_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000) + } + } + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + def prepareInterTypeTable(tablePath: Path, numRows: Int, col1Range: Int = 1000000, col2Range: Int = 1000000, skewed: Boolean = false): Unit = { + import spark.implicits._ + val df = spark.range(numRows).map(_ => (Random.nextInt(col1Range), Random.nextInt(col2Range))).toDF("c1_int", "c2_int") + val dfOptimizeByMap = SpaceCurveSortingHelper.orderDataFrameByMappingValues(df, LayoutOptimizationStrategy.ZORDER, Seq("c1_int", "c2_int"), 200) + val dfOptimizeBySample = SpaceCurveSortingHelper.orderDataFrameBySamplingValues(df, LayoutOptimizationStrategy.ZORDER, Seq("c1_int", "c2_int"), 200) + + val dfHilbertOptimizeByMap = SpaceCurveSortingHelper.orderDataFrameByMappingValues(df, LayoutOptimizationStrategy.HILBERT, Seq("c1_int", "c2_int"), 200) + val dfHilbertOptimizeBySample = SpaceCurveSortingHelper.orderDataFrameBySamplingValues(df, LayoutOptimizationStrategy.HILBERT, Seq("c1_int", "c2_int"), 200) + + saveAsTable(dfOptimizeByMap, tablePath, if (skewed) "z_sort_byMap_skew" else "z_sort_byMap") + saveAsTable(dfOptimizeBySample, tablePath, if (skewed) "z_sort_bySample_skew" else "z_sort_bySample") + saveAsTable(dfHilbertOptimizeByMap, tablePath, if (skewed) "hilbert_sort_byMap_skew" else "hilbert_sort_byMap") + saveAsTable(dfHilbertOptimizeBySample, tablePath, if (skewed) "hilbert_sort_bySample_skew" else "hilbert_sort_bySample") + } + + def saveAsTable(df: DataFrame, savePath: Path, suffix: String): Unit = { + df.write.mode("overwrite").save(new Path(savePath, suffix).toString) + spark.read.parquet(new Path(savePath, suffix).toString).createOrReplaceTempView("table_" + suffix) + } +} + diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala index e1bc4a1f7176..e2521047f68c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala @@ -38,7 +38,7 @@ class TestAlterTable extends TestHoodieSqlBase { | ts long |) using hudi | location '$tablePath' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' @@ -127,7 +127,7 @@ class TestAlterTable extends TestHoodieSqlBase { | dt string |) using hudi | location '${tmp.getCanonicalPath}/$partitionedTable' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala index b2ada77c2194..22d20cdd09cc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala @@ -18,12 +18,11 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} import org.apache.spark.sql.SaveMode -import scala.util.control.NonFatal - class TestAlterTableDropPartition extends TestHoodieSqlBase { test("Drop non-partitioned table") { @@ -38,7 +37,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { | dt string | ) | using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -47,7 +46,31 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""") checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01')")( - s"dt is not a valid partition column in table `default`.`$tableName`.") + s"$tableName is a non-partitioned table that is not allowed to drop partition") + } + + test("Purge drop non-partitioned table") { + val tableName = generateTableName + // create table + spark.sql( + s""" + | create table $tableName ( + | id bigint, + | name string, + | ts string, + | dt string + | ) + | using hudi + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + |""".stripMargin) + // insert data + spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""") + + checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01') purge")( + s"$tableName is a non-partitioned table that is not allowed to drop partition") } Seq(false, true).foreach { urlencode => @@ -77,7 +100,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -88,7 +111,62 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { // drop 2021-10-01 partition spark.sql(s"alter table $tableName drop partition (dt='2021/10/01')") - checkAnswer(s"select dt from $tableName") (Seq(s"2021/10/02")) + val partitionPath = if (urlencode) { + PartitionPathEncodeUtils.escapePathName("2021/10/01") + } else { + "2021/10/01" + } + checkAnswer(s"select dt from $tableName")(Seq(s"2021/10/02")) + assertResult(true)(existsPath(s"${tmp.getCanonicalPath}/$tableName/$partitionPath")) + } + } + } + + Seq(false, true).foreach { urlencode => + test(s"Purge drop single-partition table' partitions, urlencode: $urlencode") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + + import spark.implicits._ + val df = Seq((1, "z3", "v1", "2021/10/01"), (2, "l4", "v1", "2021/10/02")) + .toDF("id", "name", "ts", "dt") + + df.write.format("hudi") + .option(HoodieWriteConfig.TBL_NAME.key, tableName) + .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL) + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(PARTITIONPATH_FIELD.key, "dt") + .option(URL_ENCODE_PARTITIONING.key(), urlencode) + .option(KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName) + .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") + .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") + .mode(SaveMode.Overwrite) + .save(tablePath) + + // register meta to spark catalog by creating table + spark.sql( + s""" + |create table $tableName using hudi + |tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + |) + |partitioned by (dt) + |location '$tablePath' + |""".stripMargin) + + // drop 2021-10-01 partition + spark.sql(s"alter table $tableName drop partition (dt='2021/10/01') purge") + + val partitionPath = if (urlencode) { + PartitionPathEncodeUtils.escapePathName("2021/10/01") + } else { + "2021/10/01" + } + checkAnswer(s"select dt from $tableName")(Seq(s"2021/10/02")) + assertResult(false)(existsPath(s"${tmp.getCanonicalPath}/$tableName/$partitionPath")) } } } @@ -105,7 +183,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { | dt string | ) | using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -151,7 +229,7 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -172,4 +250,51 @@ class TestAlterTableDropPartition extends TestHoodieSqlBase { } } } + + Seq(false, true).foreach { hiveStyle => + test(s"Purge drop multi-level partitioned table's partitions, isHiveStylePartitioning: $hiveStyle") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + + import spark.implicits._ + val df = Seq((1, "z3", "v1", "2021", "10", "01"), (2, "l4", "v1", "2021", "10","02")) + .toDF("id", "name", "ts", "year", "month", "day") + + df.write.format("hudi") + .option(HoodieWriteConfig.TBL_NAME.key, tableName) + .option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL) + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(PARTITIONPATH_FIELD.key, "year,month,day") + .option(HIVE_STYLE_PARTITIONING.key, hiveStyle) + .option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName) + .option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1") + .option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1") + .mode(SaveMode.Overwrite) + .save(tablePath) + + // register meta to spark catalog by creating table + spark.sql( + s""" + |create table $tableName using hudi + |tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + |) + |partitioned by (year, month, day) + |location '$tablePath' + |""".stripMargin) + + // drop 2021-10-01 partition + spark.sql(s"alter table $tableName drop partition (year='2021', month='10', day='01') purge") + + checkAnswer(s"select id, name, ts, year, month, day from $tableName")( + Seq(2, "l4", "v1", "2021", "10", "02") + ) + assertResult(false)(existsPath( + s"${tmp.getCanonicalPath}/$tableName/year=2021/month=10/day=01")) + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala index e40a48421018..20238a6e4318 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala @@ -31,7 +31,7 @@ class TestCompactionTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts' @@ -82,7 +82,7 @@ class TestCompactionTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 2af8fd782cb8..14bc70428df8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -43,7 +43,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long | ) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -62,6 +62,53 @@ class TestCreateTable extends TestHoodieSqlBase { )(table.schema.fields) } + test("Test Create Hoodie Table With Options") { + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | dt string + | ) using hudi + | partitioned by (dt) + | options ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assertResult(table.properties("type"))("cow") + assertResult(table.properties("primaryKey"))("id") + assertResult(table.properties("preCombineField"))("ts") + assertResult(tableName)(table.identifier.table) + assertResult("hudi")(table.provider.get) + assertResult(CatalogTableType.MANAGED)(table.tableType) + assertResult( + HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType)) + ++ Seq( + StructField("id", IntegerType), + StructField("name", StringType), + StructField("price", DoubleType), + StructField("ts", LongType), + StructField("dt", StringType)) + )(table.schema.fields) + + val tablePath = table.storage.properties("path") + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(tablePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + val tableConfig = metaClient.getTableConfig.getProps.asScala.toMap + assertResult(true)(tableConfig.contains(HoodieTableConfig.CREATE_SCHEMA.key)) + assertResult("dt")(tableConfig(HoodieTableConfig.PARTITION_FIELDS.key)) + assertResult("id")(tableConfig(HoodieTableConfig.RECORDKEY_FIELDS.key)) + assertResult("ts")(tableConfig(HoodieTableConfig.PRECOMBINE_FIELD.key)) + assertResult(classOf[ComplexKeyGenerator].getCanonicalName)(tableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) + } + test("Test Create External Hoodie Table") { withTempDir { tmp => // Test create cow table. @@ -74,7 +121,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id,name', | type = 'cow' | ) @@ -93,8 +140,8 @@ class TestCreateTable extends TestHoodieSqlBase { StructField("price", DoubleType), StructField("ts", LongType)) )(table.schema.fields) - assertResult(table.storage.properties("type"))("cow") - assertResult(table.storage.properties("primaryKey"))("id,name") + assertResult(table.properties("type"))("cow") + assertResult(table.properties("primaryKey"))("id,name") spark.sql(s"drop table $tableName") // Test create mor partitioned table @@ -108,15 +155,15 @@ class TestCreateTable extends TestHoodieSqlBase { | dt string |) using hudi | partitioned by (dt) - | options ( + | tblproperties ( | primaryKey = 'id', | type = 'mor' | ) | location '${tmp.getCanonicalPath}/h0' """.stripMargin) val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - assertResult(table2.storage.properties("type"))("mor") - assertResult(table2.storage.properties("primaryKey"))("id") + assertResult(table2.properties("type"))("mor") + assertResult(table2.properties("primaryKey"))("id") assertResult(Seq("dt"))(table2.partitionColumnNames) assertResult(classOf[HoodieParquetRealtimeInputFormat].getCanonicalName)(table2.storage.inputFormat.get) @@ -129,8 +176,8 @@ class TestCreateTable extends TestHoodieSqlBase { |location '${tmp.getCanonicalPath}/h0' """.stripMargin) val table3 = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName3)) - assertResult(table3.storage.properties("type"))("mor") - assertResult(table3.storage.properties("primaryKey"))("id") + assertResult(table3.properties("type"))("mor") + assertResult(table3.properties("primaryKey"))("id") assertResult( HoodieRecord.HOODIE_META_COLUMNS.asScala.map(StructField(_, StringType)) ++ Seq( @@ -156,7 +203,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id1', | type = 'cow' | ) @@ -173,7 +220,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts1', | type = 'cow' @@ -191,7 +238,7 @@ class TestCreateTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts', | type = 'cow1' @@ -208,7 +255,8 @@ class TestCreateTable extends TestHoodieSqlBase { val tableName1 = generateTableName spark.sql( s""" - |create table $tableName1 using hudi + | create table $tableName1 using hudi + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName1' | AS | select 1 as id, 'a1' as name, 10 as price, 1000 as ts @@ -223,6 +271,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName2 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName2' | AS | select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt @@ -240,7 +289,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName3 using hudi | partitioned by (dt) - | options(primaryKey = 'id') + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName3' | AS | select null as id, 'a1' as name, 10 as price, '2021-05-07' as dt @@ -252,6 +301,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName3 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName3' | AS | select cast('2021-05-06 00:00:00' as timestamp) as dt, 1 as id, 'a1' as name, 10 as @@ -267,6 +317,7 @@ class TestCreateTable extends TestHoodieSqlBase { s""" | create table $tableName4 using hudi | partitioned by (dt) + | tblproperties(primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tableName4' | AS | select cast('2021-05-06' as date) as dt, 1 as id, 'a1' as name, 10 as @@ -303,7 +354,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -380,7 +431,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -455,7 +506,7 @@ class TestCreateTable extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - | options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' |) @@ -514,6 +565,7 @@ class TestCreateTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + |tblproperties(primaryKey = 'id') |""".stripMargin ) @@ -527,6 +579,7 @@ class TestCreateTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + |tblproperties(primaryKey = 'id') |""".stripMargin ) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala index 9ad717aba457..6137c4c6394b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala @@ -33,7 +33,7 @@ class TestDeleteTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala new file mode 100644 index 000000000000..c53eb9127c88 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala @@ -0,0 +1,75 @@ +/* + * 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 org.apache.spark.sql.hudi + +class TestDropTable extends TestHoodieSqlBase { + + test("Test Drop Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"DROP TABLE $tableName") + checkAnswer(s"show tables like '$tableName'")() + assertResult(true)(existsPath(s"${tmp.getCanonicalPath}/$tableName")) + } + } + } + + test("Test Drop Table with purge") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + spark.sql(s"DROP TABLE $tableName PURGE") + checkAnswer(s"show tables like '$tableName'")() + assertResult(false)(existsPath(s"${tmp.getCanonicalPath}/$tableName")) + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala new file mode 100644 index 000000000000..4c0c60385104 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -0,0 +1,184 @@ +/* + * 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 org.apache.spark.sql.hudi + +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.testutils.HoodieClientTestBase + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types._ + +import org.junit.jupiter.api.Assertions.assertTrue +import org.junit.jupiter.api.{BeforeEach, Test} + +import org.scalatest.Matchers.intercept + +class TestHoodieOptionConfig extends HoodieClientTestBase { + + var spark: SparkSession = _ + + /** + * Setup method running before each test. + */ + @BeforeEach override def setUp() { + initSparkContexts() + spark = sqlContext.sparkSession + } + + @Test + def testWithDefaultSqlOptions(): Unit = { + val ops1 = Map("primaryKey" -> "id") + val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1) + assertTrue(with1.size == 3) + assertTrue(with1("primaryKey") == "id") + assertTrue(with1("type") == "cow") + assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName) + + val ops2 = Map("primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "mor", + "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName + ) + val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2) + assertTrue(ops2 == with2) + } + + @Test + def testMappingSqlOptionToTableConfig(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true" + ) + val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) + + assertTrue(tableConfigs.size == 5) + assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr") + assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp") + assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ") + assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY") + assertTrue(tableConfigs("hoodie.compact.inline") == "true") + } + + @Test + def testDeleteHoodieOptions(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true", + "key123" -> "value456" + ) + val tableConfigs = HoodieOptionConfig.deleteHoodieOptions(sqlOptions) + assertTrue(tableConfigs.size == 1) + assertTrue(tableConfigs("key123") == "value456") + } + + @Test + def testExtractSqlOptions(): Unit = { + val sqlOptions = Map("primaryKey" -> "id,addr", + "preCombineField" -> "timestamp", + "type" -> "mor", + "hoodie.index.type" -> "INMEMORY", + "hoodie.compact.inline" -> "true", + "key123" -> "value456" + ) + val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions) + assertTrue(tableConfigs.size == 3) + assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type")) + } + + @Test + def testValidateTable(): Unit = { + val baseSqlOptions = Map( + "hoodie.datasource.write.hive_style_partitioning" -> "true", + "hoodie.datasource.write.partitionpath.urlencode" -> "false", + "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator" + ) + + val schema = StructType( + Seq(StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("timestamp", TimestampType, true), + StructField("dt", StringType, true)) + ) + + // miss primaryKey parameter + val sqlOptions1 = baseSqlOptions ++ Map( + "type" -> "mor" + ) + + val e1 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions1) + } + assertTrue(e1.getMessage.contains("No `primaryKey` is specified.")) + + // primary field not found + val sqlOptions2 = baseSqlOptions ++ Map( + "primaryKey" -> "xxx", + "type" -> "mor" + ) + val e2 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions2) + } + assertTrue(e2.getMessage.contains("Can't find primaryKey")) + + // preCombine field not found + val sqlOptions3 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "ts", + "type" -> "mor" + ) + val e3 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions3) + } + assertTrue(e3.getMessage.contains("Can't find preCombineKey")) + + // miss type parameter + val sqlOptions4 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp" + ) + val e4 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions4) + } + assertTrue(e4.getMessage.contains("No `type` is specified.")) + + // type is invalid + val sqlOptions5 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "abc" + ) + val e5 = intercept[IllegalArgumentException] { + HoodieOptionConfig.validateTable(spark, schema, sqlOptions5) + } + assertTrue(e5.getMessage.contains("'type' must be 'cow' or 'mor'")) + + // right options and schema + val sqlOptions6 = baseSqlOptions ++ Map( + "primaryKey" -> "id", + "preCombineField" -> "timestamp", + "type" -> "cow" + ) + HoodieOptionConfig.validateTable(spark, schema, sqlOptions6) + } + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala index 5413bf404489..a5b49cc3683d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.hudi -import java.io.File +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.fs.FSUtils import org.apache.log4j.Level import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.{Row, SparkSession} @@ -25,6 +26,7 @@ import org.apache.spark.util.Utils import org.scalactic.source import org.scalatest.{BeforeAndAfterAll, FunSuite, Tag} +import java.io.File import java.util.TimeZone class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll { @@ -59,14 +61,18 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll { } override protected def test(testName: String, testTags: Tag*)(testFun: => Any /* Assertion */)(implicit pos: source.Position): Unit = { - try super.test(testName, testTags: _*)(try testFun finally { - val catalog = spark.sessionState.catalog - catalog.listDatabases().foreach{db => - catalog.listTables(db).foreach {table => - catalog.dropTable(table, true, true) + super.test(testName, testTags: _*)( + try { + testFun + } finally { + val catalog = spark.sessionState.catalog + catalog.listDatabases().foreach{db => + catalog.listTables(db).foreach {table => + catalog.dropTable(table, true, true) + } } } - }) + ) } protected def generateTableName: String = { @@ -115,4 +121,10 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll { case _=> value } } + + protected def existsPath(filePath: String): Boolean = { + val path = new Path(filePath) + val fs = FSUtils.getFs(filePath, spark.sparkContext.hadoopConfiguration) + fs.exists(path) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 337317b8ef7b..ee1e2e6f42cf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hudi -import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.exception.HoodieDuplicateKeyException @@ -36,6 +35,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}' """.stripMargin) @@ -75,7 +75,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = 'cow', | primaryKey = 'id', | preCombineField = 'ts' @@ -115,7 +115,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName2' - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -146,6 +146,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' """.stripMargin) @@ -191,6 +192,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | ts long | ) using hudi + | tblproperties (primaryKey = 'id') | location '${tmp.getCanonicalPath}/$tblNonPartition' """.stripMargin) spark.sql(s"insert into $tblNonPartition select 1, 'a1', 10, 1000") @@ -245,6 +247,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt $partitionType |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' """.stripMargin) @@ -261,10 +264,6 @@ class TestInsertTable extends TestHoodieSqlBase { test("Test insert for uppercase table name") { withTempDir{ tmp => val tableName = s"H_$generateTableName" - HoodieTableMetaClient.withPropertyBuilder() - .setTableName(tableName) - .setTableType(HoodieTableType.COPY_ON_WRITE.name()) - .initTable(spark.sessionState.newHadoopConf(), tmp.getCanonicalPath) spark.sql( s""" @@ -273,6 +272,7 @@ class TestInsertTable extends TestHoodieSqlBase { | name string, | price double |) using hudi + | tblproperties (primaryKey = 'id') | location '${tmp.getCanonicalPath}' """.stripMargin) @@ -280,6 +280,11 @@ class TestInsertTable extends TestHoodieSqlBase { checkAnswer(s"select id, name, price from $tableName")( Seq(1, "a1", 10.0) ) + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(tmp.getCanonicalPath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + assertResult(metaClient.getTableConfig.getTableName)(tableName) } } @@ -293,6 +298,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) """.stripMargin) checkException(s"insert into $tableName partition(dt = '2021-06-20')" + @@ -305,7 +311,7 @@ class TestInsertTable extends TestHoodieSqlBase { " count: 3,columns: (1,a1,10)" ) spark.sql("set hoodie.sql.bulk.insert.enable = true") - spark.sql("set hoodie.sql.insert.mode= strict") + spark.sql("set hoodie.sql.insert.mode = strict") val tableName2 = generateTableName spark.sql( @@ -316,7 +322,7 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | ts long |) using hudi - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -325,6 +331,7 @@ class TestInsertTable extends TestHoodieSqlBase { "Table with primaryKey can not use bulk insert in strict mode." ) + spark.sql("set hoodie.sql.insert.mode = non-strict") val tableName3 = generateTableName spark.sql( s""" @@ -334,16 +341,18 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi + | tblproperties (primaryKey = 'id') | partitioned by (dt) """.stripMargin) checkException(s"insert overwrite table $tableName3 values(1, 'a1', 10, '2021-07-18')")( "Insert Overwrite Partition can not use bulk insert." ) spark.sql("set hoodie.sql.bulk.insert.enable = false") - spark.sql("set hoodie.sql.insert.mode= upsert") + spark.sql("set hoodie.sql.insert.mode = upsert") } test("Test bulk insert") { + spark.sql("set hoodie.sql.insert.mode = non-strict") withTempDir { tmp => Seq("cow", "mor").foreach {tableType => // Test bulk insert for single partition @@ -356,8 +365,9 @@ class TestInsertTable extends TestHoodieSqlBase { | price double, | dt string |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | partitioned by (dt) | location '${tmp.getCanonicalPath}/$tableName' @@ -391,8 +401,9 @@ class TestInsertTable extends TestHoodieSqlBase { | dt string, | hh string |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | partitioned by (dt, hh) | location '${tmp.getCanonicalPath}/$tableMultiPartition' @@ -423,8 +434,9 @@ class TestInsertTable extends TestHoodieSqlBase { | name string, | price double |) using hudi - | options ( - | type = '$tableType' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id' | ) | location '${tmp.getCanonicalPath}/$nonPartitionedTable' """.stripMargin) @@ -445,7 +457,7 @@ class TestInsertTable extends TestHoodieSqlBase { s""" |create table $tableName2 |using hudi - |options( + |tblproperties( | type = '$tableType', | primaryKey = 'id' |) @@ -459,9 +471,11 @@ class TestInsertTable extends TestHoodieSqlBase { ) } } + spark.sql("set hoodie.sql.insert.mode = upsert") } test("Test combine before insert") { + spark.sql("set hoodie.sql.bulk.insert.enable = false") withTempDir{tmp => val tableName = generateTableName spark.sql( @@ -473,7 +487,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -495,6 +509,7 @@ class TestInsertTable extends TestHoodieSqlBase { } test("Test insert pk-table") { + spark.sql("set hoodie.sql.bulk.insert.enable = false") withTempDir{tmp => val tableName = generateTableName spark.sql( @@ -506,7 +521,7 @@ class TestInsertTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala index d911ace62a23..5139825f9428 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala @@ -34,7 +34,7 @@ class TestMergeIntoLogOnlyTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts', diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala index bd8558710d0f..baac82f4bd15 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala @@ -35,7 +35,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -137,7 +137,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$targetTable' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -203,7 +203,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -313,7 +313,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id' | ) @@ -369,7 +369,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | v long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'v' @@ -439,7 +439,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -457,7 +457,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -553,7 +553,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | c $dataType |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'c' | ) @@ -604,7 +604,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}' - | options ( + | tblproperties ( | primaryKey ='id', | type = 'mor', | preCombineField = 'ts', @@ -665,7 +665,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -711,7 +711,7 @@ class TestMergeIntoTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala index bf73251e947d..5041a543168b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hudi +import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.spark.sql.Row @@ -35,7 +36,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long, | dt string | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -145,7 +146,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { spark.sql( s""" |create table $tableName using hudi - |options(primaryKey = 'id') + |tblproperties(primaryKey = 'id') |location '${tmp.getCanonicalPath}' |as |select 1 as id, 'a1' as name @@ -187,7 +188,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | m_value map, | ts long | ) using hudi - | options ( + | tblproperties ( | type = 'mor', | primaryKey = 'id', | preCombineField = 'ts' @@ -251,7 +252,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | dt string |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -333,7 +334,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) @@ -352,7 +353,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | when not matched and flag = '1' then insert * |""".stripMargin - if (HoodieSqlUtils.isSpark3) { + if (HoodieSparkUtils.isSpark3) { checkExceptionContain(mergeSql)("Columns aliases are not allowed in MERGE") } else { spark.sql(mergeSql) @@ -376,7 +377,7 @@ class TestMergeIntoTable2 extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | primaryKey ='id', | preCombineField = 'ts' | ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala index 357954ebb1d5..2524d04ec81f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala @@ -32,7 +32,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id', | preCombineField = '_ts' @@ -60,7 +60,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | name string, | price double |) using hudi - |options( + |tblproperties( | type ='$tableType', | primaryKey = 'id' |) @@ -92,7 +92,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type = 'cow', | primaryKey = 'id', | preCombineField = '_ts' @@ -117,7 +117,7 @@ class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { | price double, | _ts long |) using hudi - |options( + |tblproperties( | type = 'mor', | primaryKey = 'id', | preCombineField = '_ts' diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala index 05ee61c4879f..868bfc43d57f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala @@ -32,7 +32,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | price double, | ts long |) using hudi - |options ( + |tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' ) @@ -59,7 +59,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | dt string ) using hudi | partitioned by (dt) - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -109,7 +109,7 @@ class TestShowPartitions extends TestHoodieSqlBase { | day string | ) using hudi | partitioned by (year, month, day) - | options ( + | tblproperties ( | primaryKey = 'id', | preCombineField = 'ts' | ) @@ -154,7 +154,7 @@ class TestShowPartitions extends TestHoodieSqlBase { Seq("year=2021/month=02/day=default"), Seq("year=2021/month=02/day=01") ) - checkAnswer(s"show partitions $tableName partition(day=01)")( + checkAnswer(s"show partitions $tableName partition(day='01')")( Seq("year=2021/month=02/day=01"), Seq("year=2021/month=default/day=01"), Seq("year=2021/month=01/day=01"), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala new file mode 100644 index 000000000000..1a8ac0e64589 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala @@ -0,0 +1,104 @@ +/* + * 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 org.apache.spark.sql.hudi + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.config.DFSPropertiesConfiguration +import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} + +import java.io.File +import java.nio.file.{Files, Paths} + +import org.scalatest.BeforeAndAfter + +class TestSqlConf extends TestHoodieSqlBase with BeforeAndAfter { + + def setEnv(key: String, value: String): String = { + val field = System.getenv().getClass.getDeclaredField("m") + field.setAccessible(true) + val map = field.get(System.getenv()).asInstanceOf[java.util.Map[java.lang.String, java.lang.String]] + map.put(key, value) + } + + test("Test Hudi Conf") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = tmp.getCanonicalPath + val partitionVal = "2021" + // Create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | year string + |) using hudi + | partitioned by (year) + | location '$tablePath' + | options ( + | primaryKey ='id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // First merge with a extra input field 'flag' (insert a new record) + spark.sql( + s""" + | merge into $tableName + | using ( + | select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '1' as flag, $partitionVal as year + | ) s0 + | on s0.id = $tableName.id + | when matched and flag = '1' then update set + | id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, year = s0.year + | when not matched and flag = '1' then insert * + """.stripMargin) + checkAnswer(s"select id, name, price, ts, year from $tableName")( + Seq(1, "a1", 10.0, 1000, partitionVal) + ) + + // By default, Spark DML would set table type to COW and use Hive style partitioning, here we + // set table type to MOR and disable Hive style partitioning in the hudi conf file, and check + // if Hudi DML can load these configs correctly + assertResult(true)(Files.exists(Paths.get(s"$tablePath/$partitionVal"))) + assertResult(HoodieTableType.MERGE_ON_READ)(new HoodieTableConfig( + new Path(tablePath).getFileSystem(new Configuration), + s"$tablePath/" + HoodieTableMetaClient.METAFOLDER_NAME, + HoodieTableConfig.PAYLOAD_CLASS_NAME.defaultValue).getTableType) + + // delete the record + spark.sql(s"delete from $tableName where year = $partitionVal") + val cnt = spark.sql(s"select * from $tableName where year = $partitionVal").count() + assertResult(0)(cnt) + } + } + + before { + val testPropsFilePath = new File("src/test/resources/external-config").getAbsolutePath + setEnv(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, testPropsFilePath) + DFSPropertiesConfiguration.refreshGlobalProps() + } + + after { + DFSPropertiesConfiguration.clearGlobalProps() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 8e9c81b12cb7..82d067cfb1bf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -20,6 +20,47 @@ package org.apache.spark.sql.hudi class TestUpdateTable extends TestHoodieSqlBase { test("Test Update Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach {tableType => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | ID int, + | NAME string, + | PRICE double, + | TS long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | options ( + | type = '$tableType', + | primaryKey = 'ID', + | preCombineField = 'TS' + | ) + """.stripMargin) + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000) + ) + + // update data + spark.sql(s"update $tableName set price = 20 where id = 1") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 20.0, 1000) + ) + + // update data + spark.sql(s"update $tableName set price = price * 2 where id = 1") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 40.0, 1000) + ) + } + } + } + + test("Test ignoring case for Update Table") { withTempDir { tmp => Seq("cow", "mor").foreach {tableType => val tableName = generateTableName @@ -33,7 +74,7 @@ class TestUpdateTable extends TestHoodieSqlBase { | ts long |) using hudi | location '${tmp.getCanonicalPath}/$tableName' - | options ( + | tblproperties ( | type = '$tableType', | primaryKey = 'id', | preCombineField = 'ts' @@ -46,13 +87,13 @@ class TestUpdateTable extends TestHoodieSqlBase { ) // update data - spark.sql(s"update $tableName set price = 20 where id = 1") + spark.sql(s"update $tableName set PRICE = 20 where ID = 1") checkAnswer(s"select id, name, price, ts from $tableName")( Seq(1, "a1", 20.0, 1000) ) // update data - spark.sql(s"update $tableName set price = price * 2 where id = 1") + spark.sql(s"update $tableName set PRICE = PRICE * 2 where ID = 1") checkAnswer(s"select id, name, price, ts from $tableName")( Seq(1, "a1", 40.0, 1000) ) diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index 0a5710b0603c..2283603542ee 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-spark2_${scala.binary.version} - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-spark2_${scala.binary.version} jar diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java index addbc899d7b5..e607b2fdcbc0 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -33,8 +33,11 @@ import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.types.StructType; +import java.util.Map; import java.util.Optional; +import static org.apache.hudi.DataSourceUtils.mayBeOverwriteParquetWriteLegacyFormatProp; + /** * DataSource V2 implementation for managing internal write logic. Only called internally. */ @@ -64,8 +67,11 @@ public Optional createWriter(String writeUUID, StructType sche String tblName = options.get(HoodieWriteConfig.TBL_NAME.key()).get(); boolean populateMetaFields = options.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(), Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())); + Map properties = options.asMap(); + // Auto set the value of "hoodie.parquet.writeLegacyFormat.enabled" + mayBeOverwriteParquetWriteLegacyFormatProp(properties, schema); // 1st arg to createHoodieConfig is not really required to be set. but passing it anyways. - HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()).get(), path, tblName, options.asMap()); + HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()).get(), path, tblName, properties); boolean arePartitionRecordsSorted = HoodieInternalConfig.getBulkInsertIsPartitionRecordsSorted( options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).isPresent() ? options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).get() : null); diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/Spark2ParsePartitionUtil.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/Spark2ParsePartitionUtil.scala index 5bf028408cce..c3cbcc407587 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/Spark2ParsePartitionUtil.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/Spark2ParsePartitionUtil.scala @@ -16,18 +16,26 @@ */ package org.apache.spark.sql.execution.datasources + import java.util.TimeZone import org.apache.hadoop.fs.Path -import org.apache.spark.sql.execution.datasources.PartitioningUtils.PartitionValues -import org.apache.spark.sql.types.DataType + +import org.apache.spark.sql.types._ +import org.apache.spark.sql.catalyst.InternalRow class Spark2ParsePartitionUtil extends SparkParsePartitionUtil { - override def parsePartition(path: Path, typeInference: Boolean, - basePaths: Set[Path], - userSpecifiedDataTypes: Map[String, DataType], - timeZone: TimeZone): Option[PartitionValues] = { - PartitioningUtils.parsePartition(path, typeInference, - basePaths, userSpecifiedDataTypes, timeZone)._1 + + override def parsePartition( + path: Path, + typeInference: Boolean, + basePaths: Set[Path], + userSpecifiedDataTypes: Map[String, DataType], + timeZone: TimeZone): InternalRow = { + val (partitionValues, _) = PartitioningUtils.parsePartition(path, typeInference, + basePaths, userSpecifiedDataTypes, timeZone) + + partitionValues.map(_.literals.map(_.value)).map(InternalRow.fromSeq) + .getOrElse(InternalRow.empty) } } diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml index fb6e6b0b5f17..ca09d8359f96 100644 --- a/hudi-spark-datasource/hudi-spark3/pom.xml +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -17,12 +17,12 @@ hudi-spark-datasource org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 hudi-spark3_2.12 - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT hudi-spark3_2.12 jar diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java index eda8faead986..b124853c8057 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java @@ -31,8 +31,11 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import java.util.HashMap; import java.util.Map; +import static org.apache.hudi.DataSourceUtils.mayBeOverwriteParquetWriteLegacyFormatProp; + /** * DataSource V2 implementation for managing internal write logic. Only called internally. * This class is only compatible with datasource V2 API in Spark 3. @@ -53,9 +56,13 @@ public Table getTable(StructType schema, Transform[] partitioning, Map newProps = new HashMap<>(properties); + // Auto set the value of "hoodie.parquet.writeLegacyFormat.enabled" + mayBeOverwriteParquetWriteLegacyFormatProp(newProps, schema); // 1st arg to createHoodieConfig is not really required to be set. but passing it anyways. - HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()), path, tblName, properties); + HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(newProps.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()), path, tblName, newProps); return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(), - getConfiguration(), properties, populateMetaFields, arePartitionRecordsSorted); + getConfiguration(), newProps, populateMetaFields, arePartitionRecordsSorted); } } diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java index c7a70438fc3c..236fbe933c85 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/ReflectUtil.java @@ -17,20 +17,25 @@ package org.apache.hudi.spark3.internal; +import org.apache.hudi.HoodieSparkUtils; import org.apache.spark.sql.catalyst.plans.logical.InsertIntoStatement; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.util.DateFormatter; + import scala.Option; import scala.collection.Seq; import scala.collection.immutable.Map; import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.time.ZoneId; public class ReflectUtil { - public static InsertIntoStatement createInsertInto(boolean isSpark30, LogicalPlan table, Map> partition, Seq userSpecifiedCols, + public static InsertIntoStatement createInsertInto(LogicalPlan table, Map> partition, Seq userSpecifiedCols, LogicalPlan query, boolean overwrite, boolean ifPartitionNotExists) { try { - if (isSpark30) { + if (HoodieSparkUtils.isSpark3_0()) { Constructor constructor = InsertIntoStatement.class.getConstructor( LogicalPlan.class, Map.class, LogicalPlan.class, boolean.class, boolean.class); return constructor.newInstance(table, partition, query, overwrite, ifPartitionNotExists); @@ -43,4 +48,23 @@ public static InsertIntoStatement createInsertInto(boolean isSpark30, LogicalPla throw new RuntimeException("Error in create InsertIntoStatement", e); } } + + public static DateFormatter getDateFormatter(ZoneId zoneId) { + try { + ClassLoader loader = Thread.currentThread().getContextClassLoader(); + if (HoodieSparkUtils.isSpark3_2()) { + Class clazz = loader.loadClass(DateFormatter.class.getName()); + Method applyMethod = clazz.getDeclaredMethod("apply"); + applyMethod.setAccessible(true); + return (DateFormatter)applyMethod.invoke(null); + } else { + Class clazz = loader.loadClass(DateFormatter.class.getName()); + Method applyMethod = clazz.getDeclaredMethod("apply", ZoneId.class); + applyMethod.setAccessible(true); + return (DateFormatter)applyMethod.invoke(null, zoneId); + } + } catch (Exception e) { + throw new RuntimeException("Error in apply DateFormatter", e); + } + } } diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala index 87d80d0b42bf..7e806f7407b2 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.adapter import org.apache.hudi.Spark3RowSerDe import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.spark3.internal.ReflectUtil -import org.apache.spark.SPARK_VERSION + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -79,7 +79,7 @@ class Spark3Adapter extends SparkAdapter { override def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = { - ReflectUtil.createInsertInto(SPARK_VERSION.startsWith("3.0"), table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists) + ReflectUtil.createInsertInto(table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists) } override def createSparkParsePartitionUtil(conf: SQLConf): SparkParsePartitionUtil = { diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala index ea9cc788aff4..d993b980367f 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParsePartitionUtil.scala @@ -16,24 +16,259 @@ */ package org.apache.spark.sql.execution.datasources -import java.util.TimeZone + +import java.lang.{Double => JDouble, Long => JLong} +import java.math.{BigDecimal => JBigDecimal} +import java.time.ZoneId +import java.util.{Locale, TimeZone} import org.apache.hadoop.fs.Path + +import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH +import org.apache.hudi.spark3.internal.ReflectUtil + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} -import org.apache.spark.sql.execution.datasources.PartitioningUtils.{PartitionValues, timestampPartitionPattern} +import org.apache.spark.sql.execution.datasources.PartitioningUtils.timestampPartitionPattern import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import scala.collection.mutable.ArrayBuffer +import scala.util.Try +import scala.util.control.NonFatal class Spark3ParsePartitionUtil(conf: SQLConf) extends SparkParsePartitionUtil { - override def parsePartition(path: Path, typeInference: Boolean, - basePaths: Set[Path], userSpecifiedDataTypes: Map[String, DataType], - timeZone: TimeZone): Option[PartitionValues] = { - val dateFormatter = DateFormatter(timeZone.toZoneId) + /** + * The definition of PartitionValues has been changed by SPARK-34314 in Spark3.2. + * To solve the compatibility between 3.1 and 3.2, copy some codes from PartitioningUtils in Spark3.2 here. + * And this method will generate and return `InternalRow` directly instead of `PartitionValues`. + */ + override def parsePartition( + path: Path, + typeInference: Boolean, + basePaths: Set[Path], + userSpecifiedDataTypes: Map[String, DataType], + timeZone: TimeZone): InternalRow = { + val dateFormatter = ReflectUtil.getDateFormatter(timeZone.toZoneId) val timestampFormatter = TimestampFormatter(timestampPartitionPattern, timeZone.toZoneId, isParsing = true) - PartitioningUtils.parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, - conf.validatePartitionColumns, timeZone.toZoneId, dateFormatter, timestampFormatter)._1 + val (partitionValues, _) = parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, + conf.validatePartitionColumns, timeZone.toZoneId, dateFormatter, timestampFormatter) + + partitionValues.map { + case PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue]) => + val rowValues = columnNames.zip(typedValues).map { case (columnName, typedValue) => + try { + castPartValueToDesiredType(typedValue.dataType, typedValue.value, timeZone.toZoneId) + } catch { + case NonFatal(_) => + if (conf.validatePartitionColumns) { + throw new RuntimeException(s"Failed to cast value `${typedValue.value}` to " + + s"`${typedValue.dataType}` for partition column `$columnName`") + } else null + } + } + InternalRow.fromSeq(rowValues) + }.getOrElse(InternalRow.empty) + } + + case class TypedPartValue(value: String, dataType: DataType) + + case class PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue]) + { + require(columnNames.size == typedValues.size) } + + private def parsePartition( + path: Path, + typeInference: Boolean, + basePaths: Set[Path], + userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = { + + val columns = ArrayBuffer.empty[(String, TypedPartValue)] + // 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) { + // Sometimes (e.g., when speculative task is enabled), temporary directories may be left + // uncleaned. Here we simply ignore them. + if (currentPath.getName.toLowerCase(Locale.ROOT) == "_temporary") { + // scalastyle:off return + return (None, None) + // scalastyle:on return + } + + if (basePaths.contains(currentPath)) { + // If the currentPath is one of base paths. We should stop. + finished = true + } else { + // 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 maybeColumn = + parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, + validatePartitionColumns, zoneId, dateFormatter, timestampFormatter) + maybeColumn.foreach(columns += _) + + // Now, we determine if we should stop. + // When we hit any of the following cases, we will stop: + // - In this iteration, we could not parse the value of partition column and value, + // i.e. maybeColumn is None, and columns is not empty. At here we check if columns is + // empty to handle cases like /table/a=1/_temporary/something (we need to find a=1 in + // this case). + // - After we get the new currentPath, this new currentPath represent the top level dir + // i.e. currentPath.getParent == null. For the example of "/table/a=1/", + // the top level dir is "/table". + finished = + (maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null + + if (!finished) { + // For the above example, currentPath will be "/table/". + currentPath = currentPath.getParent + } + } + } + + if (columns.isEmpty) { + (None, Some(path)) + } else { + val (columnNames, values) = columns.reverse.unzip + (Some(PartitionValues(columnNames.toSeq, values.toSeq)), Some(currentPath)) + } + } + + private def parsePartitionColumn( + columnSpec: String, + typeInference: Boolean, + userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = { + 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'") + + val dataType = if (userSpecifiedDataTypes.contains(columnName)) { + // SPARK-26188: if user provides corresponding column schema, get the column value without + // inference, and then cast it as user specified data type. + userSpecifiedDataTypes(columnName) + } else { + inferPartitionColumnValue( + rawColumnValue, + typeInference, + zoneId, + dateFormatter, + timestampFormatter) + } + Some(columnName -> TypedPartValue(rawColumnValue, dataType)) + } + } + + private def inferPartitionColumnValue( + raw: String, + typeInference: Boolean, + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): DataType = { + val decimalTry = Try { + // `BigDecimal` conversion can fail when the `field` is not a form of number. + val bigDecimal = new JBigDecimal(raw) + // It reduces the cases for decimals by disallowing values having scale (e.g. `1.1`). + require(bigDecimal.scale <= 0) + // `DecimalType` conversion can fail when + // 1. The precision is bigger than 38. + // 2. scale is bigger than precision. + fromDecimal(Decimal(bigDecimal)) + } + + val dateTry = Try { + // try and parse the date, if no exception occurs this is a candidate to be resolved as + // DateType + dateFormatter.parse(raw) + // SPARK-23436: Casting the string to date may still return null if a bad Date is provided. + // This can happen since DateFormat.parse may not use the entire text of the given string: + // so if there are extra-characters after the date, it returns correctly. + // We need to check that we can cast the raw string since we later can use Cast to get + // the partition values with the right DataType (see + // org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning) + val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval() + // Disallow DateType if the cast returned null + require(dateValue != null) + DateType + } + + val timestampTry = Try { + val unescapedRaw = unescapePathName(raw) + // the inferred data type is consistent with the default timestamp type + val timestampType = TimestampType + // try and parse the date, if no exception occurs this is a candidate to be resolved as TimestampType + timestampFormatter.parse(unescapedRaw) + + // SPARK-23436: see comment for date + val timestampValue = Cast(Literal(unescapedRaw), timestampType, Some(zoneId.getId)).eval() + // Disallow TimestampType if the cast returned null + require(timestampValue != null) + timestampType + } + + if (typeInference) { + // First tries integral types + Try({ Integer.parseInt(raw); IntegerType }) + .orElse(Try { JLong.parseLong(raw); LongType }) + .orElse(decimalTry) + // Then falls back to fractional types + .orElse(Try { JDouble.parseDouble(raw); DoubleType }) + // Then falls back to date/timestamp types + .orElse(timestampTry) + .orElse(dateTry) + // Then falls back to string + .getOrElse { + if (raw == DEFAULT_PARTITION_PATH) NullType else StringType + } + } else { + if (raw == DEFAULT_PARTITION_PATH) NullType else StringType + } + } + + def castPartValueToDesiredType( + desiredType: DataType, + value: String, + zoneId: ZoneId): Any = desiredType match { + case _ if value == DEFAULT_PARTITION_PATH => null + case NullType => null + case StringType => UTF8String.fromString(unescapePathName(value)) + case IntegerType => Integer.parseInt(value) + case LongType => JLong.parseLong(value) + case DoubleType => JDouble.parseDouble(value) + case _: DecimalType => Literal(new JBigDecimal(value)).value + case DateType => + Cast(Literal(value), DateType, Some(zoneId.getId)).eval() + // Timestamp types + case dt: TimestampType => + Try { + Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval() + }.getOrElse { + Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval() + } + case dt => throw new IllegalArgumentException(s"Unexpected type $dt") + } + + private def fromDecimal(d: Decimal): DecimalType = DecimalType(d.precision, d.scale) } diff --git a/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestReflectUtil.java b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestReflectUtil.java index 284b2aaf1f81..0d1867047847 100644 --- a/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestReflectUtil.java +++ b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestReflectUtil.java @@ -40,7 +40,6 @@ public void testDataSourceWriterExtraCommitMetadata() throws Exception { InsertIntoStatement statement = (InsertIntoStatement) spark.sessionState().sqlParser().parsePlan(insertIntoSql); InsertIntoStatement newStatment = ReflectUtil.createInsertInto( - spark.version().startsWith("3.0"), statement.table(), statement.partitionSpec(), scala.collection.immutable.List.empty(), @@ -50,9 +49,5 @@ public void testDataSourceWriterExtraCommitMetadata() throws Exception { Assertions.assertTrue( ((UnresolvedRelation)newStatment.table()).multipartIdentifier().contains("test_reflect_util")); - - if (!spark.version().startsWith("3.0")) { - Assertions.assertTrue(newStatment.userSpecifiedCols().isEmpty()); - } } } diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml index 763b348dc1aa..ae7cb8e0b8af 100644 --- a/hudi-spark-datasource/pom.xml +++ b/hudi-spark-datasource/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-sync/hudi-dla-sync/pom.xml b/hudi-sync/hudi-dla-sync/pom.xml index 0a4e1d2ba0c8..afb5717318f9 100644 --- a/hudi-sync/hudi-dla-sync/pom.xml +++ b/hudi-sync/hudi-dla-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml @@ -116,24 +116,6 @@ org.apache.hive hive-exec ${hive.version} - - - commons-lang - commons-lang - - - org.apache.commons - commons-lang3 - - - org.apache.zookeeper - zookeeper - - - org.pentaho - * - - diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java index 20f94f01ef0b..77d7362fa816 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java @@ -287,6 +287,11 @@ public void updatePartitionsToTable(String tableName, List changedPartit } } + @Override + public void dropPartitionsToTable(String tableName, List partitionsToDrop) { + throw new UnsupportedOperationException("Not support dropPartitionsToTable yet."); + } + public Map, String> scanTablePartitions(String tableName) { String sql = constructShowPartitionSQL(tableName); Statement stmt = null; diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index 23ba1f96971e..7cc51a306813 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml @@ -104,16 +104,7 @@ ${hive.groupid} hive-service ${hive.version} - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - + test ${hive.groupid} @@ -160,6 +151,13 @@ + + org.apache.logging.log4j + log4j-core + 2.17.0 + test + + org.eclipse.jetty.aggregate jetty-all diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 0c2abdbf571a..8dc3c6e0e468 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -70,6 +70,9 @@ public class HiveSyncConfig implements Serializable { + "org.apache.hudi input format.") public Boolean usePreApacheInputFormat = false; + @Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in metastore", required = false) + public String bucketSpec; + @Deprecated @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") public Boolean useJdbc = true; @@ -120,6 +123,9 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--with-operation-field"}, description = "Whether to include the '_hoodie_operation' field in the metadata fields") public Boolean withOperationField = false; + @Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.") + public Boolean isConditionalSync = false; + // enhance the similar function in child class public static HiveSyncConfig copy(HiveSyncConfig cfg) { HiveSyncConfig newConfig = new HiveSyncConfig(); @@ -132,6 +138,7 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; newConfig.jdbcUrl = cfg.jdbcUrl; newConfig.tableName = cfg.tableName; + newConfig.bucketSpec = cfg.bucketSpec; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; newConfig.supportTimestamp = cfg.supportTimestamp; @@ -143,6 +150,7 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.syncAsSparkDataSourceTable = cfg.syncAsSparkDataSourceTable; newConfig.sparkSchemaLengthThreshold = cfg.sparkSchemaLengthThreshold; newConfig.withOperationField = cfg.withOperationField; + newConfig.isConditionalSync = cfg.isConditionalSync; return newConfig; } @@ -151,6 +159,7 @@ public String toString() { return "HiveSyncConfig{" + "databaseName='" + databaseName + '\'' + ", tableName='" + tableName + '\'' + + ", bucketSpec='" + bucketSpec + '\'' + ", baseFileFormat='" + baseFileFormat + '\'' + ", hiveUser='" + hiveUser + '\'' + ", hivePass='" + hivePass + '\'' @@ -174,6 +183,11 @@ public String toString() { + ", syncAsSparkDataSourceTable=" + syncAsSparkDataSourceTable + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold + ", withOperationField=" + withOperationField + + ", isConditionalSync=" + isConditionalSync + '}'; } + + public static String getBucketSpec(String bucketCols, int bucketNum) { + return "CLUSTERED BY (" + bucketCols + " INTO " + bucketNum + " BUCKETS"; + } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 6a1d930c5e89..f8b015714f95 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -18,6 +18,11 @@ package org.apache.hudi.hive; +import com.beust.jcommander.JCommander; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; @@ -28,16 +33,9 @@ import org.apache.hudi.hive.util.ConfigUtils; import org.apache.hudi.hive.util.HiveSchemaUtil; import org.apache.hudi.hive.util.Parquet2SparkSchemaUtils; - import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.hudi.sync.common.AbstractSyncTool; - -import com.beust.jcommander.JCommander; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.GroupType; @@ -166,6 +164,9 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Check if the necessary table exists boolean tableExists = hoodieHiveClient.doesTableExist(tableName); + // check if isDropPartition + boolean isDropPartition = hoodieHiveClient.isDropPartition(); + // Get the parquet schema for this table looking at the latest commit MessageType schema = hoodieHiveClient.getDataSchema(); @@ -178,8 +179,9 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, && !readAsOptimized) { cfg.syncAsSparkDataSourceTable = false; } + // Sync schema if needed - syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema); + boolean schemaChanged = syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema); LOG.info("Schema sync complete. Syncing partitions for " + tableName); // Get the last time we successfully synced partitions @@ -192,8 +194,11 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size()); // Sync the partitions if needed - syncPartitions(tableName, writtenPartitionsSince); - hoodieHiveClient.updateLastCommitTimeSynced(tableName); + boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition); + boolean meetSyncConditions = schemaChanged || partitionsChanged; + if (!cfg.isConditionalSync || meetSyncConditions) { + hoodieHiveClient.updateLastCommitTimeSynced(tableName); + } LOG.info("Sync complete for " + tableName); } @@ -204,7 +209,7 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, * @param tableExists - does table exist * @param schema - extracted schema */ - private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, + private boolean syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, boolean readAsOptimized, MessageType schema) { // Append spark table properties & serde properties Map tableProperties = ConfigUtils.toMap(cfg.tableProperties); @@ -215,6 +220,7 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); } + boolean schemaChanged = false; // Check and sync schema if (!tableExists) { LOG.info("Hive table " + tableName + " is not found. Creating it"); @@ -236,6 +242,7 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi // /ql/exec/DDLTask.java#L3488 hoodieHiveClient.createTable(tableName, schema, inputFormatClassName, outputFormatClassName, serDeFormatClassName, serdeProperties, tableProperties); + schemaChanged = true; } else { // Check if the table schema has evolved Map tableSchema = hoodieHiveClient.getTableSchema(tableName); @@ -248,10 +255,12 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi hoodieHiveClient.updateTableProperties(tableName, tableProperties); LOG.info("Sync table properties for " + tableName + ", table properties is: " + cfg.tableProperties); } + schemaChanged = true; } else { LOG.info("No Schema difference for " + tableName); } } + return schemaChanged; } /** @@ -324,20 +333,36 @@ private Map getSparkSerdeProperties(boolean readAsOptimized) { * Syncs the list of storage partitions passed in (checks if the partition is in hive, if not adds it or if the * partition path does not match, it updates the partition path). */ - private void syncPartitions(String tableName, List writtenPartitionsSince) { + private boolean syncPartitions(String tableName, List writtenPartitionsSince, boolean isDropPartition) { + boolean partitionsChanged; try { List hivePartitions = hoodieHiveClient.scanTablePartitions(tableName); List partitionEvents = - hoodieHiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); + hoodieHiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, isDropPartition); + List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); - LOG.info("New Partitions " + newPartitions); - hoodieHiveClient.addPartitionsToTable(tableName, newPartitions); + if (!newPartitions.isEmpty()) { + LOG.info("New Partitions " + newPartitions); + hoodieHiveClient.addPartitionsToTable(tableName, newPartitions); + } + List updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE); - LOG.info("Changed Partitions " + updatePartitions); - hoodieHiveClient.updatePartitionsToTable(tableName, updatePartitions); + if (!updatePartitions.isEmpty()) { + LOG.info("Changed Partitions " + updatePartitions); + hoodieHiveClient.updatePartitionsToTable(tableName, updatePartitions); + } + + List dropPartitions = filterPartitions(partitionEvents, PartitionEventType.DROP); + if (!dropPartitions.isEmpty()) { + LOG.info("Drop Partitions " + dropPartitions); + hoodieHiveClient.dropPartitionsToTable(tableName, dropPartitions); + } + + partitionsChanged = !updatePartitions.isEmpty() || !newPartitions.isEmpty() || !dropPartitions.isEmpty(); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to sync partitions for table " + tableName, e); } + return partitionsChanged; } private List filterPartitions(List events, PartitionEventType eventType) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 265ab750d5ae..acaf6caa18e6 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -19,6 +19,7 @@ package org.apache.hudi.hive; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -122,6 +123,14 @@ public void updatePartitionsToTable(String tableName, List changedPartit ddlExecutor.updatePartitionsToTable(tableName, changedPartitions); } + /** + * Partition path has changed - drop the following partitions. + */ + @Override + public void dropPartitionsToTable(String tableName, List partitionsToDrop) { + ddlExecutor.dropPartitionsToTable(tableName, partitionsToDrop); + } + /** * Update the table properties to the table. */ @@ -147,6 +156,14 @@ public void updateTableProperties(String tableName, Map tablePro * Generate a list of PartitionEvent based on the changes required. */ List getPartitionEvents(List tablePartitions, List partitionStoragePartitions) { + return getPartitionEvents(tablePartitions, partitionStoragePartitions, false); + } + + /** + * Iterate over the storage partitions and find if there are any new partitions that need to be added or updated. + * Generate a list of PartitionEvent based on the changes required. + */ + List getPartitionEvents(List tablePartitions, List partitionStoragePartitions, boolean isDropPartition) { Map paths = new HashMap<>(); for (Partition tablePartition : tablePartitions) { List hivePartitionValues = tablePartition.getValues(); @@ -161,12 +178,17 @@ List getPartitionEvents(List tablePartitions, List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); - if (!storagePartitionValues.isEmpty()) { - String storageValue = String.join(", ", storagePartitionValues); - if (!paths.containsKey(storageValue)) { - events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); - } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { - events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + + if (isDropPartition) { + events.add(PartitionEvent.newPartitionDropEvent(storagePartition)); + } else { + if (!storagePartitionValues.isEmpty()) { + String storageValue = String.join(", ", storagePartitionValues); + if (!paths.containsKey(storageValue)) { + events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); + } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { + events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + } } } } @@ -310,13 +332,15 @@ List getAllTables(String db) throws Exception { @Override public void updateLastCommitTimeSynced(String tableName) { // Set the last commit time from the TBLproperties - String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); - try { - Table table = client.getTable(syncConfig.databaseName, tableName); - table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced); - client.alter_table(syncConfig.databaseName, tableName, table); - } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e); + Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::getTimestamp); + if (lastCommitSynced.isPresent()) { + try { + Table table = client.getTable(syncConfig.databaseName, tableName); + table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get()); + client.alter_table(syncConfig.databaseName, tableName, table); + } catch (Exception e) { + throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e); + } } } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SchemaDifference.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SchemaDifference.java index e207b85f8825..f48208a439fd 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SchemaDifference.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SchemaDifference.java @@ -28,7 +28,6 @@ import java.util.Map; import java.util.StringJoiner; - /** * Represents the schema difference between the storage schema and hive table schema. */ diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java index 0e1e223aab55..dc37d92b2498 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java @@ -81,5 +81,13 @@ public void createTable(String tableName, MessageType storageSchema, String inpu */ public void updatePartitionsToTable(String tableName, List changedPartitions); + /** + * Drop partitions for a given table. + * + * @param tableName + * @param partitionsToDrop + */ + public void dropPartitionsToTable(String tableName, List partitionsToDrop); + public void close(); } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java index 37aa54abd33b..c3c5226cd0a4 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java @@ -24,6 +24,7 @@ import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.hudi.hive.PartitionValueExtractor; +import org.apache.hudi.hive.util.HivePartitionUtil; import org.apache.hudi.hive.util.HiveSchemaUtil; import org.apache.hadoop.fs.FileSystem; @@ -226,6 +227,26 @@ public void updatePartitionsToTable(String tableName, List changedPartit } } + @Override + public void dropPartitionsToTable(String tableName, List partitionsToDrop) { + if (partitionsToDrop.isEmpty()) { + LOG.info("No partitions to drop for " + tableName); + return; + } + + LOG.info("Drop partitions " + partitionsToDrop.size() + " on " + tableName); + try { + for (String dropPartition : partitionsToDrop) { + String partitionClause = HivePartitionUtil.getPartitionClauseForDrop(dropPartition, partitionValueExtractor, syncConfig); + client.dropPartition(syncConfig.databaseName, tableName, partitionClause, false); + LOG.info("Drop partition " + dropPartition + " on " + tableName); + } + } catch (TException e) { + LOG.error(syncConfig.databaseName + "." + tableName + " drop partition failed", e); + throw new HoodieHiveSyncException(syncConfig.databaseName + "." + tableName + " drop partition failed", e); + } + } + @Override public void close() { if (client != null) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java index e2635eef0b8f..a4debfbab960 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hudi.hive.util.HivePartitionUtil; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -126,6 +127,26 @@ public Map getTableSchema(String tableName) { } } + @Override + public void dropPartitionsToTable(String tableName, List partitionsToDrop) { + if (partitionsToDrop.isEmpty()) { + LOG.info("No partitions to drop for " + tableName); + return; + } + + LOG.info("Drop partitions " + partitionsToDrop.size() + " on " + tableName); + try { + for (String dropPartition : partitionsToDrop) { + String partitionClause = HivePartitionUtil.getPartitionClauseForDrop(dropPartition, partitionValueExtractor, config); + metaStoreClient.dropPartition(config.databaseName, tableName, partitionClause, false); + LOG.info("Drop partition " + dropPartition + " on " + tableName); + } + } catch (Exception e) { + LOG.error(config.databaseName + "." + tableName + " drop partition failed", e); + throw new HoodieHiveSyncException(config.databaseName + "." + tableName + " drop partition failed", e); + } + } + @Override public void close() { if (metaStoreClient != null) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java index 1603191c6694..997d6e087c1b 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.hive.ddl; +import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; + import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; @@ -31,7 +33,9 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; @@ -141,6 +145,53 @@ public Map getTableSchema(String tableName) { } } + @Override + public void dropPartitionsToTable(String tableName, List partitionsToDrop) { + if (partitionsToDrop.isEmpty()) { + LOG.info("No partitions to add for " + tableName); + return; + } + LOG.info("Adding partitions " + partitionsToDrop.size() + " to table " + tableName); + List sqls = constructDropPartitions(tableName, partitionsToDrop); + sqls.stream().forEach(sql -> runSQL(sql)); + } + + private List constructDropPartitions(String tableName, List partitions) { + if (config.batchSyncNum <= 0) { + throw new HoodieHiveSyncException("batch-sync-num for sync hive table must be greater than 0, pls check your parameter"); + } + List result = new ArrayList<>(); + int batchSyncPartitionNum = config.batchSyncNum; + StringBuilder alterSQL = getAlterTableDropPrefix(tableName); + + for (int i = 0; i < partitions.size(); i++) { + String partitionClause = getPartitionClause(partitions.get(i)); + if (i == 0) { + alterSQL.append(" PARTITION (").append(partitionClause).append(")"); + } else { + alterSQL.append(", PARTITION (").append(partitionClause).append(")"); + } + + if ((i + 1) % batchSyncPartitionNum == 0) { + result.add(alterSQL.toString()); + alterSQL = getAlterTableDropPrefix(tableName); + } + } + // add left partitions to result + if (partitions.size() % batchSyncPartitionNum != 0) { + result.add(alterSQL.toString()); + } + return result; + } + + public StringBuilder getAlterTableDropPrefix(String tableName) { + StringBuilder alterSQL = new StringBuilder("ALTER TABLE "); + alterSQL.append(HIVE_ESCAPE_CHARACTER).append(config.databaseName) + .append(HIVE_ESCAPE_CHARACTER).append(".").append(HIVE_ESCAPE_CHARACTER) + .append(tableName).append(HIVE_ESCAPE_CHARACTER).append(" DROP IF EXISTS "); + return alterSQL; + } + @Override public void close() { try { @@ -151,4 +202,4 @@ public void close() { LOG.error("Could not close connection ", e); } } -} +} \ No newline at end of file diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java index 6fbcfa93752e..a1cc7721053c 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java @@ -39,13 +39,14 @@ import java.util.Map; import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; -/* -This class adds functionality for all query based DDLExecutors. The classes extending it only have to provide runSQL(sql) functions. + +/** + * This class adds functionality for all query based DDLExecutors. The classes extending it only have to provide runSQL(sql) functions. */ public abstract class QueryBasedDDLExecutor implements DDLExecutor { private static final Logger LOG = LogManager.getLogger(QueryBasedDDLExecutor.class); private final HiveSyncConfig config; - private final PartitionValueExtractor partitionValueExtractor; + public final PartitionValueExtractor partitionValueExtractor; private final FileSystem fs; public QueryBasedDDLExecutor(HiveSyncConfig config, FileSystem fs) { @@ -159,7 +160,7 @@ private StringBuilder getAlterTablePrefix(String tableName) { return alterSQL; } - private String getPartitionClause(String partition) { + public String getPartitionClause(String partition) { List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); ValidationUtils.checkArgument(config.partitionFields.size() == partitionValues.size(), "Partition key parts " + config.partitionFields + " does not match with partition values " + partitionValues diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java new file mode 100644 index 000000000000..27e3a73cee5a --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java @@ -0,0 +1,51 @@ +/* + * 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 org.apache.hudi.hive.util; + +import org.apache.hudi.common.util.PartitionPathEncodeUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.PartitionValueExtractor; + +import java.util.ArrayList; +import java.util.List; + +public class HivePartitionUtil { + + /** + * Build String, example as year=2021/month=06/day=25 + */ + public static String getPartitionClauseForDrop(String partition, PartitionValueExtractor partitionValueExtractor, HiveSyncConfig config) { + List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); + ValidationUtils.checkArgument(config.partitionFields.size() == partitionValues.size(), + "Partition key parts " + config.partitionFields + " does not match with partition values " + partitionValues + + ". Check partition strategy. "); + List partBuilder = new ArrayList<>(); + for (int i = 0; i < config.partitionFields.size(); i++) { + String partitionValue = partitionValues.get(i); + // decode the partition before sync to hive to prevent multiple escapes of HIVE + if (config.decodePartition) { + // This is a decode operator for encode in KeyGenUtils#getRecordPartitionPath + partitionValue = PartitionPathEncodeUtils.unescapePathName(partitionValue); + } + partBuilder.add(config.partitionFields.get(i) + "=" + partitionValue); + } + return String.join("/", partBuilder); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java index 7cef6abf8cb3..2d700596f055 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java @@ -471,6 +471,9 @@ public static String generateCreateDDL(String tableName, MessageType storageSche if (!config.partitionFields.isEmpty()) { sb.append(" PARTITIONED BY (").append(partitionsStr).append(")"); } + if (config.bucketSpec != null) { + sb.append(' ' + config.bucketSpec + ' '); + } sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'"); if (serdeProperties != null && !serdeProperties.isEmpty()) { sb.append(" WITH SERDEPROPERTIES (").append(propertyToString(serdeProperties)).append(")"); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 64043a5bb29f..c515c9dda138 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.NetworkTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; @@ -737,6 +738,99 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).size()); } + @ParameterizedTest + @MethodSource("syncMode") + public void testDropPartitionKeySync(String syncMode) throws Exception { + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + String instantTime = "100"; + HiveTestUtil.createCOWTable(instantTime, 1, true); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); + // Lets do the sync + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), + "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + hiveClient.getDataSchema().getColumns().size() + 1, + "Hive Schema should match the table schema + partition field"); + assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + "Table partitions should match the number of partitions we wrote"); + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + "The last commit that was synced should be updated in the TBLPROPERTIES"); + + // Adding of new partitions + List newPartition = Arrays.asList("2050/01/01"); + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); + assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + "No new partition should be added"); + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, newPartition); + assertEquals(2, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + "New partition should be added"); + + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + + // Drop 1 partition. + ddlExecutor.runSQL("ALTER TABLE `" + hiveSyncConfig.tableName + + "` DROP PARTITION (`datestr`='2050-01-01')"); + + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + assertEquals(1, hivePartitions.size(), + "Table should have 1 partition because of the drop 1 partition"); + } + + @ParameterizedTest + @MethodSource("syncMode") + public void testDropPartition(String syncMode) throws Exception { + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + String instantTime = "100"; + HiveTestUtil.createCOWTable(instantTime, 1, true); + + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); + // Lets do the sync + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), + "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + hiveClient.getDataSchema().getColumns().size() + 1, + "Hive Schema should match the table schema + partition field"); + List partitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + assertEquals(1, partitions.size(), + "Table partitions should match the number of partitions we wrote"); + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + "The last commit that was synced should be updated in the TBLPROPERTIES"); + String partitiontoDelete = partitions.get(0).getValues().get(0).replace("-","/"); + // create a replace commit to delete current partitions+ + HiveTestUtil.createReplaceCommit("101", partitiontoDelete, WriteOperationType.DELETE_PARTITION, true, true); + + // sync drop partitins + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + assertEquals(0, hivePartitions.size(), + "Table should have 0 partition because of the drop the only one partition"); + } + @ParameterizedTest @MethodSource("syncMode") public void testNonPartitionedSync(String syncMode) throws Exception { @@ -1017,4 +1111,35 @@ public void testTypeConverter(String syncMode) throws Exception { .containsValue("BIGINT"), errorMsg); ddlExecutor.runSQL(dropTableSql); } + + @ParameterizedTest + @MethodSource("syncMode") + public void testSyncWithoutDiffs(String syncMode) throws Exception { + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.isConditionalSync = true; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + String tableName = HiveTestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + + String commitTime0 = "100"; + String commitTime1 = "101"; + String commitTime2 = "102"; + HiveTestUtil.createMORTable(commitTime0, commitTime1, 2, true, true); + + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + assertTrue(hiveClient.doesTableExist(tableName)); + assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); + + HiveTestUtil.addMORPartitions(0, true, true, true, ZonedDateTime.now().plusDays(2), commitTime1, commitTime2); + + tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + hiveClient = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); + } + } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index a3bc2268dcac..34158d4e7899 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -28,8 +28,10 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; @@ -176,6 +178,17 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo createCommitFile(commitMetadata, instantTime); } + public static void createReplaceCommit(String instantTime, String partitions, WriteOperationType type, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata) + throws IOException { + HoodieReplaceCommitMetadata replaceCommitMetadata = new HoodieReplaceCommitMetadata(); + addSchemaToCommitMetadata(replaceCommitMetadata, isParquetSchemaSimple, useSchemaFromCommitMetadata); + replaceCommitMetadata.setOperationType(type); + Map> partitionToReplaceFileIds = new HashMap<>(); + partitionToReplaceFileIds.put(partitions, new ArrayList<>()); + replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + createReplaceCommitFile(replaceCommitMetadata, instantTime); + } + public static void createCOWTableWithSchema(String instantTime, String schemaFileName) throws IOException, URISyntaxException { Path path = new Path(hiveSyncConfig.basePath); @@ -442,6 +455,15 @@ public static void createCommitFile(HoodieCommitMetadata commitMetadata, String fsout.close(); } + public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException { + byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); + Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + HoodieTimeline.makeReplaceFileName(instantTime)); + FSDataOutputStream fsout = fileSystem.create(fullPath, true); + fsout.write(bytes); + fsout.close(); + } + public static void createCommitFileWithSchema(HoodieCommitMetadata commitMetadata, String instantTime, boolean isSimpleSchema) throws IOException { addSchemaToCommitMetadata(commitMetadata, isSimpleSchema, true); createCommitFile(commitMetadata, instantTime); diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index 5484290654a1..1f1abb4f177f 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java index ce4720ac0090..98b11f2f37cc 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java @@ -20,16 +20,18 @@ import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; @@ -98,6 +100,8 @@ public abstract void createTable(String tableName, MessageType storageSchema, public abstract void updatePartitionsToTable(String tableName, List changedPartitions); + public abstract void dropPartitionsToTable(String tableName, List partitionsToDrop); + public void updateTableProperties(String tableName, Map tableProperties) {} public abstract Map getTableSchema(String tableName); @@ -155,6 +159,25 @@ public MessageType getDataSchema() { } } + public boolean isDropPartition() { + try { + Option hoodieCommitMetadata; + if (withOperationField) { + hoodieCommitMetadata = new TableSchemaResolver(metaClient, true).getLatestCommitMetadata(); + } else { + hoodieCommitMetadata = new TableSchemaResolver(metaClient).getLatestCommitMetadata(); + } + + if (hoodieCommitMetadata.isPresent() + && WriteOperationType.DELETE_PARTITION.equals(hoodieCommitMetadata.get().getOperationType())) { + return true; + } + } catch (Exception e) { + throw new HoodieSyncException("Failed to get commit metadata", e); + } + return false; + } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") public List getPartitionsWrittenToSince(Option lastCommitTimeSynced) { if (!lastCommitTimeSynced.isPresent()) { @@ -224,7 +247,7 @@ private MessageType readSchemaFromLogFile(Option lastCompactionCo public static class PartitionEvent { public enum PartitionEventType { - ADD, UPDATE + ADD, UPDATE, DROP } public PartitionEventType eventType; @@ -242,5 +265,9 @@ public static PartitionEvent newPartitionAddEvent(String storagePartition) { public static PartitionEvent newPartitionUpdateEvent(String storagePartition) { return new PartitionEvent(PartitionEventType.UPDATE, storagePartition); } + + public static PartitionEvent newPartitionDropEvent(String storagePartition) { + return new PartitionEvent(PartitionEventType.DROP, storagePartition); + } } } diff --git a/hudi-sync/pom.xml b/hudi-sync/pom.xml index be1bdaea4c90..776a19416844 100644 --- a/hudi-sync/pom.xml +++ b/hudi-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index 011cf8d76589..cb2c643c7874 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 051e2602be5b..4744fbb6b4e1 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -33,7 +33,6 @@ import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.timeline.service.handlers.BaseFileHandler; import org.apache.hudi.timeline.service.handlers.FileSliceHandler; @@ -42,6 +41,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import io.javalin.BadRequestResponse; import io.javalin.Context; import io.javalin.Handler; import io.javalin.Javalin; @@ -500,20 +500,26 @@ public void handle(@NotNull Context context) throws Exception { if (refreshCheck) { long beginFinalCheck = System.currentTimeMillis(); - String errMsg = - "Last known instant from client was " - + context.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, - HoodieTimeline.INVALID_INSTANT_TS) - + " but server has the following timeline " - + viewManager.getFileSystemView(context.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM)) - .getTimeline().getInstants().collect(Collectors.toList()); - ValidationUtils.checkArgument(!isLocalViewBehind(context), errMsg); + if (isLocalViewBehind(context)) { + String errMsg = + "Last known instant from client was " + + context.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, + HoodieTimeline.INVALID_INSTANT_TS) + + " but server has the following timeline " + + viewManager.getFileSystemView(context.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM)) + .getTimeline().getInstants().collect(Collectors.toList()); + throw new BadRequestResponse(errMsg); + } long endFinalCheck = System.currentTimeMillis(); finalCheckTimeTaken = endFinalCheck - beginFinalCheck; } } catch (RuntimeException re) { success = false; - LOG.error("Got runtime exception servicing request " + context.queryString(), re); + if (re instanceof BadRequestResponse) { + LOG.warn("Bad request response due to client view behind server view. " + re.getMessage()); + } else { + LOG.error("Got runtime exception servicing request " + context.queryString(), re); + } throw re; } finally { long endTs = System.currentTimeMillis(); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 77c7870f5f3a..7b8257705146 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -321,8 +321,10 @@ public void close() { if (requestHandler != null) { this.requestHandler.stop(); } - this.app.stop(); - this.app = null; + if (this.app != null) { + this.app.stop(); + this.app = null; + } this.fsViewsManager.close(); LOG.info("Closed Timeline Service"); } diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index 089b7801baf8..2e68039c1322 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT 4.0.0 @@ -148,6 +148,23 @@ ${kafka.version} + + + org.apache.pulsar + pulsar-client + ${pulsar.version} + + + org.slf4j + slf4j-api + + + com.google.protobuf + protobuf-java + + + + log4j diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index 5f9b199fe695..8651e30c044c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -112,7 +112,7 @@ private boolean isUpsert() { public int dataImport(JavaSparkContext jsc, int retry) { this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) - : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + : UtilHelpers.readConfig(fs.getConf(), new Path(cfg.propsFilePath), cfg.configs).getProps(true); LOG.info("Starting data import with configs : " + props.toString()); int ret = -1; try { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java index 7f58d3c679f0..0cd80419d31e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java @@ -106,14 +106,14 @@ public static class Config implements Serializable { private Connection connection; protected final Config config; - private final ST incrementalPullSQLtemplate; + private final ST incrementalPullSQLTemplate; public HiveIncrementalPuller(Config config) throws IOException { this.config = config; validateConfig(config); String templateContent = FileIOUtils.readAsUTFString(this.getClass().getResourceAsStream("/IncrementalPull.sqltemplate")); - incrementalPullSQLtemplate = new ST(templateContent); + incrementalPullSQLTemplate = new ST(templateContent); } private void validateConfig(Config config) { @@ -165,19 +165,19 @@ public void saveDelta() throws IOException { stmt.close(); } } catch (SQLException e) { - LOG.error("Could not close the resultset opened ", e); + LOG.error("Could not close the resultSet opened ", e); } } } private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, Statement stmt) throws FileNotFoundException, SQLException { - incrementalPullSQLtemplate.add("tempDbTable", tempDbTable); - incrementalPullSQLtemplate.add("tempDbTablePath", tempDbTablePath); + incrementalPullSQLTemplate.add("tempDbTable", tempDbTable); + incrementalPullSQLTemplate.add("tempDbTablePath", tempDbTablePath); String storedAsClause = getStoredAsClause(); - incrementalPullSQLtemplate.add("storedAsClause", storedAsClause); + incrementalPullSQLTemplate.add("storedAsClause", storedAsClause); String incrementalSQL = new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next(); if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) { LOG.error("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable @@ -194,8 +194,8 @@ private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, S + "means its not pulling incrementally"); } - incrementalPullSQLtemplate.add("incrementalSQL", String.format(incrementalSQL, config.fromCommitTime)); - String sql = incrementalPullSQLtemplate.render(); + incrementalPullSQLTemplate.add("incrementalSQL", String.format(incrementalSQL, config.fromCommitTime)); + String sql = incrementalPullSQLTemplate.render(); // Check if the SQL is pulling from the right database executeStatement(sql, stmt); } @@ -208,13 +208,13 @@ private void initHiveBeelineProperties(Statement stmt) throws SQLException { LOG.info("Setting up Hive JDBC Session with properties"); // set the queue executeStatement("set mapred.job.queue.name=" + config.yarnQueueName, stmt); - // Set the inputformat to HoodieCombineHiveInputFormat + // Set the inputFormat to HoodieCombineHiveInputFormat executeStatement("set hive.input.format=org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat", stmt); // Allow queries without partition predicate executeStatement("set hive.strict.checks.large.query=false", stmt); - // Dont gather stats for the table created + // Don't gather stats for the table created executeStatement("set hive.stats.autogather=false", stmt); - // Set the hoodie modie + // Set the hoodie mode executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt); // Set the from commit time executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp=" + config.fromCommitTime, stmt); @@ -263,7 +263,7 @@ private String getTableLocation(String db, String table) { resultSet.close(); } } catch (SQLException e) { - LOG.error("Could not close the resultset opened ", e); + LOG.error("Could not close the resultSet opened ", e); } } return null; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java index 24e2828a5e2f..39d16e2f11b7 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java @@ -18,16 +18,13 @@ package org.apache.hudi.utilities; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.config.HoodieWriteConfig; - -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -61,9 +58,8 @@ public HoodieCleaner(Config cfg, JavaSparkContext jssc) { /* * Filesystem used. */ - FileSystem fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration()); this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) - : UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); + : UtilHelpers.readConfig(jssc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs).getProps(true); LOG.info("Creating Cleaner with configs : " + props.toString()); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 2bd9ab43a747..a4ee8089f831 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -18,11 +18,6 @@ package org.apache.hudi.utilities; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; @@ -31,10 +26,20 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieSparkTable; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -42,34 +47,34 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.Date; import java.util.List; import java.util.stream.Collectors; public class HoodieClusteringJob { + public static final String EXECUTE = "execute"; + public static final String SCHEDULE = "schedule"; + public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute"; private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class); private final Config cfg; private transient FileSystem fs; private TypedProperties props; private final JavaSparkContext jsc; - public static final String EXECUTE = "execute"; - public static final String SCHEDULE = "schedule"; - public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute"; + private final HoodieTableMetaClient metaClient; public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) { this.cfg = cfg; this.jsc = jsc; - this.props = cfg.propsFilePath == null + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) ? UtilHelpers.buildProperties(cfg.configs) : readConfigFromFileSystem(jsc, cfg); + this.metaClient = UtilHelpers.createMetaClient(jsc, cfg.basePath, true); } private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { - final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); - - return UtilHelpers - .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) - .getConfig(); + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); } public static class Config implements Serializable { @@ -77,8 +82,10 @@ public static class Config implements Serializable { public String basePath = null; @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) public String tableName = null; - @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only need when set --mode execute. " - + "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.", required = false) + @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only used when set --mode execute. " + + "If the instant time is not provided with --mode execute, " + + "the earliest scheduled clustering instant time is used by default. " + + "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.", required = false) public String clusteringInstantTime = null; @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) public int parallelism = 1; @@ -92,6 +99,10 @@ public static class Config implements Serializable { @Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering @desperate soon please use \"--mode schedule\" instead") public Boolean runSchedule = false; + @Parameter(names = {"--retry-last-failed-clustering-job", "-rc"}, description = "Take effect when using --mode/-m scheduleAndExecute. Set true means " + + "check, rollback and execute last failed clustering plan instead of planing a new clustering job directly.", required = false) + public Boolean retryLastFailedClusteringJob = false; + @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" means make a cluster plan; " + "Set \"execute\" means execute a cluster plan at given instant which means --instant-time is needed here; " + "Set \"scheduleAndExecute\" means make a cluster plan first and execute that plan immediately", required = false) @@ -100,6 +111,10 @@ public static class Config implements Serializable { @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; + @Parameter(names = {"--job-max-processing-time-ms", "-jt"}, description = "Take effect when using --mode/-m scheduleAndExecute and --retry-last-failed-clustering-job/-rc true. " + + "If maxProcessingTimeMs passed but clustering job is still unfinished, hoodie would consider this job as failed and relaunch.", required = false) + public long maxProcessingTimeMs = 0; + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + "hoodie client for clustering") public String propsFilePath = null; @@ -139,10 +154,6 @@ private static void validateRunningMode(Config cfg) { if (StringUtils.isNullOrEmpty(cfg.runningMode)) { cfg.runningMode = cfg.runSchedule ? SCHEDULE : EXECUTE; } - - if (cfg.runningMode.equalsIgnoreCase(EXECUTE) && cfg.clusteringInstantTime == null) { - throw new RuntimeException("--instant-time couldn't be null when executing clustering plan."); - } } public int cluster(int retry) { @@ -178,7 +189,6 @@ public int cluster(int retry) { } private String getSchemaFromLatestInstant() throws Exception { - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(cfg.basePath).setLoadActiveTimelineOnLoad(true).build(); TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) { throw new HoodieException("Cannot run clustering without any completed commits"); @@ -190,6 +200,20 @@ private String getSchemaFromLatestInstant() throws Exception { private int doCluster(JavaSparkContext jsc) throws Exception { String schemaStr = getSchemaFromLatestInstant(); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + if (StringUtils.isNullOrEmpty(cfg.clusteringInstantTime)) { + // Instant time is not specified + // Find the earliest scheduled clustering instant for execution + Option firstClusteringInstant = + metaClient.getActiveTimeline().firstInstant( + HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieInstant.State.REQUESTED); + if (firstClusteringInstant.isPresent()) { + cfg.clusteringInstantTime = firstClusteringInstant.get().getTimestamp(); + LOG.info("Found the earliest scheduled clustering instant which will be executed: " + + cfg.clusteringInstantTime); + } else { + throw new HoodieClusteringException("There is no scheduled clustering in the table."); + } + } Option commitMetadata = client.cluster(cfg.clusteringInstantTime, true).getCommitMetadata(); return handleErrors(commitMetadata.get(), cfg.clusteringInstantTime); @@ -216,17 +240,32 @@ private Option doSchedule(SparkRDDWriteClient clien return client.scheduleClustering(Option.empty()); } - public int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { + private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { LOG.info("Step 1: Do schedule"); String schemaStr = getSchemaFromLatestInstant(); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + Option instantTime = Option.empty(); + + if (cfg.retryLastFailedClusteringJob) { + HoodieSparkTable table = HoodieSparkTable.create(client.getConfig(), client.getEngineContext()); + HoodieTimeline inflightHoodieTimeline = table.getActiveTimeline().filterPendingReplaceTimeline().filterInflights(); + if (!inflightHoodieTimeline.empty()) { + HoodieInstant inflightClusteringInstant = inflightHoodieTimeline.lastInstant().get(); + Date clusteringStartTime = HoodieActiveTimeline.parseDateFromInstantTime(inflightClusteringInstant.getTimestamp()); + if (clusteringStartTime.getTime() + cfg.maxProcessingTimeMs < System.currentTimeMillis()) { + // if there has failed clustering, then we will use the failed clustering instant-time to trigger next clustering action which will rollback and clustering. + LOG.info("Found failed clustering instant at : " + inflightClusteringInstant + "; Will rollback the failed clustering and re-trigger again."); + instantTime = Option.of(inflightHoodieTimeline.lastInstant().get().getTimestamp()); + } else { + LOG.info(inflightClusteringInstant + " might still be in progress, will trigger a new clustering job."); + } + } + } - Option instantTime = doSchedule(client); - int result = instantTime.isPresent() ? 0 : -1; - - if (result == -1) { + instantTime = instantTime.isPresent() ? instantTime : doSchedule(client); + if (!instantTime.isPresent()) { LOG.info("Couldn't generate cluster plan"); - return result; + return -1; } LOG.info("The schedule instant time is " + instantTime.get()); @@ -238,7 +277,7 @@ public int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { private int handleErrors(HoodieCommitMetadata metadata, String instantTime) { List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> - e.getValue().stream()).collect(Collectors.toList()); + e.getValue().stream()).collect(Collectors.toList()); long errorsCount = writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum(); if (errorsCount == 0) { LOG.info(String.format("Table imported into hoodie with %s instant time.", instantTime)); @@ -248,5 +287,4 @@ private int handleErrors(HoodieCommitMetadata metadata, String instantTime) { LOG.error(String.format("Import failed with %d errors.", errorsCount)); return -1; } - } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java index 1996fb8b5f6e..706d1d9df4b9 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java @@ -21,9 +21,15 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieCompactionException; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; @@ -41,6 +47,7 @@ public class HoodieCompactor { private static final Logger LOG = LogManager.getLogger(HoodieCompactor.class); + private static ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private final Config cfg; private transient FileSystem fs; private TypedProperties props; @@ -55,11 +62,8 @@ public HoodieCompactor(JavaSparkContext jsc, Config cfg) { } private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { - final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); - - return UtilHelpers - .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) - .getConfig(); + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); } public static class Config implements Serializable { @@ -67,7 +71,7 @@ public static class Config implements Serializable { public String basePath = null; @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) public String tableName = null; - @Parameter(names = {"--instant-time", "-it"}, description = "Compaction Instant time", required = true) + @Parameter(names = {"--instant-time", "-it"}, description = "Compaction Instant time", required = false) public String compactionInstantTime = null; @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = true) public int parallelism = 1; @@ -134,6 +138,21 @@ private int doCompact(JavaSparkContext jsc) throws Exception { String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); + // If no compaction instant is provided by --instant-time, find the earliest scheduled compaction + // instant from the active timeline + if (StringUtils.isNullOrEmpty(cfg.compactionInstantTime)) { + HoodieTableMetaClient metaClient = UtilHelpers.createMetaClient(jsc, cfg.basePath, true); + Option firstCompactionInstant = + metaClient.getActiveTimeline().firstInstant( + HoodieTimeline.COMPACTION_ACTION, HoodieInstant.State.REQUESTED); + if (firstCompactionInstant.isPresent()) { + cfg.compactionInstantTime = firstCompactionInstant.get().getTimestamp(); + LOG.info("Found the earliest scheduled compaction instant which will be executed: " + + cfg.compactionInstantTime); + } else { + throw new HoodieCompactionException("There is no scheduled compaction in the table."); + } + } JavaRDD writeResponse = client.compact(cfg.compactionInstantTime); return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse); } @@ -142,6 +161,10 @@ private int doSchedule(JavaSparkContext jsc) throws Exception { // Get schema. SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Option.of(cfg.strategyClassName), props); + if (StringUtils.isNullOrEmpty(cfg.compactionInstantTime)) { + throw new IllegalArgumentException("No instant time is provided for scheduling compaction. " + + "Please specify the compaction instant time by using --instant-time."); + } client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty()); return 0; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieRepairTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieRepairTool.java new file mode 100644 index 000000000000..d7fa70889fb8 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieRepairTool.java @@ -0,0 +1,582 @@ +/* + * 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 org.apache.hudi.utilities; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.metadata.FileSystemBackedTableMetadata; +import org.apache.hudi.table.repair.RepairUtils; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.io.Serializable; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import scala.Tuple2; + +/** + * A tool with spark-submit to repair Hudi table by finding and deleting dangling + * base and log files. + *

+ * You can run this tool with the following command: + * ``` + * spark-submit \ + * --class org.apache.hudi.utilities.HoodieRepairTool \ + * --driver-memory 4g \ + * --executor-memory 1g \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.sql.catalogImplementation=hive \ + * --conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ + * --packages org.apache.spark:spark-avro_2.12:3.1.2 \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.12-0.11.0-SNAPSHOT.jar \ + * --mode dry_run \ + * --base-path base_path \ + * --assume-date-partitioning + * ``` + *

+ * You can specify the running mode of the tool through `--mode`. + * There are three modes of the {@link HoodieRepairTool}: + * - REPAIR ("repair"): repairs the table by removing dangling data and log files not belonging to any commit. + * The removed files are going to be backed up at the backup path provided, in case recovery is needed. + * In this mode, backup path is required through `--backup-path`. You can also provide a range for repairing + * only the instants within the range, through `--start-instant-time` and `--end-instant-time`. You can also + * specify only one of them. If no range is provided, all instants are going to be repaired. + *

+ * Example command: + * ``` + * spark-submit \ + * --class org.apache.hudi.utilities.HoodieRepairTool \ + * --driver-memory 4g \ + * --executor-memory 1g \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.sql.catalogImplementation=hive \ + * --conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ + * --packages org.apache.spark:spark-avro_2.12:3.1.2 \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.12-0.11.0-SNAPSHOT.jar \ + * --mode repair \ + * --base-path base_path \ + * --backup-path backup_path \ + * --start-instant-time ts1 \ + * --end-instant-time ts2 \ + * --assume-date-partitioning + * ``` + *

+ * - DRY_RUN ("dry_run"): only looks for dangling data and log files. You can also provide a range for looking + * at only the instants within the range, through `--start-instant-time` and `--end-instant-time`. You can also + * specify only one of them. If no range is provided, all instants are going to be scanned. + *

+ * Example command: + * ``` + * spark-submit \ + * --class org.apache.hudi.utilities.HoodieRepairTool \ + * --driver-memory 4g \ + * --executor-memory 1g \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.sql.catalogImplementation=hive \ + * --conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ + * --packages org.apache.spark:spark-avro_2.12:3.1.2 \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.12-0.11.0-SNAPSHOT.jar \ + * --mode dry_run \ + * --base-path base_path \ + * --start-instant-time ts1 \ + * --end-instant-time ts2 \ + * --assume-date-partitioning + * ``` + *

+ * - UNDO ("undo"): undoes the repair by copying back the files from backup directory to the table base path. + * In this mode, backup path is required through `--backup-path`. + *

+ * Example command: + * ``` + * spark-submit \ + * --class org.apache.hudi.utilities.HoodieRepairTool \ + * --driver-memory 4g \ + * --executor-memory 1g \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.sql.catalogImplementation=hive \ + * --conf spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension \ + * --packages org.apache.spark:spark-avro_2.12:3.1.2 \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.12-0.11.0-SNAPSHOT.jar \ + * --mode undo \ + * --base-path base_path \ + * --backup-path backup_path + * ``` + */ +public class HoodieRepairTool { + + private static final Logger LOG = LogManager.getLogger(HoodieRepairTool.class); + private static final String BACKUP_DIR_PREFIX = "hoodie_repair_backup_"; + // Repair config + private final Config cfg; + // Properties with source, hoodie client, key generator etc. + private TypedProperties props; + // Spark context + private final JavaSparkContext jsc; + private final HoodieTableMetaClient metaClient; + private final FileSystemBackedTableMetadata tableMetadata; + + public HoodieRepairTool(JavaSparkContext jsc, Config cfg) { + if (cfg.propsFilePath != null) { + cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString(); + } + this.jsc = jsc; + this.cfg = cfg; + this.props = cfg.propsFilePath == null + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + this.metaClient = HoodieTableMetaClient.builder() + .setConf(jsc.hadoopConfiguration()).setBasePath(cfg.basePath) + .setLoadActiveTimelineOnLoad(true) + .build(); + this.tableMetadata = new FileSystemBackedTableMetadata( + new HoodieSparkEngineContext(jsc), + new SerializableConfiguration(jsc.hadoopConfiguration()), + cfg.basePath, cfg.assumeDatePartitioning); + } + + public void run() { + Option startingInstantOption = Option.ofNullable(cfg.startingInstantTime); + Option endingInstantOption = Option.ofNullable(cfg.endingInstantTime); + + if (startingInstantOption.isPresent() && endingInstantOption.isPresent()) { + LOG.info(String.format("Start repairing completed instants between %s and %s (inclusive)", + startingInstantOption.get(), endingInstantOption.get())); + } else if (startingInstantOption.isPresent()) { + LOG.info(String.format("Start repairing completed instants from %s (inclusive)", + startingInstantOption.get())); + } else if (endingInstantOption.isPresent()) { + LOG.info(String.format("Start repairing completed instants till %s (inclusive)", + endingInstantOption.get())); + } else { + LOG.info("Start repairing all completed instants"); + } + + try { + Mode mode = Mode.valueOf(cfg.runningMode.toUpperCase()); + switch (mode) { + case REPAIR: + LOG.info(" ****** The repair tool is in REPAIR mode, dangling data and logs files " + + "not belonging to any commit are going to be DELETED from the table ******"); + if (checkBackupPathForRepair() < 0) { + LOG.error("Backup path check failed."); + break; + } + doRepair(startingInstantOption, endingInstantOption, false); + break; + case DRY_RUN: + LOG.info(" ****** The repair tool is in DRY_RUN mode, " + + "only LOOKING FOR dangling data and log files from the table ******"); + doRepair(startingInstantOption, endingInstantOption, true); + break; + case UNDO: + if (checkBackupPathAgainstBasePath() < 0) { + LOG.error("Backup path check failed."); + break; + } + undoRepair(); + break; + default: + LOG.info("Unsupported running mode [" + cfg.runningMode + "], quit the job directly"); + } + } catch (IOException e) { + throw new HoodieIOException("Unable to repair table in " + cfg.basePath, e); + } + } + + public static void main(String[] args) { + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + final JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-table-repair", cfg.sparkMaster, cfg.sparkMemory); + try { + new HoodieRepairTool(jsc, cfg).run(); + } catch (Throwable throwable) { + LOG.error("Fail to run table repair for " + cfg.basePath, throwable); + } finally { + jsc.stop(); + } + } + + /** + * Copies the list of files from source base path to destination base path. + * The destination file path (base + relative) should not already exist. + * + * @param jsc {@link JavaSparkContext} instance. + * @param relativeFilePaths A {@link List} of relative file paths for copying. + * @param sourceBasePath Source base path. + * @param destBasePath Destination base path. + * @param parallelism Parallelism. + * @return {@code true} if all successful; {@code false} otherwise. + */ + static boolean copyFiles( + JavaSparkContext jsc, List relativeFilePaths, String sourceBasePath, + String destBasePath, int parallelism) { + SerializableConfiguration conf = new SerializableConfiguration(jsc.hadoopConfiguration()); + List allResults = jsc.parallelize(relativeFilePaths, parallelism) + .mapPartitions(iterator -> { + List results = new ArrayList<>(); + FileSystem fs = FSUtils.getFs(destBasePath, conf.get()); + iterator.forEachRemaining(filePath -> { + boolean success = false; + Path destPath = new Path(destBasePath, filePath); + try { + if (!fs.exists(destPath)) { + FileIOUtils.copy(fs, new Path(sourceBasePath, filePath), destPath); + success = true; + } + } catch (IOException e) { + // Copy Fail + } finally { + results.add(success); + } + }); + return results.iterator(); + }) + .collect(); + return allResults.stream().reduce((r1, r2) -> r1 && r2).orElse(false); + } + + /** + * Lists all Hoodie files from the table base path. + * + * @param basePathStr Table base path. + * @param conf {@link Configuration} instance. + * @return An array of {@link FileStatus} of all Hoodie files. + * @throws IOException upon errors. + */ + static FileStatus[] listFilesFromBasePath(String basePathStr, Configuration conf) throws IOException { + final Set validFileExtensions = Arrays.stream(HoodieFileFormat.values()) + .map(HoodieFileFormat::getFileExtension).collect(Collectors.toCollection(HashSet::new)); + final String logFileExtension = HoodieFileFormat.HOODIE_LOG.getFileExtension(); + FileSystem fs = FSUtils.getFs(basePathStr, conf); + Path basePath = new Path(basePathStr); + + try { + return Arrays.stream(fs.listStatus(basePath, path -> { + String extension = FSUtils.getFileExtension(path.getName()); + return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension); + })).filter(FileStatus::isFile).toArray(FileStatus[]::new); + } catch (IOException e) { + // return empty FileStatus if partition does not exist already + if (!fs.exists(basePath)) { + return new FileStatus[0]; + } else { + throw e; + } + } + } + + /** + * Does repair, either in REPAIR or DRY_RUN mode. + * + * @param startingInstantOption {@link Option} of starting instant for scanning, can be empty. + * @param endingInstantOption {@link Option} of ending instant for scanning, can be empty. + * @param isDryRun Is dry run. + * @throws IOException upon errors. + */ + void doRepair( + Option startingInstantOption, Option endingInstantOption, boolean isDryRun) throws IOException { + // Scans all partitions to find base and log files in the base path + List allFilesInPartitions = getBaseAndLogFilePathsFromFileSystem(); + // Buckets the files based on instant time + // instant time -> relative paths of base and log files to base path + Map> instantToFilesMap = RepairUtils.tagInstantsOfBaseAndLogFiles( + metaClient.getBasePath(), + metaClient.getTableConfig().getBaseFileFormat().getFileExtension(), allFilesInPartitions); + List instantTimesToRepair = instantToFilesMap.keySet().stream() + .filter(instant -> (!startingInstantOption.isPresent() + || instant.compareTo(startingInstantOption.get()) >= 0) + && (!endingInstantOption.isPresent() + || instant.compareTo(endingInstantOption.get()) <= 0) + ).collect(Collectors.toList()); + + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); + // This assumes that the archived timeline only has completed instants so this is safe + archivedTimeline.loadCompletedInstantDetailsInMemory(); + + int parallelism = Math.max(Math.min(instantTimesToRepair.size(), cfg.parallelism), 1); + List>> instantFilesToRemove = + jsc.parallelize(instantTimesToRepair, parallelism) + .mapToPair(instantToRepair -> + new Tuple2<>(instantToRepair, RepairUtils.findInstantFilesToRemove(instantToRepair, + instantToFilesMap.get(instantToRepair), activeTimeline, archivedTimeline))) + .collect(); + + List>> instantsWithDanglingFiles = + instantFilesToRemove.stream().filter(e -> !e._2.isEmpty()).collect(Collectors.toList()); + printRepairInfo(instantTimesToRepair, instantsWithDanglingFiles); + if (!isDryRun) { + List relativeFilePathsToDelete = + instantsWithDanglingFiles.stream().flatMap(e -> e._2.stream()).collect(Collectors.toList()); + if (relativeFilePathsToDelete.size() > 0) { + parallelism = Math.max(Math.min(relativeFilePathsToDelete.size(), cfg.parallelism), 1); + if (!backupFiles(relativeFilePathsToDelete, parallelism)) { + LOG.error("Error backing up dangling files. Exiting..."); + return; + } + deleteFiles(relativeFilePathsToDelete, parallelism); + } + LOG.info(String.format("Table repair on %s is successful", cfg.basePath)); + } + } + + /** + * @return All hoodie files of the table from the file system. + * @throws IOException upon errors. + */ + List getBaseAndLogFilePathsFromFileSystem() throws IOException { + List allPartitionPaths = tableMetadata.getAllPartitionPaths() + .stream().map(partitionPath -> + FSUtils.getPartitionPath(cfg.basePath, partitionPath).toString()) + .collect(Collectors.toList()); + return tableMetadata.getAllFilesInPartitions(allPartitionPaths).values().stream() + .map(fileStatuses -> + Arrays.stream(fileStatuses).map(fileStatus -> fileStatus.getPath()).collect(Collectors.toList())) + .flatMap(list -> list.stream()) + .collect(Collectors.toList()); + } + + /** + * Undoes repair for UNDO mode. + * + * @throws IOException upon errors. + */ + void undoRepair() throws IOException { + FileSystem fs = metaClient.getFs(); + String backupPathStr = cfg.backupPath; + Path backupPath = new Path(backupPathStr); + if (!fs.exists(backupPath)) { + LOG.error("Cannot find backup path: " + backupPath); + return; + } + + List relativeFilePaths = Arrays.stream( + listFilesFromBasePath(backupPathStr, jsc.hadoopConfiguration())) + .map(fileStatus -> + FSUtils.getPartitionPath(backupPathStr, fileStatus.getPath().toString()).toString()) + .collect(Collectors.toList()); + int parallelism = Math.max(Math.min(relativeFilePaths.size(), cfg.parallelism), 1); + restoreFiles(relativeFilePaths, parallelism); + } + + /** + * Verifies the backup path for repair. + * If there is no backup path configured, creates a new one in temp folder. + * If the backup path already has files, throws an error to the user. + * If the backup path is within the table base path, throws an error too. + * + * @return {@code 0} if successful; {@code -1} otherwise. + * @throws IOException upon errors. + */ + int checkBackupPathForRepair() throws IOException { + if (cfg.backupPath == null) { + SecureRandom random = new SecureRandom(); + long randomLong = random.nextLong(); + cfg.backupPath = "/tmp/" + BACKUP_DIR_PREFIX + randomLong; + } + + Path backupPath = new Path(cfg.backupPath); + if (metaClient.getFs().exists(backupPath) + && metaClient.getFs().listStatus(backupPath).length > 0) { + LOG.error(String.format("Cannot use backup path %s: it is not empty", cfg.backupPath)); + return -1; + } + + return checkBackupPathAgainstBasePath(); + } + + /** + * Verifies the backup path against table base path. + * If the backup path is within the table base path, throws an error. + * + * @return {@code 0} if successful; {@code -1} otherwise. + */ + int checkBackupPathAgainstBasePath() { + if (cfg.backupPath == null) { + LOG.error("Backup path is not configured"); + return -1; + } + + if (cfg.backupPath.contains(cfg.basePath)) { + LOG.error(String.format("Cannot use backup path %s: it resides in the base path %s", + cfg.backupPath, cfg.basePath)); + return -1; + } + return 0; + } + + /** + * Backs up dangling files from table base path to backup path. + * + * @param relativeFilePaths A {@link List} of relative file paths for backup. + * @param parallelism Parallelism for copying. + * @return {@code true} if all successful; {@code false} otherwise. + */ + boolean backupFiles(List relativeFilePaths, int parallelism) { + return copyFiles(jsc, relativeFilePaths, cfg.basePath, cfg.backupPath, parallelism); + } + + /** + * Restores dangling files from backup path to table base path. + * + * @param relativeFilePaths A {@link List} of relative file paths for restoring. + * @param parallelism Parallelism for copying. + * @return {@code true} if all successful; {@code false} otherwise. + */ + boolean restoreFiles(List relativeFilePaths, int parallelism) { + return copyFiles(jsc, relativeFilePaths, cfg.backupPath, cfg.basePath, parallelism); + } + + /** + * Deletes files from table base path. + * + * @param relativeFilePaths A {@link List} of relative file paths for deleting. + * @param parallelism Parallelism for deleting. + */ + void deleteFiles(List relativeFilePaths, int parallelism) { + jsc.parallelize(relativeFilePaths, parallelism) + .mapPartitions(iterator -> { + FileSystem fs = metaClient.getFs(); + List results = new ArrayList<>(); + iterator.forEachRemaining(filePath -> { + boolean success = false; + try { + success = fs.delete(new Path(filePath), false); + } catch (IOException e) { + LOG.warn("Failed to delete file " + filePath); + } finally { + results.add(success); + } + }); + return results.iterator(); + }) + .collect(); + } + + /** + * Prints the repair info. + * + * @param instantTimesToRepair A list instant times in consideration for repair + * @param instantsWithDanglingFiles A list of instants with dangling files. + */ + private void printRepairInfo( + List instantTimesToRepair, List>> instantsWithDanglingFiles) { + int numInstantsToRepair = instantsWithDanglingFiles.size(); + LOG.warn("Number of instants verified based on the base and log files: " + + instantTimesToRepair.size()); + LOG.warn("Instant timestamps: " + instantTimesToRepair); + LOG.warn("Number of instants to repair: " + numInstantsToRepair); + if (numInstantsToRepair > 0) { + instantsWithDanglingFiles.forEach(e -> { + LOG.warn(" -> Instant " + numInstantsToRepair); + LOG.warn(" ** Removing files: " + e._2); + }); + } + } + + /** + * Reads config from the file system. + * + * @param jsc {@link JavaSparkContext} instance. + * @param cfg {@link Config} instance. + * @return the {@link TypedProperties} instance. + */ + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); + } + + public static class Config implements Serializable { + @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true) + public String basePath = null; + @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"repair\" means repairing the table " + + "by removing dangling data and log files not belonging to any commit; " + + "Set \"dry_run\" means only looking for dangling data and log files; " + + "Set \"undo\" means undoing the repair by copying back the files from backup directory", required = true) + public String runningMode = null; + @Parameter(names = {"--start-instant-time", "-si"}, description = "Starting Instant time " + + "for repair (inclusive)", required = false) + public String startingInstantTime = null; + @Parameter(names = {"--end-instant-time", "-ei"}, description = "Ending Instant time " + + "for repair (inclusive)", required = false) + public String endingInstantTime = null; + @Parameter(names = {"--backup-path", "-bp"}, description = "Backup path for storing dangling data " + + "and log files from the table", required = false) + public String backupPath = null; + @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for repair", required = false) + public int parallelism = 2; + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = false) + public String sparkMemory = "1g"; + @Parameter(names = {"--assume-date-partitioning", "-dp"}, description = "whether the partition path " + + "is date with three levels", required = false) + public Boolean assumeDatePartitioning = false; + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for table repair") + public String propsFilePath = null; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", + splitter = IdentitySplitter.class) + public List configs = new ArrayList<>(); + } + + public enum Mode { + // Repairs the table by removing dangling data and log files not belonging to any commit + REPAIR, + // Dry run by only looking for dangling data and log files + DRY_RUN, + // Undoes the repair by copying back the files from backup directory + UNDO + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 3771a7d34b5a..81c5caf82142 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -18,6 +18,7 @@ package org.apache.hudi.utilities; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -157,22 +158,12 @@ public static InitialCheckPointProvider createInitialCheckpointProvider( } } - /** - * - */ - public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List overriddenProps) { - DFSPropertiesConfiguration conf; - try { - conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath); - } catch (Exception e) { - conf = new DFSPropertiesConfiguration(); - LOG.warn("Unexpected error read props file at :" + cfgPath, e); - } - + public static DFSPropertiesConfiguration readConfig(Configuration hadoopConfig, Path cfgPath, List overriddenProps) { + DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath); try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); @@ -186,7 +177,7 @@ public static DFSPropertiesConfiguration getConfig(List overriddenProps) try { if (!overriddenProps.isEmpty()) { LOG.info("Adding overridden properties to file properties."); - conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); } } catch (IOException ioe) { throw new HoodieIOException("Unexpected error adding config overrides", ioe); @@ -196,7 +187,7 @@ public static DFSPropertiesConfiguration getConfig(List overriddenProps) } public static TypedProperties buildProperties(List props) { - TypedProperties properties = new TypedProperties(); + TypedProperties properties = DFSPropertiesConfiguration.getGlobalProps(); props.forEach(x -> { String[] kv = x.split("="); ValidationUtils.checkArgument(kv.length == 2); @@ -476,6 +467,15 @@ public Schema getTargetSchema() { }; } + public static HoodieTableMetaClient createMetaClient( + JavaSparkContext jsc, String basePath, boolean shouldLoadActiveTimelineOnLoad) { + return HoodieTableMetaClient.builder() + .setConf(jsc.hadoopConfiguration()) + .setBasePath(basePath) + .setLoadActiveTimelineOnLoad(shouldLoadActiveTimelineOnLoad) + .build(); + } + @FunctionalInterface public interface CheckedSupplier { T get() throws Throwable; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallback.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallback.java new file mode 100644 index 000000000000..2009c2460ce1 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallback.java @@ -0,0 +1,180 @@ +/* + * 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 org.apache.hudi.utilities.callback.pulsar; + +import org.apache.hudi.callback.HoodieWriteCommitCallback; +import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; +import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; + +import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import static org.apache.hudi.common.util.DateTimeUtils.parseDuration; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.BROKER_SERVICE_URL; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.CONNECTION_TIMEOUT; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.KEEPALIVE_INTERVAL; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.OPERATION_TIMEOUT; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.PRODUCER_BLOCK_QUEUE_FULL; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.PRODUCER_PENDING_QUEUE_SIZE; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.PRODUCER_PENDING_SIZE; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.PRODUCER_ROUTE_MODE; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.PRODUCER_SEND_TIMEOUT; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.REQUEST_TIMEOUT; +import static org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig.TOPIC; + +/** + * Pulsar implementation of {@link HoodieWriteCommitCallback}. + */ +public class HoodieWriteCommitPulsarCallback implements HoodieWriteCommitCallback, Closeable { + + private static final Logger LOG = LogManager.getLogger(HoodieWriteCommitPulsarCallback.class); + + private final String serviceUrl; + private final String topic; + + /** + * The pulsar client. + */ + private final transient PulsarClient client; + + /** + * The pulsar producer. + */ + private final transient Producer producer; + + public HoodieWriteCommitPulsarCallback(HoodieWriteConfig config) throws PulsarClientException { + this.serviceUrl = config.getString(BROKER_SERVICE_URL); + this.topic = config.getString(TOPIC); + this.client = createClient(config); + this.producer = createProducer(config); + } + + @Override + public void call(HoodieWriteCommitCallbackMessage callbackMessage) { + String callbackMsg = HoodieWriteCommitCallbackUtil.convertToJsonString(callbackMessage); + try { + producer.newMessage().key(callbackMessage.getTableName()).value(callbackMsg).send(); + LOG.info("Send callback message succeed"); + } catch (Exception e) { + LOG.error("Send pulsar callback msg failed : ", e); + } + } + + /** + * Method helps to create {@link Producer}. + * + * @param hoodieConfig Pulsar configs + * @return A {@link Producer} + */ + public Producer createProducer(HoodieConfig hoodieConfig) throws PulsarClientException { + MessageRoutingMode routeMode = Enum.valueOf(MessageRoutingMode.class, + PRODUCER_ROUTE_MODE.defaultValue()); + Duration sendTimeout = + parseDuration(hoodieConfig.getString(PRODUCER_SEND_TIMEOUT)); + int pendingQueueSize = + hoodieConfig.getInt(PRODUCER_PENDING_QUEUE_SIZE); + int pendingSize = + hoodieConfig.getInt(PRODUCER_PENDING_SIZE); + boolean blockIfQueueFull = + hoodieConfig.getBoolean(PRODUCER_BLOCK_QUEUE_FULL); + + return client + .newProducer(Schema.STRING) + .topic(topic) + .messageRoutingMode(routeMode) + .sendTimeout((int) sendTimeout.toMillis(), TimeUnit.MILLISECONDS) + .maxPendingMessages(pendingQueueSize) + .maxPendingMessagesAcrossPartitions(pendingSize) + .blockIfQueueFull(blockIfQueueFull) + .create(); + } + + public PulsarClient createClient(HoodieConfig hoodieConfig) throws PulsarClientException { + validatePulsarConfig(); + + Duration operationTimeout = + parseDuration(hoodieConfig.getString(OPERATION_TIMEOUT)); + Duration connectionTimeout = + parseDuration(hoodieConfig.getString(CONNECTION_TIMEOUT)); + Duration requestTimeout = + parseDuration(hoodieConfig.getString(REQUEST_TIMEOUT)); + Duration keepAliveInterval = + parseDuration(hoodieConfig.getString(KEEPALIVE_INTERVAL)); + + ClientConfigurationData clientConfigurationData = + new ClientConfigurationData(); + clientConfigurationData + .setServiceUrl(serviceUrl); + clientConfigurationData + .setOperationTimeoutMs(operationTimeout.toMillis()); + clientConfigurationData + .setConnectionTimeoutMs((int) connectionTimeout.toMillis()); + clientConfigurationData + .setRequestTimeoutMs((int) requestTimeout.toMillis()); + clientConfigurationData + .setKeepAliveIntervalSeconds((int) keepAliveInterval.getSeconds()); + + return new PulsarClientImpl(clientConfigurationData); + } + + /** + * Validate whether both pulsar's brokerServiceUrl and topic are configured. + * Exception will be thrown if anyone of them is not configured. + */ + private void validatePulsarConfig() { + ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(serviceUrl), String.format("Config %s can not be " + + "null or empty", BROKER_SERVICE_URL.key())); + ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(topic), String.format("Config %s can not be null or empty", + TOPIC.key())); + } + + @Override + public void close() throws IOException { + if (producer != null) { + try { + producer.close(); + } catch (Throwable t) { + LOG.warn("Could not properly close the producer.", t); + } + } + + if (client != null) { + try { + client.close(); + } catch (Throwable t) { + LOG.warn("Could not properly close the client.", t); + } + } + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallbackConfig.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallbackConfig.java new file mode 100644 index 000000000000..f185e6b51763 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/pulsar/HoodieWriteCommitPulsarCallbackConfig.java @@ -0,0 +1,118 @@ +/* + * 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 org.apache.hudi.utilities.callback.pulsar; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import static org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_PREFIX; + +/** + * pulsar write callback related config. + */ +@ConfigClassProperty(name = "Write commit pulsar callback configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Controls notifications sent to pulsar, on events happening to a hudi table.") +public class HoodieWriteCommitPulsarCallbackConfig extends HoodieConfig { + + public static final ConfigProperty BROKER_SERVICE_URL = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.broker.service.url") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("Server's url of pulsar cluster, to be used for publishing commit metadata."); + + public static final ConfigProperty TOPIC = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.topic") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("pulsar topic name to publish timeline activity into."); + + public static final ConfigProperty PRODUCER_ROUTE_MODE = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.producer.route-mode") + .defaultValue("RoundRobinPartition") + .sinceVersion("0.11.0") + .withDocumentation("Message routing logic for producers on partitioned topics."); + + public static final ConfigProperty PRODUCER_PENDING_QUEUE_SIZE = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.producer.pending-queue-size") + .defaultValue(1000) + .sinceVersion("0.11.0") + .withDocumentation("The maximum size of a queue holding pending messages."); + + public static final ConfigProperty PRODUCER_PENDING_SIZE = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.producer.pending-total-size") + .defaultValue(50000) + .sinceVersion("0.11.0") + .withDocumentation("The maximum number of pending messages across partitions."); + + public static final ConfigProperty PRODUCER_BLOCK_QUEUE_FULL = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.producer.block-if-queue-full") + .defaultValue(true) + .sinceVersion("0.11.0") + .withDocumentation("When the queue is full, the method is blocked " + + "instead of an exception is thrown."); + + public static final ConfigProperty PRODUCER_SEND_TIMEOUT = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.producer.send-timeout") + .defaultValue("30s") + .sinceVersion("0.11.0") + .withDocumentation("The timeout in each sending to pulsar."); + + public static final ConfigProperty OPERATION_TIMEOUT = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.operation-timeout") + .defaultValue("30s") + .sinceVersion("0.11.0") + .withDocumentation("Duration of waiting for completing an operation."); + + public static final ConfigProperty CONNECTION_TIMEOUT = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.connection-timeout") + .defaultValue("10s") + .sinceVersion("0.11.0") + .withDocumentation("Duration of waiting for a connection to a " + + "broker to be established."); + + public static final ConfigProperty REQUEST_TIMEOUT = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.request-timeout") + .defaultValue("60s") + .sinceVersion("0.11.0") + .withDocumentation("Duration of waiting for completing a request."); + + public static final ConfigProperty KEEPALIVE_INTERVAL = ConfigProperty + .key(CALLBACK_PREFIX + "pulsar.keepalive-interval") + .defaultValue("30s") + .sinceVersion("0.11.0") + .withDocumentation("Duration of keeping alive interval for each " + + "client broker connection."); + + /** + * Set default value for {@link HoodieWriteCommitPulsarCallbackConfig} if needed. + */ + public static void setCallbackPulsarConfigIfNeeded(HoodieConfig config) { + config.setDefaultValue(PRODUCER_ROUTE_MODE); + config.setDefaultValue(OPERATION_TIMEOUT); + config.setDefaultValue(CONNECTION_TIMEOUT); + config.setDefaultValue(REQUEST_TIMEOUT); + config.setDefaultValue(KEEPALIVE_INTERVAL); + config.setDefaultValue(PRODUCER_SEND_TIMEOUT); + config.setDefaultValue(PRODUCER_PENDING_QUEUE_SIZE); + config.setDefaultValue(PRODUCER_PENDING_SIZE); + config.setDefaultValue(PRODUCER_BLOCK_QUEUE_FULL); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index b5239e929359..96a7c25de881 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -50,6 +50,7 @@ import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.keygen.KeyGenerator; @@ -59,6 +60,8 @@ import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig; +import org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallback; +import org.apache.hudi.utilities.callback.pulsar.HoodieWriteCommitPulsarCallbackConfig; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config; import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; import org.apache.hudi.utilities.schema.DelegatingSchemaProvider; @@ -106,7 +109,6 @@ import static org.apache.hudi.config.HoodieWriteConfig.AUTO_COMMIT_ENABLE; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT; -import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE; import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY; import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE; @@ -172,6 +174,8 @@ public class DeltaSync implements Serializable { /** * Bag of properties with source, hoodie client, key generator etc. + * + * NOTE: These properties are already consolidated w/ CLI provided config-overrides */ private final TypedProperties props; @@ -334,16 +338,19 @@ public Pair>> readFromSource( } else if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_KEY))) { //if previous checkpoint is an empty string, skip resume use Option.empty() resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); - } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) { - // incase of CLUSTER commit, no checkpoint will be available in metadata. - resumeCheckpointStr = Option.empty(); } else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, HoodieTimeline.LESSER_THAN, lastCommit.get().getTimestamp())) { - throw new HoodieDeltaStreamerException( - "Unable to find previous checkpoint. Please double check if this table " - + "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :" - + commitTimelineOpt.get().getInstants().collect(Collectors.toList()) + ", CommitMetadata=" - + commitMetadata.toJsonString()); + // if previous commit metadata did not have the checkpoint key, try traversing previous commits until we find one. + Option prevCheckpoint = getPreviousCheckpoint(commitTimelineOpt.get()); + if (prevCheckpoint.isPresent()) { + resumeCheckpointStr = prevCheckpoint; + } else { + throw new HoodieDeltaStreamerException( + "Unable to find previous checkpoint. Please double check if this table " + + "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :" + + commitTimelineOpt.get().getInstants().collect(Collectors.toList()) + ", CommitMetadata=" + + commitMetadata.toJsonString()); + } } // KAFKA_CHECKPOINT_TYPE will be honored only for first batch. if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_RESET_KEY))) { @@ -434,6 +441,7 @@ public Pair>> readFromSource( return null; } + jssc.setJobGroup(this.getClass().getSimpleName(), "Checking if input is empty"); if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) { LOG.info("No new data, perform empty commit."); return Pair.of(schemaProvider, Pair.of(checkpointStr, jssc.emptyRDD())); @@ -451,6 +459,18 @@ public Pair>> readFromSource( return Pair.of(schemaProvider, Pair.of(checkpointStr, records)); } + protected Option getPreviousCheckpoint(HoodieTimeline timeline) throws IOException { + return timeline.getReverseOrderedInstants().map(instant -> { + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + return Option.ofNullable(commitMetadata.getMetadata(CHECKPOINT_KEY)); + } catch (IOException e) { + throw new HoodieIOException("Failed to parse HoodieCommitMetadata for " + instant.toString(), e); + } + }).filter(Option::isPresent).findFirst().orElse(Option.empty()); + } + /** * Perform Hoodie Write. Run Cleaner, schedule compaction and syncs to hive if needed. * @@ -680,48 +700,62 @@ private HoodieWriteConfig getHoodieClientConfig(SchemaProvider schemaProvider) { private HoodieWriteConfig getHoodieClientConfig(Schema schema) { final boolean combineBeforeUpsert = true; final boolean autoCommit = false; + + // NOTE: Provided that we're injecting combined properties + // (from {@code props}, including CLI overrides), there's no + // need to explicitly set up some configuration aspects that + // are based on these (for ex Clustering configuration) HoodieWriteConfig.Builder builder = - HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, combineBeforeUpsert) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName) - // Inline compaction is disabled for continuous mode. otherwise enabled for MOR - .withInlineCompaction(cfg.isInlineCompactionEnabled()).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder() - .withInlineClustering(cfg.isInlineClusteringEnabled()) - .withAsyncClustering(cfg.isAsyncClusteringEnabled()).build()) - .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) - .build()) + HoodieWriteConfig.newBuilder() + .withPath(cfg.targetBasePath) + .combineInput(cfg.filterDupes, combineBeforeUpsert) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder() + .withPayloadClass(cfg.payloadClassName) + .withInlineCompaction(cfg.isInlineCompactionEnabled()) + .build() + ) + .withPayloadConfig( + HoodiePayloadConfig.newBuilder() + .withPayloadOrderingField(cfg.sourceOrderingField) + .build()) .forTable(cfg.targetTableName) - .withAutoCommit(autoCommit).withProps(props); + .withAutoCommit(autoCommit) + .withProps(props); - if (null != schema) { - builder = builder.withSchema(schema.toString()); + if (schema != null) { + builder.withSchema(schema.toString()); } + HoodieWriteConfig config = builder.build(); - // set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if needed. - if (config.writeCommitCallbackOn() && HoodieWriteCommitKafkaCallback.class.getName().equals(config.getCallbackClass())) { - HoodieWriteCommitKafkaCallbackConfig.setCallbackKafkaConfigIfNeeded(config); + if (config.writeCommitCallbackOn()) { + // set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if needed. + if (HoodieWriteCommitKafkaCallback.class.getName().equals(config.getCallbackClass())) { + HoodieWriteCommitKafkaCallbackConfig.setCallbackKafkaConfigIfNeeded(config); + } + + // set default value for {@link HoodieWriteCommitPulsarCallbackConfig} if needed. + if (HoodieWriteCommitPulsarCallback.class.getName().equals(config.getCallbackClass())) { + HoodieWriteCommitPulsarCallbackConfig.setCallbackPulsarConfigIfNeeded(config); + } } + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.from(props); + // Validate what deltastreamer assumes of write-config to be really safe ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(), String.format("%s should be set to %s", INLINE_COMPACT.key(), cfg.isInlineCompactionEnabled())); - ValidationUtils.checkArgument(config.inlineClusteringEnabled() == cfg.isInlineClusteringEnabled(), - String.format("%s should be set to %s", INLINE_CLUSTERING.key(), cfg.isInlineClusteringEnabled())); - ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == cfg.isAsyncClusteringEnabled(), - String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(), cfg.isAsyncClusteringEnabled())); + ValidationUtils.checkArgument(config.inlineClusteringEnabled() == clusteringConfig.isInlineClusteringEnabled(), + String.format("%s should be set to %s", INLINE_CLUSTERING.key(), clusteringConfig.isInlineClusteringEnabled())); + ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == clusteringConfig.isAsyncClusteringEnabled(), + String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(), clusteringConfig.isAsyncClusteringEnabled())); ValidationUtils.checkArgument(!config.shouldAutoCommit(), String.format("%s should be set to %s", AUTO_COMMIT_ENABLE.key(), autoCommit)); ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes, String.format("%s should be set to %s", COMBINE_BEFORE_INSERT.key(), cfg.filterDupes)); ValidationUtils.checkArgument(config.shouldCombineBeforeUpsert(), String.format("%s should be set to %s", COMBINE_BEFORE_UPSERT.key(), combineBeforeUpsert)); - ValidationUtils.checkArgument(!config.mergeDeltastreamerStateFromPreviousCommit(), - String.format( - "Deltastreamer processes should not merge state from previous deltastreamer commits. Please unset '%s'", - WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE.key()) - ); - return config; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 1649759f7252..2522a605c3e7 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -29,6 +29,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.OperationConverter; import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -45,8 +46,10 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringUpdateException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.utilities.HiveIncrementalPuller; import org.apache.hudi.utilities.IdentitySplitter; @@ -96,6 +99,9 @@ public class HoodieDeltaStreamer implements Serializable { protected final transient Config cfg; + /** + * NOTE: These properties are already consolidated w/ CLI provided config-overrides. + */ private final TypedProperties properties; protected transient Option deltaSyncService; @@ -119,17 +125,8 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Con } public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf, - Option props) throws IOException { - // Resolving the properties first in a consistent way - if (props.isPresent()) { - this.properties = setDefaults(props.get()); - } else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) { - this.properties = setDefaults(UtilHelpers.getConfig(cfg.configs).getConfig()); - } else { - this.properties = setDefaults(UtilHelpers.readConfig( - FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()), - new Path(cfg.propsFilePath), cfg.configs).getConfig()); - } + Option propsOverride) throws IOException { + this.properties = combineProperties(cfg, propsOverride, jssc.hadoopConfiguration()); if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) { InitialCheckPointProvider checkPointProvider = @@ -137,6 +134,7 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Con checkPointProvider.init(conf); cfg.checkpoint = checkPointProvider.getCheckpoint(); } + this.cfg = cfg; this.bootstrapExecutor = Option.ofNullable( cfg.runBootstrap ? new BootstrapExecutor(cfg, jssc, fs, conf, this.properties) : null); @@ -144,15 +142,27 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Con cfg.runBootstrap ? null : new DeltaSyncService(cfg, jssc, fs, conf, Option.ofNullable(this.properties))); } - public void shutdownGracefully() { - deltaSyncService.ifPresent(ds -> ds.shutdown(false)); + private static TypedProperties combineProperties(Config cfg, Option propsOverride, Configuration hadoopConf) { + HoodieConfig hoodieConfig = new HoodieConfig(); + // Resolving the properties in a consistent way: + // 1. Properties override always takes precedence + // 2. Otherwise, check if there's no props file specified (merging in CLI overrides) + // 3. Otherwise, parse provided specified props file (merging in CLI overrides) + if (propsOverride.isPresent()) { + hoodieConfig.setAll(propsOverride.get()); + } else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) { + hoodieConfig.setAll(UtilHelpers.getConfig(cfg.configs).getProps()); + } else { + hoodieConfig.setAll(UtilHelpers.readConfig(hadoopConf, new Path(cfg.propsFilePath), cfg.configs).getProps()); + } + + hoodieConfig.setDefaultValue(DataSourceWriteOptions.RECONCILE_SCHEMA()); + + return hoodieConfig.getProps(true); } - private TypedProperties setDefaults(TypedProperties props) { - if (!props.containsKey(DataSourceWriteOptions.RECONCILE_SCHEMA().key())) { - props.setProperty(DataSourceWriteOptions.RECONCILE_SCHEMA().key(), DataSourceWriteOptions.RECONCILE_SCHEMA().defaultValue().toString()); - } - return props; + public void shutdownGracefully() { + deltaSyncService.ifPresent(ds -> ds.shutdown(false)); } /** @@ -365,20 +375,11 @@ public boolean isAsyncCompactionEnabled() { } public boolean isInlineCompactionEnabled() { + // Inline compaction is disabled for continuous mode, otherwise enabled for MOR return !continuousMode && !forceDisableCompaction && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType)); } - public boolean isAsyncClusteringEnabled() { - return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig() - .getOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE.key(), false))); - } - - public boolean isInlineClusteringEnabled() { - return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig() - .getOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING.key(), false))); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -627,6 +628,8 @@ protected Pair startService() { LOG.info("Setting Spark Pool name for delta-sync to " + DELTASYNC_POOL_NAME); jssc.setLocalProperty("spark.scheduler.pool", DELTASYNC_POOL_NAME); } + + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.from(props); try { while (!isShutdownRequested()) { try { @@ -638,7 +641,7 @@ protected Pair startService() { HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.get().getLeft().get())); asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions); } - if (cfg.isAsyncClusteringEnabled()) { + if (clusteringConfig.isAsyncClusteringEnabled()) { Option clusteringInstant = deltaSync.getClusteringInstantOpt(); if (clusteringInstant.isPresent()) { LOG.info("Scheduled async clustering for instant: " + clusteringInstant.get()); @@ -652,6 +655,8 @@ protected Pair startService() { + toSleepMs + " ms."); Thread.sleep(toSleepMs); } + } catch (HoodieUpsertException ue) { + handleUpsertException(ue); } catch (Exception e) { LOG.error("Shutting down delta-sync due to exception", e); error = true; @@ -665,6 +670,21 @@ protected Pair startService() { }, executor), executor); } + private void handleUpsertException(HoodieUpsertException ue) { + if (ue.getCause() instanceof HoodieClusteringUpdateException) { + LOG.warn("Write rejected due to conflicts with pending clustering operation. Going to retry after 1 min with the hope " + + "that clustering will complete by then.", ue); + try { + Thread.sleep(60000); // Intentionally not using cfg.minSyncIntervalSeconds, since it could be too high or it could be 0. + // Once the delta streamer gets past this clustering update exception, regular syncs will honor cfg.minSyncIntervalSeconds. + } catch (InterruptedException e) { + throw new HoodieException("Deltastreamer interrupted while waiting for next round ", e); + } + } else { + throw ue; + } + } + /** * Shutdown async services like compaction/clustering as DeltaSync is shutdown. */ @@ -711,7 +731,7 @@ protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) { } } // start async clustering if required - if (cfg.isAsyncClusteringEnabled()) { + if (HoodieClusteringConfig.from(props).isAsyncClusteringEnabled()) { if (asyncClusteringService.isPresent()) { asyncClusteringService.get().updateWriteClient(writeClient); } else { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java index 0acae5871909..d361179a1db1 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java @@ -74,7 +74,7 @@ private Timer createTimer(String name) { } String getMetricsName(String action, String metric) { - return config == null ? null : String.format("%s.%s.%s", tableName, action, metric); + return config == null ? null : String.format("%s.%s.%s", config.getMetricReporterMetricsNamePrefix(), action, metric); } public void updateDeltaStreamerMetrics(long durationInNs) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index 7e49d9b88f69..dc150803e8b3 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -77,7 +77,7 @@ public HoodieMultiTableDeltaStreamer(Config config, JavaSparkContext jssc) throw FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration()); configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder; checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs); - TypedProperties commonProperties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig(); + TypedProperties commonProperties = UtilHelpers.readConfig(fs.getConf(), new Path(commonPropsFile), new ArrayList()).getProps(); //get the tables to be ingested and their corresponding config files from this properties instance populateTableExecutionContextList(commonProperties, configFolder, fs, config); } @@ -116,7 +116,7 @@ private void populateTableExecutionContextList(TypedProperties properties, Strin String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX; String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable)); checkIfTableConfigFileExists(configFolder, fs, configFilePath); - TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig(); + TypedProperties tableProperties = UtilHelpers.readConfig(fs.getConf(), new Path(configFilePath), new ArrayList()).getProps(); properties.forEach((k, v) -> { if (tableProperties.get(k) == null) { tableProperties.setProperty(k.toString(), v.toString()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/HiveSchemaProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/HiveSchemaProvider.java new file mode 100644 index 000000000000..219b1ae57886 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/HiveSchemaProvider.java @@ -0,0 +1,99 @@ +/* + * 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 org.apache.hudi.utilities.schema; + +import org.apache.avro.Schema; +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.types.StructType; + +import java.util.Collections; + +public class HiveSchemaProvider extends SchemaProvider { + + /** + * Configs supported. + */ + public static class Config { + private static final String SOURCE_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.database"; + private static final String SOURCE_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.table"; + private static final String TARGET_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.database"; + private static final String TARGET_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.table"; + } + + private static final Logger LOG = LogManager.getLogger(HiveSchemaProvider.class); + + private final Schema sourceSchema; + + private Schema targetSchema; + + public HiveSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_TABLE_PROP)); + String sourceSchemaDBName = props.getString(Config.SOURCE_SCHEMA_DATABASE_PROP, "default"); + String sourceSchemaTableName = props.getString(Config.SOURCE_SCHEMA_TABLE_PROP); + SparkSession spark = SparkSession.builder().config(jssc.getConf()).enableHiveSupport().getOrCreate(); + try { + TableIdentifier sourceSchemaTable = new TableIdentifier(sourceSchemaTableName, scala.Option.apply(sourceSchemaDBName)); + StructType sourceSchema = spark.sessionState().catalog().getTableMetadata(sourceSchemaTable).schema(); + + this.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema( + sourceSchema, + sourceSchemaTableName, + "hoodie." + sourceSchemaDBName); + + if (props.containsKey(Config.TARGET_SCHEMA_TABLE_PROP)) { + String targetSchemaDBName = props.getString(Config.TARGET_SCHEMA_DATABASE_PROP, "default"); + String targetSchemaTableName = props.getString(Config.TARGET_SCHEMA_TABLE_PROP); + TableIdentifier targetSchemaTable = new TableIdentifier(targetSchemaTableName, scala.Option.apply(targetSchemaDBName)); + StructType targetSchema = spark.sessionState().catalog().getTableMetadata(targetSchemaTable).schema(); + this.targetSchema = AvroConversionUtils.convertStructTypeToAvroSchema( + targetSchema, + targetSchemaTableName, + "hoodie." + targetSchemaDBName); + } + } catch (NoSuchTableException | NoSuchDatabaseException e) { + String message = String.format("Can't find Hive table(s): %s", sourceSchemaTableName + "," + props.getString(Config.TARGET_SCHEMA_TABLE_PROP)); + throw new IllegalArgumentException(message, e); + } + } + + @Override + public Schema getSourceSchema() { + return sourceSchema; + } + + @Override + public Schema getTargetSchema() { + if (targetSchema != null) { + return targetSchema; + } else { + return super.getTargetSchema(); + } + } +} \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java index 32ef60967ae8..216369296ad5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java @@ -49,7 +49,7 @@ public class SchemaRegistryProvider extends SchemaProvider { */ public static class Config { - private static final String SRC_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url"; + public static final String SRC_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url"; private static final String TARGET_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.targetUrl"; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java index aed6c6b7358b..4e80009747e5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java @@ -45,6 +45,7 @@ public class AvroDFSSource extends AvroSource { public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider) throws IOException { super(props, sparkContext, sparkSession, schemaProvider); + sparkContext.hadoopConfiguration().set("avro.schema.input.key", schemaProvider.getSourceSchema().toString()); this.pathSelector = DFSPathSelector .createSourceSelector(props, sparkContext.hadoopConfiguration()); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java index a217e6b7a800..ebb359390be0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java @@ -72,12 +72,19 @@ static class Config { /** * {@value #READ_LATEST_INSTANT_ON_MISSING_CKPT} allows delta-streamer to incrementally fetch from latest committed - * instant when checkpoint is not provided. + * instant when checkpoint is not provided. This config is deprecated. Please refer to {@link #MISSING_CHECKPOINT_STRATEGY}. */ + @Deprecated static final String READ_LATEST_INSTANT_ON_MISSING_CKPT = "hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt"; static final Boolean DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT = false; + /** + * {@value #MISSING_CHECKPOINT_STRATEGY} allows delta-streamer to decide the checkpoint to consume from when checkpoint is not set. + * instant when checkpoint is not provided. + */ + static final String MISSING_CHECKPOINT_STRATEGY = "hoodie.deltastreamer.source.hoodieincr.missing.checkpoint.strategy"; + /** * {@value #SOURCE_FILE_FORMAT} is passed to the reader while loading dataset. Default value is parquet. */ @@ -106,13 +113,18 @@ public Pair>, String> fetchNextBatch(Option lastCkpt int numInstantsPerFetch = props.getInteger(Config.NUM_INSTANTS_PER_FETCH, Config.DEFAULT_NUM_INSTANTS_PER_FETCH); boolean readLatestOnMissingCkpt = props.getBoolean(Config.READ_LATEST_INSTANT_ON_MISSING_CKPT, Config.DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT); + IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy = (props.containsKey(Config.MISSING_CHECKPOINT_STRATEGY)) + ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(props.getString(Config.MISSING_CHECKPOINT_STRATEGY)) : null; + if (readLatestOnMissingCkpt) { + missingCheckpointStrategy = IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST; + } // Use begin Instant if set and non-empty Option beginInstant = lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() : lastCkptStr : Option.empty(); Pair instantEndpts = IncrSourceHelper.calculateBeginAndEndInstants(sparkContext, srcPath, - numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt); + numInstantsPerFetch, beginInstant, missingCheckpointStrategy); if (instantEndpts.getKey().equals(instantEndpts.getValue())) { LOG.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java index f752e0d5a15e..b2f6f784ca98 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java @@ -39,9 +39,7 @@ public InputBatch(Option batch, String checkpointForNextBatch, SchemaProvider } public InputBatch(Option batch, String checkpointForNextBatch) { - this.batch = batch; - this.checkpointForNextBatch = checkpointForNextBatch; - this.schemaProvider = null; + this(batch, checkpointForNextBatch, null); } public Option getBatch() { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java index 79e4abbcf900..769bd3280003 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; @@ -49,7 +50,6 @@ import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.NUM_INSTANTS_PER_FETCH; import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.READ_LATEST_INSTANT_ON_MISSING_CKPT; import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.SOURCE_FILE_FORMAT; -import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_PREFIX; /** * This source will use the S3 events meta information from hoodie table generate by {@link S3EventsSource}. @@ -62,6 +62,10 @@ static class Config { // control whether we do existence check for files before consuming them static final String ENABLE_EXISTS_CHECK = "hoodie.deltastreamer.source.s3incr.check.file.exists"; static final Boolean DEFAULT_ENABLE_EXISTS_CHECK = false; + + // control whether to filter the s3 objects starting with this prefix + static final String S3_KEY_PREFIX = "hoodie.deltastreamer.source.s3incr.key.prefix"; + static final String S3_FS_PREFIX = "hoodie.deltastreamer.source.s3incr.fs.prefix"; } public S3EventsHoodieIncrSource( @@ -79,6 +83,11 @@ public Pair>, String> fetchNextBatch(Option lastCkpt int numInstantsPerFetch = props.getInteger(NUM_INSTANTS_PER_FETCH, DEFAULT_NUM_INSTANTS_PER_FETCH); boolean readLatestOnMissingCkpt = props.getBoolean( READ_LATEST_INSTANT_ON_MISSING_CKPT, DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT); + IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy = (props.containsKey(HoodieIncrSource.Config.MISSING_CHECKPOINT_STRATEGY)) + ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(props.getString(HoodieIncrSource.Config.MISSING_CHECKPOINT_STRATEGY)) : null; + if (readLatestOnMissingCkpt) { + missingCheckpointStrategy = IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST; + } // Use begin Instant if set and non-empty Option beginInstant = @@ -88,7 +97,7 @@ public Pair>, String> fetchNextBatch(Option lastCkpt Pair instantEndpts = IncrSourceHelper.calculateBeginAndEndInstants( - sparkContext, srcPath, numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt); + sparkContext, srcPath, numInstantsPerFetch, beginInstant, missingCheckpointStrategy); if (instantEndpts.getKey().equals(instantEndpts.getValue())) { LOG.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey()); @@ -101,9 +110,22 @@ public Pair>, String> fetchNextBatch(Option lastCkpt .option(DataSourceReadOptions.BEGIN_INSTANTTIME().key(), instantEndpts.getLeft()) .option(DataSourceReadOptions.END_INSTANTTIME().key(), instantEndpts.getRight()); Dataset source = metaReader.load(srcPath); + + if (source.isEmpty()) { + return Pair.of(Option.empty(), instantEndpts.getRight()); + } + + String filter = "s3.object.size > 0"; + if (!StringUtils.isNullOrEmpty(props.getString(Config.S3_KEY_PREFIX))) { + filter = filter + " and s3.object.key like '" + props.getString(Config.S3_KEY_PREFIX) + "%'"; + } + + String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase(); + String s3Prefix = s3FS + "://"; + // Extract distinct file keys from s3 meta hoodie table final List cloudMetaDf = source - .filter("s3.object.size > 0") + .filter(filter) .select("s3.bucket.name", "s3.object.key") .distinct() .collectAsList(); @@ -113,9 +135,9 @@ public Pair>, String> fetchNextBatch(Option lastCkpt for (Row row : cloudMetaDf) { // construct file path, row index 0 refers to bucket and 1 refers to key String bucket = row.getString(0); - String filePath = S3_PREFIX + bucket + "/" + row.getString(1); + String filePath = s3Prefix + bucket + "/" + row.getString(1); if (checkExists) { - FileSystem fs = FSUtils.getFs(S3_PREFIX + bucket, sparkSession.sparkContext().hadoopConfiguration()); + FileSystem fs = FSUtils.getFs(s3Prefix + bucket, sparkSession.sparkContext().hadoopConfiguration()); try { if (fs.exists(new Path(filePath))) { cloudFiles.add(filePath); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/DebeziumSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/DebeziumSource.java new file mode 100644 index 000000000000..7018419c2d6d --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/DebeziumSource.java @@ -0,0 +1,243 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.avro.generic.GenericRecord; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.schema.SchemaRegistryProvider; +import org.apache.hudi.utilities.sources.RowSource; +import org.apache.hudi.utilities.sources.helpers.AvroConvertor; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.streaming.kafka010.KafkaUtils; +import org.apache.spark.streaming.kafka010.LocationStrategies; +import org.apache.spark.streaming.kafka010.OffsetRange; + +/** + * Base class for Debezium streaming source which expects change events as Kafka Avro records. + * Obtains the schema from the confluent schema-registry. + */ +public abstract class DebeziumSource extends RowSource { + + private static final Logger LOG = LogManager.getLogger(DebeziumSource.class); + // these are native kafka's config. do not change the config names. + private static final String NATIVE_KAFKA_KEY_DESERIALIZER_PROP = "key.deserializer"; + private static final String NATIVE_KAFKA_VALUE_DESERIALIZER_PROP = "value.deserializer"; + private static final String OVERRIDE_CHECKPOINT_STRING = "hoodie.debezium.override.initial.checkpoint.key"; + private static final String CONNECT_NAME_KEY = "connect.name"; + private static final String DATE_CONNECT_NAME = "custom.debezium.DateString"; + + private final KafkaOffsetGen offsetGen; + private final HoodieDeltaStreamerMetrics metrics; + private final SchemaRegistryProvider schemaRegistryProvider; + private final String deserializerClassName; + + public DebeziumSource(TypedProperties props, JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider, + HoodieDeltaStreamerMetrics metrics) { + super(props, sparkContext, sparkSession, schemaProvider); + + props.put(NATIVE_KAFKA_KEY_DESERIALIZER_PROP, StringDeserializer.class); + deserializerClassName = props.getString(DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER_CLASS().key(), + DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER_CLASS().defaultValue()); + + try { + props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName)); + } catch (ClassNotFoundException e) { + String error = "Could not load custom avro kafka deserializer: " + deserializerClassName; + LOG.error(error); + throw new HoodieException(error, e); + } + + // Currently, debezium source requires Confluent/Kafka schema-registry to fetch the latest schema. + if (schemaProvider == null || !(schemaProvider instanceof SchemaRegistryProvider)) { + schemaRegistryProvider = new SchemaRegistryProvider(props, sparkContext); + } else { + schemaRegistryProvider = (SchemaRegistryProvider) schemaProvider; + } + + offsetGen = new KafkaOffsetGen(props); + this.metrics = metrics; + } + + @Override + protected Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { + String overrideCheckpointStr = props.getString(OVERRIDE_CHECKPOINT_STRING, ""); + + OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCkptStr, sourceLimit, metrics); + long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); + LOG.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); + + if (totalNewMsgs == 0) { + // If there are no new messages, use empty dataframe with no schema. This is because the schema from schema registry can only be considered + // up to date if a change event has occurred. + return Pair.of(Option.of(sparkSession.emptyDataFrame()), overrideCheckpointStr.isEmpty() ? CheckpointUtils.offsetsToStr(offsetRanges) : overrideCheckpointStr); + } else { + try { + String schemaStr = schemaRegistryProvider.fetchSchemaFromRegistry(props.getString(SchemaRegistryProvider.Config.SRC_SCHEMA_REGISTRY_URL_PROP)); + Dataset dataset = toDataset(offsetRanges, offsetGen, schemaStr); + LOG.info(String.format("Spark schema of Kafka Payload for topic %s:\n%s", offsetGen.getTopicName(), dataset.schema().treeString())); + LOG.info(String.format("New checkpoint string: %s", CheckpointUtils.offsetsToStr(offsetRanges))); + return Pair.of(Option.of(dataset), overrideCheckpointStr.isEmpty() ? CheckpointUtils.offsetsToStr(offsetRanges) : overrideCheckpointStr); + } catch (IOException exc) { + LOG.error("Fatal error reading and parsing incoming debezium event", exc); + throw new HoodieException("Fatal error reading and parsing incoming debezium event", exc); + } + } + } + + /** + * Debezium Kafka Payload has a nested structure, flatten it specific to the Database type. + * @param rawKafkaData Dataset of the Debezium CDC event from the kafka + * @return A flattened dataset. + */ + protected abstract Dataset processDataset(Dataset rawKafkaData); + + /** + * Converts a Kafka Topic offset into a Spark dataset. + * + * @param offsetRanges Offset ranges + * @param offsetGen KafkaOffsetGen + * @return Spark dataset + */ + private Dataset toDataset(OffsetRange[] offsetRanges, KafkaOffsetGen offsetGen, String schemaStr) { + AvroConvertor convertor = new AvroConvertor(schemaStr); + Dataset kafkaData; + if (deserializerClassName.equals(StringDeserializer.class.getName())) { + kafkaData = AvroConversionUtils.createDataFrame( + KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, LocationStrategies.PreferConsistent()) + .map(obj -> convertor.fromJson(obj.value())) + .rdd(), schemaStr, sparkSession); + } else { + kafkaData = AvroConversionUtils.createDataFrame( + KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, LocationStrategies.PreferConsistent()) + .map(obj -> (GenericRecord) obj.value()) + .rdd(), schemaStr, sparkSession); + } + + // Flatten debezium payload, specific to each DB type (postgres/ mysql/ etc..) + Dataset debeziumDataset = processDataset(kafkaData); + + // Some required transformations to ensure debezium data types are converted to spark supported types. + return convertArrayColumnsToString(convertColumnToNullable(sparkSession, + convertDateColumns(debeziumDataset, new Schema.Parser().parse(schemaStr)))); + } + + /** + * Converts string formatted date columns into Spark date columns. + * + * @param dataset Spark dataset + * @param schema Avro schema from Debezium + * @return Converted dataset + */ + public static Dataset convertDateColumns(Dataset dataset, Schema schema) { + if (schema.getField("before") != null) { + List dateFields = schema.getField("before") + .schema() + .getTypes() + .get(1) + .getFields() + .stream() + .filter(field -> { + if (field.schema().getType() == Type.UNION) { + return field.schema().getTypes().stream().anyMatch( + schemaInUnion -> DATE_CONNECT_NAME.equals(schemaInUnion.getProp(CONNECT_NAME_KEY)) + ); + } else { + return DATE_CONNECT_NAME.equals(field.schema().getProp(CONNECT_NAME_KEY)); + } + }).map(Field::name).collect(Collectors.toList()); + + LOG.info("Date fields: " + dateFields.toString()); + + for (String dateCol : dateFields) { + dataset = dataset.withColumn(dateCol, functions.col(dateCol).cast(DataTypes.DateType)); + } + } + + return dataset; + } + + /** + * Utility function for converting columns to nullable. This is useful when required to make a column nullable to match a nullable column from Debezium change + * events. + * + * @param sparkSession SparkSession object + * @param dataset Dataframe to modify + * @return Modified dataframe + */ + private static Dataset convertColumnToNullable(SparkSession sparkSession, Dataset dataset) { + List columns = Arrays.asList(dataset.columns()); + StructField[] modifiedStructFields = Arrays.stream(dataset.schema().fields()).map(field -> columns + .contains(field.name()) ? new StructField(field.name(), field.dataType(), true, field.metadata()) : field) + .toArray(StructField[]::new); + + return sparkSession.createDataFrame(dataset.rdd(), new StructType(modifiedStructFields)); + } + + /** + * Converts Array types to String types because not all Debezium array columns are supported to be converted + * to Spark array columns. + * + * @param dataset Dataframe to modify + * @return Modified dataframe + */ + private static Dataset convertArrayColumnsToString(Dataset dataset) { + List arrayColumns = Arrays.stream(dataset.schema().fields()) + .filter(field -> field.dataType().typeName().toLowerCase().startsWith("array")) + .map(StructField::name) + .collect(Collectors.toList()); + + for (String colName : arrayColumns) { + dataset = dataset.withColumn(colName, functions.col(colName).cast(DataTypes.StringType)); + } + + return dataset; + } +} + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/MysqlDebeziumSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/MysqlDebeziumSource.java new file mode 100644 index 000000000000..64dd197479e9 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/MysqlDebeziumSource.java @@ -0,0 +1,102 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.debezium.DebeziumConstants; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF2; +import org.apache.spark.sql.types.DataTypes; + +import static org.apache.spark.sql.functions.callUDF; + +/** + * Source for incrementally ingesting debezium generated change logs for Mysql DB. + */ +public class MysqlDebeziumSource extends DebeziumSource { + + private final SQLContext sqlContext; + private final String generateUniqueSeqUdfFn = "mysql_generate_order_key"; + + public MysqlDebeziumSource(TypedProperties props, JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider, + HoodieDeltaStreamerMetrics metrics) { + super(props, sparkContext, sparkSession, schemaProvider, metrics); + this.sqlContext = sparkSession.sqlContext(); + sqlContext.udf().register(generateUniqueSeqUdfFn, (UDF2) MysqlDebeziumSource::generateUniqueSequence, DataTypes.StringType); + } + + /** + * Debezium Kafka Payload has a nested structure (see https://debezium.io/documentation/reference/1.4/connectors/mysql.html). + * This function flattens this nested structure for the Mysql data, and also extracts a subset of Debezium metadata fields. + * + * @param rowDataset Dataset containing Debezium Payloads + * @return New dataset with flattened columns + */ + @Override + protected Dataset processDataset(Dataset rowDataset) { + Dataset flattenedDataset = rowDataset; + if (rowDataset.columns().length > 0) { + // Only flatten for non-empty schemas + Dataset insertedOrUpdatedData = rowDataset + .selectExpr( + String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_FILE_FIELD, DebeziumConstants.FLATTENED_FILE_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_POS_FIELD, DebeziumConstants.FLATTENED_POS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_ROW_FIELD, DebeziumConstants.FLATTENED_ROW_COL_NAME), + String.format("%s.*", DebeziumConstants.INCOMING_AFTER_FIELD) + ) + .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).notEqual(DebeziumConstants.DELETE_OP)); + + Dataset deletedData = rowDataset + .selectExpr( + String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_FILE_FIELD, DebeziumConstants.FLATTENED_FILE_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_POS_FIELD, DebeziumConstants.FLATTENED_POS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_ROW_FIELD, DebeziumConstants.FLATTENED_ROW_COL_NAME), + String.format("%s.*", DebeziumConstants.INCOMING_BEFORE_FIELD) + ) + .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).equalTo(DebeziumConstants.DELETE_OP)); + + flattenedDataset = insertedOrUpdatedData.union(deletedData); + } + + return flattenedDataset.withColumn(DebeziumConstants.ADDED_SEQ_COL_NAME, + callUDF(generateUniqueSeqUdfFn, flattenedDataset.col(DebeziumConstants.FLATTENED_FILE_COL_NAME), + flattenedDataset.col(DebeziumConstants.FLATTENED_POS_COL_NAME))); + } + + private static String generateUniqueSequence(String fileId, Long pos) { + return fileId.substring(fileId.lastIndexOf('.') + 1).concat("." + pos); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/PostgresDebeziumSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/PostgresDebeziumSource.java new file mode 100644 index 000000000000..bf4381792cfc --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/debezium/PostgresDebeziumSource.java @@ -0,0 +1,87 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.debezium.DebeziumConstants; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +/** + * Source for incrementally ingesting debezium generated change logs for PostgresDB. + */ +public class PostgresDebeziumSource extends DebeziumSource { + + public PostgresDebeziumSource(TypedProperties props, JavaSparkContext sparkContext, + SparkSession sparkSession, + SchemaProvider schemaProvider, + HoodieDeltaStreamerMetrics metrics) { + super(props, sparkContext, sparkSession, schemaProvider, metrics); + } + + /** + * Debezium Kafka Payload has a nested structure (see https://debezium.io/documentation/reference/1.4/connectors/postgresql.html#postgresql-create-events). + * This function flattens this nested structure for the Postgres data, and also extracts a subset of Debezium metadata fields. + * + * @param rowDataset Dataset containing Debezium Payloads + * @return New dataset with flattened columns + */ + @Override + protected Dataset processDataset(Dataset rowDataset) { + if (rowDataset.columns().length > 0) { + // Pick selective debezium and postgres meta fields: pick the row values from before field for delete record + // and row values from after field for insert or update records. + Dataset insertedOrUpdatedData = rowDataset + .selectExpr( + String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TXID_FIELD, DebeziumConstants.FLATTENED_TX_ID_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_LSN_FIELD, DebeziumConstants.FLATTENED_LSN_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_XMIN_FIELD, DebeziumConstants.FLATTENED_XMIN_COL_NAME), + String.format("%s.*", DebeziumConstants.INCOMING_AFTER_FIELD) + ) + .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).notEqual(DebeziumConstants.DELETE_OP)); + + Dataset deletedData = rowDataset + .selectExpr( + String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TXID_FIELD, DebeziumConstants.FLATTENED_TX_ID_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_LSN_FIELD, DebeziumConstants.FLATTENED_LSN_COL_NAME), + String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_XMIN_FIELD, DebeziumConstants.FLATTENED_XMIN_COL_NAME), + String.format("%s.*", DebeziumConstants.INCOMING_BEFORE_FIELD) + ) + .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).equalTo(DebeziumConstants.DELETE_OP)); + + return insertedOrUpdatedData.union(deletedData); + } else { + return rowDataset; + } + } +} + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java index 8f434a007a0b..a370c314a168 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java @@ -32,6 +32,17 @@ public class IncrSourceHelper { + private static final String DEFAULT_BEGIN_TIMESTAMP = "000"; + /** + * Kafka reset offset strategies. + */ + public enum MissingCheckpointStrategy { + // read from latest commit in hoodie source table + READ_LATEST, + // read everything upto latest commit + READ_UPTO_LATEST_COMMIT + } + /** * Get a timestamp which is the next value in a descending sequence. * @@ -47,15 +58,15 @@ private static String getStrictlyLowerTimestamp(String timestamp) { /** * Find begin and end instants to be set for the next fetch. * - * @param jssc Java Spark Context - * @param srcBasePath Base path of Hudi source table - * @param numInstantsPerFetch Max Instants per fetch - * @param beginInstant Last Checkpoint String - * @param readLatestOnMissingBeginInstant when begin instant is missing, allow reading from latest committed instant + * @param jssc Java Spark Context + * @param srcBasePath Base path of Hudi source table + * @param numInstantsPerFetch Max Instants per fetch + * @param beginInstant Last Checkpoint String + * @param missingCheckpointStrategy when begin instant is missing, allow reading based on missing checkpoint strategy * @return begin and end instants */ public static Pair calculateBeginAndEndInstants(JavaSparkContext jssc, String srcBasePath, - int numInstantsPerFetch, Option beginInstant, boolean readLatestOnMissingBeginInstant) { + int numInstantsPerFetch, Option beginInstant, MissingCheckpointStrategy missingCheckpointStrategy) { ValidationUtils.checkArgument(numInstantsPerFetch > 0, "Make sure the config hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value"); HoodieTableMetaClient srcMetaClient = HoodieTableMetaClient.builder().setConf(jssc.hadoopConfiguration()).setBasePath(srcBasePath).setLoadActiveTimelineOnLoad(true).build(); @@ -64,27 +75,38 @@ public static Pair calculateBeginAndEndInstants(JavaSparkContext srcMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(); String beginInstantTime = beginInstant.orElseGet(() -> { - if (readLatestOnMissingBeginInstant) { - Option lastInstant = activeCommitTimeline.lastInstant(); - return lastInstant.map(hoodieInstant -> getStrictlyLowerTimestamp(hoodieInstant.getTimestamp())).orElse("000"); + if (missingCheckpointStrategy != null) { + if (missingCheckpointStrategy == MissingCheckpointStrategy.READ_LATEST) { + Option lastInstant = activeCommitTimeline.lastInstant(); + return lastInstant.map(hoodieInstant -> getStrictlyLowerTimestamp(hoodieInstant.getTimestamp())).orElse(DEFAULT_BEGIN_TIMESTAMP); + } else { + return DEFAULT_BEGIN_TIMESTAMP; + } } else { throw new IllegalArgumentException("Missing begin instant for incremental pull. For reading from latest " - + "committed instant set hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt to true"); + + "committed instant set hoodie.deltastreamer.source.hoodieincr.missing.checkpoint.strategy to a valid value"); } }); - Option nthInstant = Option.fromJavaOptional(activeCommitTimeline - .findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y)); - return Pair.of(beginInstantTime, nthInstant.map(HoodieInstant::getTimestamp).orElse(beginInstantTime)); + if (!beginInstantTime.equals(DEFAULT_BEGIN_TIMESTAMP)) { + Option nthInstant = Option.fromJavaOptional(activeCommitTimeline + .findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y)); + return Pair.of(beginInstantTime, nthInstant.map(HoodieInstant::getTimestamp).orElse(beginInstantTime)); + } else { + // if beginInstant is DEFAULT_BEGIN_TIMESTAMP, MissingCheckpointStrategy should be set. + // when MissingCheckpointStrategy is set to read everything until latest. + Option lastInstant = activeCommitTimeline.lastInstant(); + return Pair.of(beginInstantTime, lastInstant.get().getTimestamp()); + } } /** * Validate instant time seen in the incoming row. * - * @param row Input Row - * @param instantTime Hoodie Instant time of the row + * @param row Input Row + * @param instantTime Hoodie Instant time of the row * @param sinceInstant begin instant of the batch - * @param endInstant end instant of the batch + * @param endInstant end instant of the batch */ public static void validateInstantTime(Row row, String instantTime, String sinceInstant, String endInstant) { Objects.requireNonNull(instantTime); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java index 043b0a4e0ad1..06898db92c81 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java @@ -20,7 +20,14 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; @@ -35,6 +42,9 @@ import org.junit.jupiter.api.BeforeEach; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.Map; import java.util.Random; public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase { @@ -277,4 +287,20 @@ protected static void prepareORCDFSFiles(int numRecords, String baseORCPath, Str } } + static void addCommitToTimeline(HoodieTableMetaClient metaCient) throws IOException { + addCommitToTimeline(metaCient, Collections.emptyMap()); + } + + static void addCommitToTimeline(HoodieTableMetaClient metaCient, Map extraMetadata) throws IOException { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.setOperationType(WriteOperationType.UPSERT); + extraMetadata.forEach((k,v) -> commitMetadata.getExtraMetadata().put(k, v)); + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + metaCient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, commitTime)); + metaCient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime)); + metaCient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } + } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java index 28ba17efa9f4..3ac490bf9163 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java @@ -230,7 +230,7 @@ public void testImportWithUpsert() throws IOException, ParseException { public List createInsertRecords(Path srcFolder) throws ParseException, IOException { Path srcFile = new Path(srcFolder.toString(), "file1.parquet"); - long startTime = HoodieActiveTimeline.parseInstantTime("20170203000000").getTime() / 1000; + long startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime() / 1000; List records = new ArrayList(); for (long recordNum = 0; recordNum < 96; recordNum++) { records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "0", "rider-" + recordNum, @@ -247,7 +247,7 @@ public List createInsertRecords(Path srcFolder) throws ParseExcep public List createUpsertRecords(Path srcFolder) throws ParseException, IOException { Path srcFile = new Path(srcFolder.toString(), "file1.parquet"); - long startTime = HoodieActiveTimeline.parseInstantTime("20170203000000").getTime() / 1000; + long startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime() / 1000; List records = new ArrayList(); // 10 for update for (long recordNum = 0; recordNum < 11; recordNum++) { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHiveSchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHiveSchemaProvider.java new file mode 100644 index 000000000000..7c2dd0c327b3 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHiveSchemaProvider.java @@ -0,0 +1,136 @@ +/* + * 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 org.apache.hudi.utilities.functional; + +import org.apache.avro.Schema; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.UtilHelpers; +import org.apache.hudi.utilities.schema.HiveSchemaProvider; +import org.apache.hudi.utilities.testutils.SparkClientFunctionalTestHarnessWithHiveSupport; +import org.apache.hudi.utilities.testutils.UtilitiesTestBase; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Basic tests against {@link HiveSchemaProvider}. + */ +@Tag("functional") +public class TestHiveSchemaProvider extends SparkClientFunctionalTestHarnessWithHiveSupport { + private static final Logger LOG = LogManager.getLogger(TestHiveSchemaProvider.class); + private static final TypedProperties PROPS = new TypedProperties(); + private static final String SOURCE_SCHEMA_TABLE_NAME = "schema_registry.source_schema_tab"; + private static final String TARGET_SCHEMA_TABLE_NAME = "schema_registry.target_schema_tab"; + + @BeforeAll + public static void init() { + Pair dbAndTableName = paresDBAndTableName(SOURCE_SCHEMA_TABLE_NAME); + PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.database", dbAndTableName.getLeft()); + PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.table", dbAndTableName.getRight()); + } + + @Disabled + @Test + public void testSourceSchema() throws Exception { + try { + createSchemaTable(SOURCE_SCHEMA_TABLE_NAME); + Schema sourceSchema = UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getSourceSchema(); + + Schema originalSchema = new Schema.Parser().parse( + UtilitiesTestBase.Helpers.readFile("delta-streamer-config/hive_schema_provider_source.avsc") + ); + for (Schema.Field field : sourceSchema.getFields()) { + Schema.Field originalField = originalSchema.getField(field.name()); + assertTrue(originalField != null); + } + } catch (HoodieException e) { + LOG.error("Failed to get source schema. ", e); + throw e; + } + } + + @Disabled + @Test + public void testTargetSchema() throws Exception { + try { + Pair dbAndTableName = paresDBAndTableName(TARGET_SCHEMA_TABLE_NAME); + PROPS.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.hive.database", dbAndTableName.getLeft()); + PROPS.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.hive.table", dbAndTableName.getRight()); + createSchemaTable(SOURCE_SCHEMA_TABLE_NAME); + createSchemaTable(TARGET_SCHEMA_TABLE_NAME); + Schema targetSchema = UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getTargetSchema(); + Schema originalSchema = new Schema.Parser().parse( + UtilitiesTestBase.Helpers.readFile("delta-streamer-config/hive_schema_provider_target.avsc")); + for (Schema.Field field : targetSchema.getFields()) { + Schema.Field originalField = originalSchema.getField(field.name()); + assertTrue(originalField != null); + } + } catch (HoodieException e) { + LOG.error("Failed to get source/target schema. ", e); + throw e; + } + } + + @Disabled + @Test + public void testNotExistTable() { + String wrongName = "wrong_schema_tab"; + PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.table", wrongName); + Assertions.assertThrows(NoSuchTableException.class, () -> { + try { + UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getSourceSchema(); + } catch (Throwable exception) { + while (exception.getCause() != null) { + exception = exception.getCause(); + } + throw exception; + } + }); + } + + private static Pair paresDBAndTableName(String fullName) { + String[] dbAndTableName = fullName.split("\\."); + if (dbAndTableName.length > 1) { + return new ImmutablePair<>(dbAndTableName[0], dbAndTableName[1]); + } else { + return new ImmutablePair<>("default", dbAndTableName[0]); + } + } + + private void createSchemaTable(String fullName) throws IOException { + SparkSession spark = spark(); + String createTableSQL = UtilitiesTestBase.Helpers.readFile(String.format("delta-streamer-config/%s.sql", fullName)); + Pair dbAndTableName = paresDBAndTableName(fullName); + spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", dbAndTableName.getLeft())); + spark.sql(createTableSQL); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 014a0c140d62..be965512460c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -20,6 +20,7 @@ import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; @@ -37,10 +38,12 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.HiveSyncConfig; @@ -48,8 +51,10 @@ import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.utilities.DummySchemaProvider; import org.apache.hudi.utilities.HoodieClusteringJob; +import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; +import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; import org.apache.hudi.utilities.sources.CsvDFSSource; import org.apache.hudi.utilities.sources.HoodieIncrSource; @@ -68,6 +73,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -102,6 +108,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -139,9 +146,13 @@ protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, i return new HoodieDeltaStreamer(cfg, jsc); } - protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, boolean runSchedule, String scheduleAndExecute) { + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute) { + return initialHoodieClusteringJob(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, null); + } + + protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, Boolean retryLastFailedClusteringJob) { HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, - clusteringInstantTime, runSchedule, scheduleAndExecute); + clusteringInstantTime, runSchedule, scheduleAndExecute, retryLastFailedClusteringJob); return new HoodieClusteringJob(jsc, scheduleClusteringConfig); } @@ -153,11 +164,13 @@ public static void cleanupClass() { } } + @Override @BeforeEach public void setup() throws Exception { super.setup(); } + @Override @AfterEach public void teardown() throws Exception { super.teardown(); @@ -244,6 +257,12 @@ static void assertRecordCount(long expected, String tablePath, SQLContext sqlCon assertEquals(expected, recordCount); } + static void assertDistinctRecordCount(long expected, String tablePath, SQLContext sqlContext) { + sqlContext.clearCache(); + long recordCount = sqlContext.read().format("org.apache.hudi").load(tablePath).select("_hoodie_record_key").distinct().count(); + assertEquals(expected, recordCount); + } + static List countsPerCommit(String tablePath, SQLContext sqlContext) { sqlContext.clearCache(); List rows = sqlContext.read().format("org.apache.hudi").load(tablePath) @@ -294,7 +313,7 @@ static void assertAtleastNCompactionCommitsAfterCommit(int minExpected, String l static void assertAtleastNDeltaCommitsAfterCommit(int minExpected, String lastSuccessfulCommit, String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build(); - HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); + HoodieTimeline timeline = meta.reloadActiveTimeline().getDeltaCommitTimeline().findInstantsAfter(lastSuccessfulCommit).filterCompletedInstants(); LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); int numDeltaCommits = (int) timeline.getInstants().count(); assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); @@ -317,7 +336,7 @@ static void waitTillCondition(Function condition, Future dsFut boolean ret = false; while (!ret && !dsFuture.isDone()) { try { - Thread.sleep(5000); + Thread.sleep(3000); ret = condition.apply(true); } catch (Throwable error) { LOG.warn("Got error :", error); @@ -345,12 +364,12 @@ static void assertAtLeastNReplaceCommits(int minExpected, String tablePath, File assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } - static void assertNoReplaceCommits(int expected, String tablePath, FileSystem fs) { + static void assertNoReplaceCommits(String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline(); LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); int numDeltaCommits = (int) timeline.getInstants().count(); - assertEquals(expected, numDeltaCommits, "Got=" + numDeltaCommits + ", exp =" + expected); + assertEquals(0, numDeltaCommits, "Got=" + numDeltaCommits + ", exp =" + 0); } static void assertAtLeastNReplaceRequests(int minExpected, String tablePath, FileSystem fs) { @@ -365,7 +384,7 @@ static void assertAtLeastNReplaceRequests(int minExpected, String tablePath, Fil @Test public void testProps() { TypedProperties props = - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig(); + new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps(); assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism")); assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field")); assertEquals("org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer$TestGenerator", @@ -485,7 +504,7 @@ public void testKafkaConnectCheckpointProvider() throws IOException { String checkpointProviderClass = "org.apache.hudi.utilities.checkpointing.KafkaConnectHdfsProvider"; HoodieDeltaStreamer.Config cfg = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT); TypedProperties props = - new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig(); + new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps(); props.put("hoodie.deltastreamer.checkpoint.provider.path", bootstrapPath); cfg.initialCheckpointProvider = checkpointProviderClass; // create regular kafka connect hdfs dirs @@ -694,14 +713,18 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { + deltaStreamerTestRunner(ds, cfg, condition, "single_ds_job"); + } + + static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition, String jobId) throws Exception { Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { try { ds.sync(); } catch (Exception ex) { + LOG.warn("DS continuous job failed, hence not proceeding with condition check for " + jobId); throw new RuntimeException(ex.getMessage(), ex); } }); - TestHelpers.waitTillCondition(condition, dsFuture, 360); ds.shutdownGracefully(); dsFuture.get(); @@ -844,35 +867,41 @@ private List getAsyncServicesConfigs(int totalRecords, String autoClean, private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePath, String clusteringInstantTime, - boolean runSchedule) { - return buildHoodieClusteringUtilConfig(basePath, clusteringInstantTime, runSchedule, null); + Boolean runSchedule) { + return buildHoodieClusteringUtilConfig(basePath, clusteringInstantTime, runSchedule, null, null); } private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePath, String clusteringInstantTime, - boolean runSchedule, - String runningMode) { + Boolean runSchedule, + String runningMode, + Boolean retryLastFailedClusteringJob) { HoodieClusteringJob.Config config = new HoodieClusteringJob.Config(); config.basePath = basePath; config.clusteringInstantTime = clusteringInstantTime; config.runSchedule = runSchedule; config.propsFilePath = dfsBasePath + "/clusteringjob.properties"; config.runningMode = runningMode; + if (retryLastFailedClusteringJob != null) { + config.retryLastFailedClusteringJob = retryLastFailedClusteringJob; + } return config; } - @Test - public void testHoodieAsyncClusteringJob() throws Exception { - String tableBasePath = dfsBasePath + "/asyncClustering"; + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testHoodieAsyncClusteringJob(boolean shouldPassInClusteringInstantTime) throws Exception { + String tableBasePath = dfsBasePath + "/asyncClusteringJob"; HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "true"); - HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, null); deltaStreamerTestRunner(ds, (r) -> { TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); Option scheduleClusteringInstantTime = Option.empty(); try { + HoodieClusteringJob scheduleClusteringJob = + initialHoodieClusteringJob(tableBasePath, null, true, null); scheduleClusteringInstantTime = scheduleClusteringJob.doSchedule(); } catch (Exception e) { LOG.warn("Schedule clustering failed", e); @@ -881,7 +910,7 @@ public void testHoodieAsyncClusteringJob() throws Exception { if (scheduleClusteringInstantTime.isPresent()) { LOG.info("Schedule clustering success, now cluster with instant time " + scheduleClusteringInstantTime.get()); HoodieClusteringJob.Config clusterClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, - scheduleClusteringInstantTime.get(), false); + shouldPassInClusteringInstantTime ? scheduleClusteringInstantTime.get() : null, false); HoodieClusteringJob clusterClusteringJob = new HoodieClusteringJob(jsc, clusterClusteringConfig); clusterClusteringJob.cluster(clusterClusteringConfig.retry); LOG.info("Cluster success"); @@ -897,44 +926,123 @@ public void testHoodieAsyncClusteringJob() throws Exception { public void testAsyncClusteringService() throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering"; // Keep it higher than batch-size to test continuous mode - int totalRecords = 3000; + int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); - cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "2")); + cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); deltaStreamerTestRunner(ds, cfg, (r) -> { - TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); - TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs); + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); return true; }); + // There should be 4 commits, one of which should be a replace commit + TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); + TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); } - @ParameterizedTest - @ValueSource(strings = {"true", "false"}) - public void testAsyncClusteringServiceWithCompaction(String preserveCommitMetadata) throws Exception { + /** + * When deltastreamer writes clashes with pending clustering, deltastreamer should keep retrying and eventually succeed(once clustering completes) + * w/o failing mid way. + * + * @throws Exception + */ + @Test + public void testAsyncClusteringServiceWithConflicts() throws Exception { + String tableBasePath = dfsBasePath + "/asyncClusteringWithConflicts"; + // Keep it higher than batch-size to test continuous mode + int totalRecords = 2000; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + cfg.continuousMode = true; + cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); + cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); + HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); + deltaStreamerTestRunner(ds, cfg, (r) -> { + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); + return true; + }); + // There should be 4 commits, one of which should be a replace commit + TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); + TestHelpers.assertDistinctRecordCount(1900, tableBasePath + "/*/*.parquet", sqlContext); + } + + @Test + public void testAsyncClusteringServiceWithCompaction() throws Exception { String tableBasePath = dfsBasePath + "/asyncClusteringCompaction"; // Keep it higher than batch-size to test continuous mode - int totalRecords = 3000; + int totalRecords = 2000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); cfg.continuousMode = true; cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); - cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "2", preserveCommitMetadata)); + cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "3")); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); deltaStreamerTestRunner(ds, cfg, (r) -> { - TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs); - TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs); + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); return true; }); + // There should be 4 commits, one of which should be a replace commit + TestHelpers.assertAtLeastNCommits(4, tableBasePath, dfs); + TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs); + TestHelpers.assertDistinctRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); } @ParameterizedTest - @ValueSource(strings = {"schedule", "execute", "scheduleAndExecute"}) + @ValueSource(booleans = {true, false}) + public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob) throws Exception { + String tableBasePath = dfsBasePath + "/asyncClustering3"; + + // ingest data + int totalRecords = 3000; + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + cfg.continuousMode = false; + cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); + cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "false", "0", "false", "0")); + HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); + ds.sync(); + + // assert ingest successful + TestHelpers.assertAtLeastNCommits(1, tableBasePath, dfs); + + // schedule a clustering job to build a clustering plan + HoodieClusteringJob schedule = initialHoodieClusteringJob(tableBasePath, null, false, "schedule"); + schedule.cluster(0); + + // do another ingestion + HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg, jsc); + ds2.sync(); + + // convert clustering request into inflight, Simulate the last clustering failed scenario + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(dfs.getConf()).setBasePath(tableBasePath).build(); + List hoodieClusteringInstants = meta.getActiveTimeline().filterPendingReplaceTimeline().getInstants().collect(Collectors.toList()); + HoodieInstant clusteringRequest = hoodieClusteringInstants.get(0); + HoodieInstant hoodieInflightInstant = meta.getActiveTimeline().transitionReplaceRequestedToInflight(clusteringRequest, Option.empty()); + + // trigger a scheduleAndExecute clustering job + // when retryFailedClustering true => will rollback and re-execute failed clustering plan with same instant timestamp. + // when retryFailedClustering false => will make and execute a new clustering plan with new instant timestamp. + HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob); + scheduleAndExecute.cluster(0); + + String completeClusteringTimeStamp = meta.getActiveTimeline().reload().getCompletedReplaceTimeline().lastInstant().get().getTimestamp(); + + if (retryLastFailedClusteringJob) { + assertEquals(clusteringRequest.getTimestamp(), completeClusteringTimeStamp); + } else { + assertFalse(clusteringRequest.getTimestamp().equalsIgnoreCase(completeClusteringTimeStamp)); + } + } + + @ParameterizedTest + @ValueSource(strings = {"execute", "schedule", "scheduleAndExecute"}) public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode) throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering2"; HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); @@ -949,7 +1057,9 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod LOG.info("Cluster success"); } else { LOG.warn("Import failed"); - return false; + if (!runningMode.toLowerCase().equals(HoodieClusteringJob.EXECUTE)) { + return false; + } } } catch (Exception e) { LOG.warn("ScheduleAndExecute clustering failed", e); @@ -965,12 +1075,11 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod } case HoodieClusteringJob.SCHEDULE: { TestHelpers.assertAtLeastNReplaceRequests(2, tableBasePath, dfs); - TestHelpers.assertNoReplaceCommits(0, tableBasePath, dfs); + TestHelpers.assertNoReplaceCommits(tableBasePath, dfs); return true; } case HoodieClusteringJob.EXECUTE: { - assertNotNull(exception); - assertEquals(exception.getMessage(), "--instant-time couldn't be null when executing clustering plan."); + TestHelpers.assertNoReplaceCommits(tableBasePath, dfs); return true; } default: @@ -1253,7 +1362,7 @@ private void testORCDFSSource(boolean useSchemaProvider, List transforme // Properties used for testing delta-streamer with orc source orcProps.setProperty("include", "base.properties"); - orcProps.setProperty("hoodie.embed.timeline.server","false"); + orcProps.setProperty("hoodie.embed.timeline.server", "false"); orcProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); orcProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there"); if (useSchemaProvider) { @@ -1267,9 +1376,9 @@ private void testORCDFSSource(boolean useSchemaProvider, List transforme String tableBasePath = dfsBasePath + "/test_orc_source_table" + testNum; HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer( - TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, ORCDFSSource.class.getName(), - transformerClassNames, PROPS_FILENAME_TEST_ORC, false, - useSchemaProvider, 100000, false, null, null, "timestamp", null), jsc); + TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, ORCDFSSource.class.getName(), + transformerClassNames, PROPS_FILENAME_TEST_ORC, false, + useSchemaProvider, 100000, false, null, null, "timestamp", null), jsc); deltaStreamer.sync(); TestHelpers.assertRecordCount(ORC_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext); testNum++; @@ -1632,6 +1741,46 @@ void testDeltaStreamerWithSpecifiedOperation(final String tableBasePath, WriteOp TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2); } + @Test + public void testFetchingCheckpointFromPreviousCommits() throws IOException { + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(dfsBasePath + "/testFetchPreviousCheckpoint", WriteOperationType.BULK_INSERT); + + TypedProperties properties = new TypedProperties(); + properties.setProperty("hoodie.datasource.write.recordkey.field","key"); + properties.setProperty("hoodie.datasource.write.partitionpath.field","pp"); + TestDeltaSync testDeltaSync = new TestDeltaSync(cfg, sparkSession, null, properties, + jsc, dfs, jsc.hadoopConfiguration(), null); + + properties.put(HoodieTableConfig.NAME.key(), "sample_tbl"); + HoodieTableMetaClient metaClient = HoodieTestUtils.init(jsc.hadoopConfiguration(), dfsBasePath, HoodieTableType.COPY_ON_WRITE, properties); + + Map extraMetadata = new HashMap<>(); + extraMetadata.put(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY, "abc"); + addCommitToTimeline(metaClient, extraMetadata); + metaClient.reloadActiveTimeline(); + assertEquals(testDeltaSync.getPreviousCheckpoint(metaClient.getActiveTimeline().getCommitsTimeline()).get(), "abc"); + + addCommitToTimeline(metaClient, Collections.emptyMap()); + metaClient.reloadActiveTimeline(); + extraMetadata.put(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY, "def"); + addCommitToTimeline(metaClient, extraMetadata); + metaClient.reloadActiveTimeline(); + assertEquals(testDeltaSync.getPreviousCheckpoint(metaClient.getActiveTimeline().getCommitsTimeline()).get(), "def"); + } + + class TestDeltaSync extends DeltaSync { + + public TestDeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider, TypedProperties props, + JavaSparkContext jssc, FileSystem fs, Configuration conf, + Function onInitializingHoodieWriteClient) throws IOException { + super(cfg, sparkSession, schemaProvider, props, jssc, fs, conf, onInitializingHoodieWriteClient); + } + + protected Option getPreviousCheckpoint(HoodieTimeline timeline) throws IOException { + return super.getPreviousCheckpoint(timeline); + } + } + /** * UDF to calculate Haversine distance. */ @@ -1738,8 +1887,8 @@ public Schema getTargetSchema() { private static Stream testORCDFSSource() { // arg1 boolean useSchemaProvider, arg2 List transformerClassNames return Stream.of( - arguments(false, null), - arguments(true, Collections.singletonList(TripsWithDistanceTransformer.class.getName())) + arguments(false, null), + arguments(true, Collections.singletonList(TripsWithDistanceTransformer.class.getName())) ); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java index e67dfcca113d..dca5f0fdf929 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java @@ -19,7 +19,6 @@ package org.apache.hudi.utilities.functional; -import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -28,7 +27,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; @@ -37,7 +35,8 @@ import org.apache.hudi.utilities.testutils.sources.config.SourceConfigs; import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.sql.SaveMode; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.params.ParameterizedTest; @@ -53,6 +52,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import static org.apache.hudi.common.testutils.FixtureUtils.prepareFixtureTable; @@ -64,6 +64,7 @@ import static org.apache.hudi.config.HoodieWriteConfig.UPSERT_PARALLELISM_VALUE; import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.PROPS_FILENAME_TEST_MULTI_WRITER; +import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.addCommitToTimeline; import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.defaultSchemaProviderClassName; import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.prepareInitialConfigs; import static org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer.deltaStreamerTestRunner; @@ -75,6 +76,7 @@ public class TestHoodieDeltaStreamerWithMultiWriter extends SparkClientFunctionalTestHarness { private static final String COW_TEST_TABLE_NAME = "testtable_COPY_ON_WRITE"; + private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamerWithMultiWriter.class); String basePath; String propsFilePath; @@ -83,16 +85,17 @@ public class TestHoodieDeltaStreamerWithMultiWriter extends SparkClientFunctiona @ParameterizedTest @EnumSource(HoodieTableType.class) - void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType) throws Exception { + void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType tableType) throws Exception { // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts setUpTestTable(tableType); prepareInitialConfigs(fs(), basePath, "foo"); - // enable carrying forward latest checkpoint TypedProperties props = prepareMultiWriterProps(fs(), basePath, propsFilePath); props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); props.setProperty("hoodie.write.lock.filesystem.path", tableBasePath); - props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); - props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "10"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "250"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY,"250"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY,"10"); UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), propsFilePath); HoodieDeltaStreamer.Config cfgIngestionJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, @@ -106,7 +109,7 @@ void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType) thr propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); cfgBackfillJob.continuousMode = false; HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(hadoopConf()).setBasePath(tableBasePath).build(); - HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = meta.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); @@ -119,7 +122,23 @@ void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType) thr // run ingestion & backfill in parallel, create conflict and fail one runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob2, - cfgIngestionJob, backfillJob, cfgBackfillJob, true); + cfgIngestionJob, backfillJob, cfgBackfillJob, true, "batch1"); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableType tableType) throws Exception { + // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts + setUpTestTable(tableType); + prepareInitialConfigs(fs(), basePath, "foo"); + TypedProperties props = prepareMultiWriterProps(fs(), basePath, propsFilePath); + props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + props.setProperty("hoodie.write.lock.filesystem.path", tableBasePath); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "10"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "250"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY,"250"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY,"10"); + UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), propsFilePath); // create new ingestion & backfill job config to generate only INSERTS to avoid conflict props = prepareMultiWriterProps(fs(), basePath, propsFilePath); @@ -127,35 +146,39 @@ void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableType) thr props.setProperty("hoodie.write.lock.filesystem.path", tableBasePath); props.setProperty("hoodie.test.source.generate.inserts", "true"); UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), basePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); - cfgBackfillJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.INSERT, + HoodieDeltaStreamer.Config cfgBackfillJob2 = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.INSERT, propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TestIdentityTransformer.class.getName())); - cfgBackfillJob.continuousMode = false; - meta = HoodieTableMetaClient.builder().setConf(hadoopConf()).setBasePath(tableBasePath).build(); - timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - commitMetadata = HoodieCommitMetadata + cfgBackfillJob2.continuousMode = false; + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(hadoopConf()).setBasePath(tableBasePath).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); - cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); - cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob2.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); + cfgBackfillJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); - cfgIngestionJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, + HoodieDeltaStreamer.Config cfgIngestionJob2 = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TestIdentityTransformer.class.getName())); - cfgIngestionJob.continuousMode = true; - cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgIngestionJob2.continuousMode = true; + cfgIngestionJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfgIngestionJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); // re-init ingestion job - HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob, jsc()); + HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob2, jsc()); // re-init backfill job - HoodieDeltaStreamer backfillJob2 = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); + HoodieDeltaStreamer backfillJob2 = new HoodieDeltaStreamer(cfgBackfillJob2, jsc()); // run ingestion & backfill in parallel, avoid conflict and succeed both runJobsInParallel(tableBasePath, tableType, totalRecords, ingestionJob3, - cfgIngestionJob, backfillJob2, cfgBackfillJob, false); + cfgIngestionJob2, backfillJob2, cfgBackfillJob2, false, "batch2"); } @ParameterizedTest @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE"}) - void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) throws Exception { + public void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) throws Exception { + testCheckpointCarryOver(tableType); + } + + private void testCheckpointCarryOver(HoodieTableType tableType) throws Exception { // NOTE : Overriding the LockProvider to FileSystemBasedLockProviderTestClass since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts setUpTestTable(tableType); prepareInitialConfigs(fs(), basePath, "foo"); @@ -196,34 +219,28 @@ void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); backfillJob.sync(); - // Save the checkpoint information from the deltastreamer run and perform next write - String checkpointAfterDeltaSync = getLatestMetadata(meta).getMetadata(CHECKPOINT_KEY); - // this writer will enable HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE.key() so that deltastreamer checkpoint will be carried over. - performWriteWithDeltastreamerStateMerge(); + meta.reloadActiveTimeline(); + int totalCommits = meta.getCommitsTimeline().filterCompletedInstants().countInstants(); + + // add a new commit to timeline which may not have the checkpoint in extra metadata + addCommitToTimeline(meta); + meta.reloadActiveTimeline(); + verifyCommitMetadataCheckpoint(meta, null); - // Verify that the checkpoint is carried over - HoodieCommitMetadata commitMetaAfterDatasourceWrite = getLatestMetadata(meta); - Assertions.assertEquals(checkpointAfterDeltaSync, commitMetaAfterDatasourceWrite.getMetadata(CHECKPOINT_KEY)); + cfgBackfillJob.checkpoint = null; + new HoodieDeltaStreamer(cfgBackfillJob, jsc()).sync(); // if deltastreamer checkpoint fetch does not walk back to older commits, this sync will fail + meta.reloadActiveTimeline(); + Assertions.assertEquals(totalCommits + 2, meta.getCommitsTimeline().filterCompletedInstants().countInstants()); + verifyCommitMetadataCheckpoint(meta, "00008"); } - /** - * Performs a hudi datasource write with deltastreamer state merge enabled. - */ - private void performWriteWithDeltastreamerStateMerge() { - spark().read() - .format("hudi") - .load(tableBasePath + "/*/*.parquet") - .limit(1) - .write() - .format("hudi") - .option(HoodieWriteConfig.TBL_NAME.key(), COW_TEST_TABLE_NAME) - .option(DataSourceWriteOptions.OPERATION().key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) - .option(DataSourceWriteOptions.INSERT_DROP_DUPS().key(), "true") - .option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp") - .option(HoodieWriteConfig.WRITE_CONCURRENCY_MERGE_DELTASTREAMER_STATE.key(), "true") - .mode(SaveMode.Append) - .save(tableBasePath + "/*/*.parquet"); + private void verifyCommitMetadataCheckpoint(HoodieTableMetaClient metaClient, String expectedCheckpoint) throws IOException { + HoodieCommitMetadata commitMeta = getLatestMetadata(metaClient); + if (expectedCheckpoint == null) { + Assertions.assertNull(commitMeta.getMetadata(CHECKPOINT_KEY)); + } else { + Assertions.assertEquals(expectedCheckpoint, commitMeta.getMetadata(CHECKPOINT_KEY)); + } } private static HoodieCommitMetadata getLatestMetadata(HoodieTableMetaClient meta) throws IOException { @@ -305,7 +322,7 @@ private void setUpTestTable(HoodieTableType tableType) throws IOException { private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, int totalRecords, HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob, - HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict) throws Exception { + HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict, String jobId) throws Exception { ExecutorService service = Executors.newFixedThreadPool(2); HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(hadoopConf()).setBasePath(tableBasePath).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); @@ -322,18 +339,27 @@ private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, return true; }; + AtomicBoolean continousFailed = new AtomicBoolean(false); + AtomicBoolean backfillFailed = new AtomicBoolean(false); try { Future regularIngestionJobFuture = service.submit(() -> { try { - deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, conditionForRegularIngestion); - } catch (Exception ex) { + deltaStreamerTestRunner(ingestionJob, cfgIngestionJob, conditionForRegularIngestion, jobId); + } catch (Throwable ex) { + continousFailed.set(true); + LOG.error("Continuous job failed " + ex.getMessage()); throw new RuntimeException(ex); } }); Future backfillJobFuture = service.submit(() -> { try { + // trigger backfill atleast after 1 requested entry is added to timline from continuous job. If not, there is a chance that backfill will complete even before + // continous job starts. + awaitCondition(new GetCommitsAfterInstant(tableBasePath, lastSuccessfulCommit)); backfillJob.sync(); - } catch (Exception ex) { + } catch (Throwable ex) { + LOG.error("Backfilling job failed " + ex.getMessage()); + backfillFailed.set(true); throw new RuntimeException(ex); } }); @@ -349,10 +375,48 @@ private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, */ if (expectConflict && e.getCause().getMessage().contains(ConcurrentModificationException.class.getName())) { // expected ConcurrentModificationException since ingestion & backfill will have overlapping writes + if (backfillFailed.get()) { + // if backfill job failed, shutdown the continuous job. + LOG.warn("Calling shutdown on ingestion job since the backfill job has failed for " + jobId); + ingestionJob.shutdownGracefully(); + } } else { + LOG.error("Conflict happened, but not expected " + e.getCause().getMessage()); throw e; } } } + class GetCommitsAfterInstant { + + String basePath; + String lastSuccessfulCommit; + HoodieTableMetaClient meta; + GetCommitsAfterInstant(String basePath, String lastSuccessfulCommit) { + this.basePath = basePath; + this.lastSuccessfulCommit = lastSuccessfulCommit; + meta = HoodieTableMetaClient.builder().setConf(fs().getConf()).setBasePath(basePath).build(); + } + + long getCommitsAfterInstant() { + HoodieTimeline timeline1 = meta.reloadActiveTimeline().getAllCommitsTimeline().findInstantsAfter(lastSuccessfulCommit); + // LOG.info("Timeline Instants=" + meta1.getActiveTimeline().getInstants().collect(Collectors.toList())); + return timeline1.getInstants().count(); + } + } + + private static void awaitCondition(GetCommitsAfterInstant callback) throws InterruptedException { + long startTime = System.currentTimeMillis(); + long soFar = 0; + while (soFar <= 5000) { + if (callback.getCommitsAfterInstant() > 0) { + break; + } else { + Thread.sleep(500); + soFar += 500; + } + } + LOG.warn("Awaiting completed in " + (System.currentTimeMillis() - startTime)); + } + } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestAvroDFSSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestAvroDFSSource.java new file mode 100644 index 000000000000..37abaa56b1bb --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestAvroDFSSource.java @@ -0,0 +1,56 @@ +/* + * 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 org.apache.hudi.utilities.sources; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.utilities.testutils.sources.AbstractDFSSourceTestBase; +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.BeforeEach; +import java.io.IOException; +import java.util.List; + +/** + * Basic tests for {@link TestAvroDFSSource}. + */ +public class TestAvroDFSSource extends AbstractDFSSourceTestBase { + + @BeforeEach + public void setup() throws Exception { + super.setup(); + this.dfsRoot = dfsBasePath + "/avroFiles"; + this.fileSuffix = ".avro"; + } + + @Override + protected Source prepareDFSSource() { + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsRoot); + try { + return new AvroDFSSource(props, jsc, sparkSession, schemaProvider); + } catch (IOException e) { + return null; + } + } + + @Override + protected void writeNewDataToFile(List records, Path path) throws IOException { + Helpers.saveAvroToDFS(Helpers.toGenericRecords(records), path); + } +} \ No newline at end of file diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java new file mode 100644 index 000000000000..250e288294ac --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java @@ -0,0 +1,123 @@ +/* + * 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 org.apache.hudi.utilities.sources; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; + +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Properties; + +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestHoodieIncrSource extends HoodieClientTestHarness { + + @BeforeEach + public void setUp() throws IOException { + initResources(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupResources(); + } + + @Test + public void testHoodieIncrSource() throws IOException { + HoodieWriteConfig writeConfig = getConfigBuilder(basePath).build(); + + SparkRDDWriteClient writeClient = new SparkRDDWriteClient(context, writeConfig); + Pair> inserts = writeRecords(writeClient, true, null); + Pair> inserts2 = writeRecords(writeClient, true, null); + Pair> inserts3 = writeRecords(writeClient, true, null); + + // read everything upto latest + readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT, 300, inserts3.getKey()); + + // read just the latest + readAndAssert(IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST, 100, inserts3.getKey()); + } + + private void readAndAssert(IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy, int expectedCount, String expectedCheckpoint) { + + Properties properties = new Properties(); + properties.setProperty("hoodie.deltastreamer.source.hoodieincr.path", basePath); + properties.setProperty("hoodie.deltastreamer.source.hoodieincr.missing.checkpoint.strategy", missingCheckpointStrategy.name()); + TypedProperties typedProperties = new TypedProperties(properties); + HoodieIncrSource incrSource = new HoodieIncrSource(typedProperties, jsc, sparkSession, new TestSchemaProvider(HoodieTestDataGenerator.AVRO_SCHEMA)); + + // read everything until latest + Pair>, String> batchCheckPoint = incrSource.fetchNextBatch(Option.empty(), 500); + Assertions.assertNotNull(batchCheckPoint.getValue()); + assertEquals(batchCheckPoint.getKey().get().count(), expectedCount); + Assertions.assertEquals(batchCheckPoint.getRight(), expectedCheckpoint); + } + + public Pair> writeRecords(SparkRDDWriteClient writeClient, boolean insert, List insertRecords) throws IOException { + String commit = writeClient.startCommit(); + List records = insert ? dataGen.generateInserts(commit, 100) : dataGen.generateUpdates(commit, insertRecords); + JavaRDD result = writeClient.upsert(jsc.parallelize(records, 1), commit); + List statuses = result.collect(); + assertNoWriteErrors(statuses); + return Pair.of(commit, records); + } + + public HoodieWriteConfig.Builder getConfigBuilder(String basePath) { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .forTable("test-hoodie-incr-source"); + } + + class TestSchemaProvider extends SchemaProvider { + + private final Schema schema; + + public TestSchemaProvider(Schema schema) { + super(new TypedProperties()); + this.schema = schema; + } + + @Override + public Schema getSourceSchema() { + return schema; + } + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java index 2ed4c42582c3..ff0a19273f37 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java @@ -23,11 +23,11 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config; -import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -39,17 +39,19 @@ import org.apache.spark.sql.Row; import org.apache.spark.streaming.kafka010.KafkaTestUtils; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.net.URL; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.UUID; import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; +import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.jsonifyRecords; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -58,41 +60,37 @@ /** * Tests against {@link JsonKafkaSource}. */ -public class TestJsonKafkaSource extends UtilitiesTestBase { +public class TestJsonKafkaSource extends SparkClientFunctionalTestHarness { - private static String TEST_TOPIC_NAME = "hoodie_test"; + private static final String TEST_TOPIC_PREFIX = "hoodie_test_"; + private static final URL SCHEMA_FILE_URL = TestJsonKafkaSource.class.getClassLoader().getResource("delta-streamer-config/source.avsc"); + private static KafkaTestUtils testUtils; + private final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); private FilebasedSchemaProvider schemaProvider; - private KafkaTestUtils testUtils; - private HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); @BeforeAll public static void initClass() throws Exception { - UtilitiesTestBase.initClass(false); + testUtils = new KafkaTestUtils(); + testUtils.setup(); } @AfterAll public static void cleanupClass() { - UtilitiesTestBase.cleanupClass(); + testUtils.teardown(); } @BeforeEach - public void setup() throws Exception { - super.setup(); - schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc); - testUtils = new KafkaTestUtils(); - testUtils.setup(); - } - - @AfterEach - public void teardown() throws Exception { - super.teardown(); - testUtils.teardown(); + public void init() throws Exception { + String schemaFilePath = Objects.requireNonNull(SCHEMA_FILE_URL).toURI().getPath(); + TypedProperties props = new TypedProperties(); + props.put("hoodie.deltastreamer.schemaprovider.source.schema.file", schemaFilePath); + schemaProvider = new FilebasedSchemaProvider(props, jsc()); } - private TypedProperties createPropsForJsonSource(Long maxEventsToReadFromKafkaSource, String resetStrategy) { + private TypedProperties createPropsForJsonSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { TypedProperties props = new TypedProperties(); - props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME); + props.setProperty("hoodie.deltastreamer.source.kafka.topic", topic); props.setProperty("bootstrap.servers", testUtils.brokerAddress()); props.setProperty("auto.offset.reset", resetStrategy); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); @@ -107,16 +105,17 @@ private TypedProperties createPropsForJsonSource(Long maxEventsToReadFromKafkaSo public void testJsonKafkaSource() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSource"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(null, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); assertEquals(900, fetch1.getBatch().get().count()); // Test Avro To DataFrame path @@ -125,7 +124,7 @@ public void testJsonKafkaSource() { assertEquals(900, fetch1AsRows.count()); // 2. Produce new data, extract new data - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 1000))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(1100, fetch2.getBatch().get().count()); @@ -155,19 +154,20 @@ public void testJsonKafkaSource() { @Test public void testJsonKafkaSourceFilterNullMsg() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceFilterNullMsg"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(null, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); // Send 1000 non-null messages to Kafka - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); // Send 100 null messages to Kafka - testUtils.sendMessages(TEST_TOPIC_NAME,new String[100]); + testUtils.sendMessages(topic, new String[100]); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); // Verify that messages with null values are filtered assertEquals(1000, fetch1.getBatch().get().count()); @@ -177,15 +177,16 @@ public void testJsonKafkaSourceFilterNullMsg() { @Test public void testJsonKafkaSourceResetStrategy() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceResetStrategy"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties earliestProps = createPropsForJsonSource(null, "earliest"); - Source earliestJsonSource = new JsonKafkaSource(earliestProps, jsc, sparkSession, schemaProvider, metrics); + TypedProperties earliestProps = createPropsForJsonSource(topic, null, "earliest"); + Source earliestJsonSource = new JsonKafkaSource(earliestProps, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter earliestKafkaSource = new SourceFormatAdapter(earliestJsonSource); - TypedProperties latestProps = createPropsForJsonSource(null, "latest"); - Source latestJsonSource = new JsonKafkaSource(latestProps, jsc, sparkSession, schemaProvider, metrics); + TypedProperties latestProps = createPropsForJsonSource(topic, null, "latest"); + Source latestJsonSource = new JsonKafkaSource(latestProps, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter latestKafkaSource = new SourceFormatAdapter(latestJsonSource); // 1. Extract with a none data kafka checkpoint @@ -195,7 +196,7 @@ public void testJsonKafkaSourceResetStrategy() { assertEquals(earFetch0.getBatch(), latFetch0.getBatch()); assertEquals(earFetch0.getCheckpointForNextBatch(), latFetch0.getCheckpointForNextBatch()); - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); // 2. Extract new checkpoint with a null / empty string pre checkpoint // => earliest fetch with max source limit will get all of data and a end offset checkpoint @@ -209,23 +210,24 @@ public void testJsonKafkaSourceResetStrategy() { @Test public void testJsonKafkaSourceWithDefaultUpperCap() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceWithDefaultUpperCap"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(Long.MAX_VALUE, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, Long.MAX_VALUE, "earliest"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); /* 1. Extract without any checkpoint => get all the data, respecting default upper cap since both sourceLimit and maxEventsFromKafkaSourceProp are set to Long.MAX_VALUE */ - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); assertEquals(1000, fetch1.getBatch().get().count()); // 2. Produce new data, extract new data based on sourceLimit - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 1000))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), 1500); assertEquals(1000, fetch2.getBatch().get().count()); @@ -234,11 +236,12 @@ public void testJsonKafkaSourceWithDefaultUpperCap() { @Test public void testJsonKafkaSourceInsertRecordsLessSourceLimit() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceInsertRecordsLessSourceLimit"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(Long.MAX_VALUE, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, Long.MAX_VALUE, "earliest"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", "500"); @@ -246,7 +249,7 @@ public void testJsonKafkaSourceInsertRecordsLessSourceLimit() { 1. maxEventsFromKafkaSourceProp set to more than generated insert records and sourceLimit less than the generated insert records num. */ - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 400))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 400))); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 300); assertEquals(300, fetch1.getBatch().get().count()); @@ -254,7 +257,7 @@ public void testJsonKafkaSourceInsertRecordsLessSourceLimit() { 2. Produce new data, extract new data based on sourceLimit and sourceLimit less than the generated insert records num. */ - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 600))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 600))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), 300); assertEquals(300, fetch2.getBatch().get().count()); @@ -263,20 +266,21 @@ public void testJsonKafkaSourceInsertRecordsLessSourceLimit() { @Test public void testJsonKafkaSourceWithConfigurableUpperCap() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceWithConfigurableUpperCap"; + testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(500L, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, 500L, "earliest"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); assertEquals(900, fetch1.getBatch().get().count()); // 2. Produce new data, extract new data based on upper cap - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 1000))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(500, fetch2.getBatch().get().count()); @@ -306,22 +310,23 @@ public void testJsonKafkaSourceWithConfigurableUpperCap() { @Test public void testCommitOffsetToKafka() { // topic setup. - testUtils.createTopic(TEST_TOPIC_NAME, 2); + final String topic = TEST_TOPIC_PREFIX + "testCommitOffsetToKafka"; + testUtils.createTopic(topic, 2); List topicPartitions = new ArrayList<>(); - TopicPartition topicPartition0 = new TopicPartition(TEST_TOPIC_NAME, 0); + TopicPartition topicPartition0 = new TopicPartition(topic, 0); topicPartitions.add(topicPartition0); - TopicPartition topicPartition1 = new TopicPartition(TEST_TOPIC_NAME, 1); + TopicPartition topicPartition1 = new TopicPartition(topic, 1); topicPartitions.add(topicPartition1); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(null, "earliest"); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); props.put(ENABLE_KAFKA_COMMIT_OFFSET.key(), "true"); - Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 599); // commit to kafka after first batch @@ -340,7 +345,7 @@ public void testCommitOffsetToKafka() { assertEquals(500L, endOffsets.get(topicPartition0)); assertEquals(500L, endOffsets.get(topicPartition1)); - testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 500))); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 500))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestAbstractDebeziumSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestAbstractDebeziumSource.java new file mode 100644 index 000000000000..f7dc6b9258c1 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestAbstractDebeziumSource.java @@ -0,0 +1,209 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.debezium.DebeziumConstants; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.utilities.UtilHelpers; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.schema.SchemaRegistryProvider; +import org.apache.hudi.utilities.sources.InputBatch; +import org.apache.hudi.utilities.testutils.UtilitiesTestBase; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.streaming.kafka010.KafkaTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.params.provider.Arguments.arguments; +import static org.mockito.Mockito.mock; + +public abstract class TestAbstractDebeziumSource extends UtilitiesTestBase { + + private static final String TEST_TOPIC_NAME = "hoodie_test"; + + private final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); + private KafkaTestUtils testUtils; + + @BeforeAll + public static void initClass() throws Exception { + UtilitiesTestBase.initClass(false); + } + + @AfterAll + public static void cleanupClass() { + UtilitiesTestBase.cleanupClass(); + } + + @BeforeEach + public void setup() throws Exception { + super.setup(); + testUtils = new KafkaTestUtils(); + testUtils.setup(); + } + + @AfterEach + public void teardown() throws Exception { + super.teardown(); + testUtils.teardown(); + } + + private TypedProperties createPropsForJsonSource() { + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME); + props.setProperty("bootstrap.servers", testUtils.brokerAddress()); + props.setProperty("auto.offset.reset", "earliest"); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.setProperty("hoodie.deltastreamer.schemaprovider.registry.url", "localhost"); + props.setProperty("hoodie.deltastreamer.source.kafka.value.deserializer.class", StringDeserializer.class.getName()); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + + return props; + } + + protected abstract String getIndexName(); + + protected abstract String getSourceClass(); + + protected abstract String getSchema(); + + protected abstract GenericRecord generateMetaFields(GenericRecord record); + + protected abstract void validateMetaFields(Dataset records); + + @ParameterizedTest + @MethodSource("testArguments") + public void testDebeziumEvents(Operation operation) throws Exception { + + String sourceClass = getSourceClass(); + + // topic setup. + testUtils.createTopic(TEST_TOPIC_NAME, 2); + TypedProperties props = createPropsForJsonSource(); + + SchemaProvider schemaProvider = new MockSchemaRegistryProvider(props, jsc, this); + SourceFormatAdapter debeziumSource = new SourceFormatAdapter(UtilHelpers.createSource(sourceClass, props, jsc, sparkSession, schemaProvider, metrics)); + + testUtils.sendMessages(TEST_TOPIC_NAME, new String[] {generateDebeziumEvent(operation).toString()}); + + InputBatch> fetch = debeziumSource.fetchNewDataInRowFormat(Option.empty(), 10); + assertEquals(1, fetch.getBatch().get().count()); + + // Ensure the before fields are picked for DELETE CDC Events, + // and after fields are picked for INSERT and UPDATE CDC Events. + final String fieldPrefix = (operation.equals(Operation.DELETE)) ? "before_" : "after_"; + assertTrue(fetch.getBatch().get().select("type").collectAsList().stream() + .allMatch(r -> r.getString(0).startsWith(fieldPrefix))); + assertTrue(fetch.getBatch().get().select("type").collectAsList().stream() + .allMatch(r -> r.getString(0).startsWith(fieldPrefix))); + + // Validate DB specific meta fields + validateMetaFields(fetch.getBatch().get()); + } + + private GenericRecord generateDebeziumEvent(Operation op) { + Schema schema = new Schema.Parser().parse(getSchema()); + String indexName = getIndexName().concat(".ghschema.gharchive.Value"); + GenericRecord rec = new GenericData.Record(schema); + rec.put(DebeziumConstants.INCOMING_OP_FIELD, op.op); + rec.put(DebeziumConstants.INCOMING_TS_MS_FIELD, 100L); + + // Before + Schema.Field beforeField = schema.getField(DebeziumConstants.INCOMING_BEFORE_FIELD); + Schema beforeSchema = beforeField.schema().getTypes().get(beforeField.schema().getIndexNamed(indexName)); + GenericRecord beforeRecord = new GenericData.Record(beforeSchema); + + beforeRecord.put("id", 1); + beforeRecord.put("date", "1/1/2020"); + beforeRecord.put("type", "before_type"); + beforeRecord.put("payload", "before_payload"); + beforeRecord.put("timestamp", 1000L); + rec.put(DebeziumConstants.INCOMING_BEFORE_FIELD, beforeRecord); + + // After + Schema.Field afterField = schema.getField(DebeziumConstants.INCOMING_AFTER_FIELD); + Schema afterSchema = afterField.schema().getTypes().get(afterField.schema().getIndexNamed(indexName)); + GenericRecord afterRecord = new GenericData.Record(afterSchema); + + afterRecord.put("id", 1); + afterRecord.put("date", "1/1/2021"); + afterRecord.put("type", "after_type"); + afterRecord.put("payload", "after_payload"); + afterRecord.put("timestamp", 3000L); + rec.put(DebeziumConstants.INCOMING_AFTER_FIELD, afterRecord); + + return generateMetaFields(rec); + } + + private static class MockSchemaRegistryProvider extends SchemaRegistryProvider { + + private final String schema; + + public MockSchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc, TestAbstractDebeziumSource source) { + super(props, jssc); + schema = source.getSchema(); + } + + @Override + public String fetchSchemaFromRegistry(String registryUrl) throws IOException { + return schema; + } + } + + private static Stream testArguments() { + return Stream.of( + arguments(Operation.INSERT), + arguments(Operation.UPDATE), + arguments(Operation.DELETE) + ); + } + + private enum Operation { + INSERT("c"), + UPDATE("u"), + DELETE("d"); + + public final String op; + + Operation(String op) { + this.op = op; + } + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestMysqlDebeziumSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestMysqlDebeziumSource.java new file mode 100644 index 000000000000..1d09cc8e4aae --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestMysqlDebeziumSource.java @@ -0,0 +1,96 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import org.apache.hudi.common.model.debezium.DebeziumConstants; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestMysqlDebeziumSource extends TestAbstractDebeziumSource { + + private static final String MYSQL_GITHUB_SCHEMA = "{\"connect.name\": \"mysql.ghschema.gharchive.Envelope\",\n" + + " \"fields\": [{\"default\": null,\"name\": \"before\",\"type\": [\"null\",{\"connect.name\": \"mysql.ghschema.gharchive.Value\",\n" + + " \"fields\": [{\"name\": \"id\",\"type\": \"string\"},{\"name\": \"date\",\"type\": \"string\"},{\"default\": null,\"name\": \"timestamp\",\n" + + " \"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"type\",\"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"payload\",\n" + + " \"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"org\",\"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"created_at\",\n" + + " \"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"public\",\"type\": [\"null\",\"boolean\"]}],\"name\": \"Value\",\"type\": \"record\"\n" + + " }]},{\"default\": null,\"name\": \"after\",\"type\": [\"null\",\"Value\"]},{\"name\": \"source\",\"type\": {\"connect.name\": \"io.debezium.connector.mysql.Source\",\n" + + " \"fields\": [{\"name\": \"connector\",\"type\": \"string\"},{\"name\": \"name\",\"type\": \"string\"},{\"name\": \"ts_ms\",\"type\": \"long\"},\n" + + " {\"name\": \"db\",\"type\": \"string\"},{\"name\": \"table\",\"type\": \"string\"},{\"default\": null,\n" + + " \"name\": \"txId\",\"type\": [\"null\",\"long\"]},{\"name\": \"file\",\"type\": \"string\"},{\"default\": null,\"name\": \"pos\",\"type\": [\"null\",\"long\"]},{\"default\": null,\n" + + " \"name\": \"row\",\"type\": [\"null\",\"long\"]}],\"name\": \"Source\",\"namespace\": \"io.debezium.connector.mysql\",\"type\": \"record\"\n" + + " }},{\"name\": \"op\",\"type\": \"string\"},{\"default\": null,\"name\": \"ts_ms\",\"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"transaction\",\n" + + " \"type\": [\"null\",{\"fields\": [{\"name\": \"id\",\"type\": \"string\"},{\"name\": \"total_order\",\"type\": \"long\"},{\"name\": \"data_collection_order\",\n" + + " \"type\": \"long\"}],\"name\": \"ConnectDefault\",\"namespace\": \"io.confluent.connect.avro\",\"type\": \"record\"}]}],\"name\": \"Envelope\",\n" + + " \"namespace\": \"mysql.ghschema.gharchive\",\"type\": \"record\"}"; + + private static final String TEST_DB = "ghschema"; + private static final String TEST_TABLE = "gharchive"; + private static final long TEST_TS_MS = 12345L; + private static final String TEST_FILE = "mysql-bin.00007"; + private static final long TEST_POS = 98765L; + private static final String EXPECTED_TEST_SEQ = "00007.98765"; + + @Override + protected String getIndexName() { + return "mysql"; + } + + @Override + protected String getSourceClass() { + return MysqlDebeziumSource.class.getName(); + } + + @Override + protected String getSchema() { + return MYSQL_GITHUB_SCHEMA; + } + + @Override + protected GenericRecord generateMetaFields(GenericRecord rec) { + Schema schema = new Schema.Parser().parse(getSchema()); + // Source fields specific to Mysql DB + GenericRecord sourceRecord = new GenericData.Record(schema.getField(DebeziumConstants.INCOMING_SOURCE_FIELD).schema()); + sourceRecord.put("name", getIndexName()); + sourceRecord.put("connector", getIndexName()); + sourceRecord.put("db", TEST_DB); + sourceRecord.put("table", TEST_TABLE); + sourceRecord.put("ts_ms", TEST_TS_MS); + sourceRecord.put("file", TEST_FILE); + sourceRecord.put("pos", TEST_POS); + rec.put(DebeziumConstants.INCOMING_SOURCE_FIELD, sourceRecord); + return rec; + } + + @Override + protected void validateMetaFields(Dataset records) { + assertTrue(records.select(DebeziumConstants.FLATTENED_SHARD_NAME).collectAsList().stream() + .allMatch(r -> r.getString(0).equals(getIndexName()))); + assertTrue(records.select(DebeziumConstants.FLATTENED_TS_COL_NAME).collectAsList().stream() + .allMatch(r -> r.getLong(0) == TEST_TS_MS)); + assertTrue(records.select(DebeziumConstants.ADDED_SEQ_COL_NAME).collectAsList().stream() + .allMatch(r -> r.getString(0).equals(EXPECTED_TEST_SEQ))); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestPostgresDebeziumSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestPostgresDebeziumSource.java new file mode 100644 index 000000000000..ef75fc61ff0f --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/debezium/TestPostgresDebeziumSource.java @@ -0,0 +1,97 @@ +/* + * 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 org.apache.hudi.utilities.sources.debezium; + +import org.apache.hudi.common.model.debezium.DebeziumConstants; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestPostgresDebeziumSource extends TestAbstractDebeziumSource { + + private static final String POSTGRES_GITHUB_SCHEMA = "{\"connect.name\": \"postgres.ghschema.gharchive.Envelope\",\n" + + " \"fields\": [{\"default\": null,\"name\": \"before\",\"type\": [\"null\",{\"connect.name\": \"postgres.ghschema.gharchive.Value\",\n" + + " \"fields\": [{\"name\": \"id\",\"type\": \"string\"},{\"name\": \"date\",\"type\": \"string\"},{\"default\": null,\"name\": \"timestamp\",\n" + + " \"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"type\",\"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"payload\",\n" + + " \"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"org\",\"type\": [\"null\",\"string\"]},{\"default\": null,\"name\": \"created_at\",\n" + + " \"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"public\",\"type\": [\"null\",\"boolean\"]}],\"name\": \"Value\",\"type\": \"record\"\n" + + " }]},{\"default\": null,\"name\": \"after\",\"type\": [\"null\",\"Value\"]},{\"name\": \"source\",\"type\": {\"connect.name\": \"io.debezium.connector.postgresql.Source\",\n" + + " \"fields\": [{\"name\": \"connector\",\"type\": \"string\"},{\"name\": \"name\",\"type\": \"string\"},{\"name\": \"ts_ms\",\"type\": \"long\"},\n" + + " {\"name\": \"db\",\"type\": \"string\"},{\"name\": \"schema\",\"type\": \"string\"},{\"name\": \"table\",\"type\": \"string\"},{\"default\": null,\n" + + " \"name\": \"txId\",\"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"lsn\",\"type\": [\"null\",\"long\"]},{\"default\": null,\n" + + " \"name\": \"xmin\",\"type\": [\"null\",\"long\"]}],\"name\": \"Source\",\"namespace\": \"io.debezium.connector.postgresql\",\"type\": \"record\"\n" + + " }},{\"name\": \"op\",\"type\": \"string\"},{\"default\": null,\"name\": \"ts_ms\",\"type\": [\"null\",\"long\"]},{\"default\": null,\"name\": \"transaction\",\n" + + " \"type\": [\"null\",{\"fields\": [{\"name\": \"id\",\"type\": \"string\"},{\"name\": \"total_order\",\"type\": \"long\"},{\"name\": \"data_collection_order\",\n" + + " \"type\": \"long\"}],\"name\": \"ConnectDefault\",\"namespace\": \"io.confluent.connect.avro\",\"type\": \"record\"}]}],\"name\": \"Envelope\",\n" + + " \"namespace\": \"postgres.ghschema.gharchive\",\"type\": \"record\"}"; + + private static final String TEST_DB = "postgres"; + private static final String TEST_SCHEMA = "ghschema"; + private static final String TEST_TABLE = "gharchive"; + private static final long TEST_TS_MS = 12345L; + private static final long TEST_TXID = 543L; + private static final long TEST_LSN = 98765L; + + @Override + protected String getIndexName() { + return "postgres"; + } + + @Override + protected String getSourceClass() { + return PostgresDebeziumSource.class.getName(); + } + + @Override + protected String getSchema() { + return POSTGRES_GITHUB_SCHEMA; + } + + @Override + protected GenericRecord generateMetaFields(GenericRecord rec) { + Schema schema = new Schema.Parser().parse(getSchema()); + // Source fields specific to Postgres DB + GenericRecord sourceRecord = new GenericData.Record(schema.getField(DebeziumConstants.INCOMING_SOURCE_FIELD).schema()); + sourceRecord.put("name", getIndexName()); + sourceRecord.put("connector", getIndexName()); + sourceRecord.put("db", TEST_DB); + sourceRecord.put("schema", TEST_SCHEMA); + sourceRecord.put("table", TEST_TABLE); + sourceRecord.put("ts_ms", TEST_TS_MS); + sourceRecord.put("txId", TEST_TXID); + sourceRecord.put("lsn", TEST_LSN); + rec.put(DebeziumConstants.INCOMING_SOURCE_FIELD, sourceRecord); + return rec; + } + + @Override + protected void validateMetaFields(Dataset records) { + assertTrue(records.select(DebeziumConstants.FLATTENED_TS_COL_NAME).collectAsList().stream() + .allMatch(r -> r.getLong(0) == TEST_TS_MS)); + assertTrue(records.select(DebeziumConstants.FLATTENED_TX_ID_COL_NAME).collectAsList().stream() + .allMatch(r -> r.getLong(0) == TEST_TXID)); + assertTrue(records.select(DebeziumConstants.FLATTENED_LSN_COL_NAME).collectAsList().stream() + .allMatch(r -> r.getLong(0) == TEST_LSN)); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/SparkClientFunctionalTestHarnessWithHiveSupport.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/SparkClientFunctionalTestHarnessWithHiveSupport.java new file mode 100644 index 000000000000..fd59d633e7ae --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/SparkClientFunctionalTestHarnessWithHiveSupport.java @@ -0,0 +1,32 @@ +/* + * 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 org.apache.hudi.utilities.testutils; + +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; +import org.apache.spark.SparkConf; + +import java.util.Collections; + +public class SparkClientFunctionalTestHarnessWithHiveSupport extends SparkClientFunctionalTestHarness { + + public SparkConf conf() { + return conf(Collections.singletonMap("spark.sql.catalogImplementation", "hive")); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index bb00d2fef732..90a3f5af3802 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -48,6 +48,8 @@ import com.fasterxml.jackson.dataformat.csv.CsvSchema; import com.fasterxml.jackson.dataformat.csv.CsvSchema.Builder; import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; @@ -78,6 +80,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; +import java.io.OutputStream; import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; @@ -341,6 +344,20 @@ public static void saveORCToDFS(List records, Path targetFile, Ty } } + public static void saveAvroToDFS(List records, Path targetFile) throws IOException { + saveAvroToDFS(records,targetFile,HoodieTestDataGenerator.AVRO_SCHEMA); + } + + public static void saveAvroToDFS(List records, Path targetFile, Schema schema) throws IOException { + FileSystem fs = targetFile.getFileSystem(HoodieTestUtils.getDefaultHadoopConf()); + OutputStream output = fs.create(targetFile); + try (DataFileWriter dataFileWriter = new DataFileWriter<>(new GenericDatumWriter(schema)).create(schema, output)) { + for (GenericRecord record : records) { + dataFileWriter.append(record); + } + } + } + public static TypedProperties setupSchemaOnDFS() throws IOException { return setupSchemaOnDFS("delta-streamer-config", "source.avsc"); } diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_source.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_source.avsc new file mode 100644 index 000000000000..5b1c62babfcb --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_source.avsc @@ -0,0 +1,103 @@ +/* + * 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. + */ + +{ + "type": "record", + "name": "source_schema_tab", + "namespace": "hoodie.schema_registry", + "fields": [ + { + "name": "id", + "type": [ + "long", + "null" + ] + }, + { + "name": "name", + "type": [ + "string", + "null" + ] + }, + { + "name": "num1", + "type": [ + "int", + "null" + ] + }, + { + "name": "num2", + "type": [ + "long", + "null" + ] + }, + { + "name": "num3", + "type": [ + { + "type": "fixed", + "name": "fixed", + "namespace": "hoodie.schema_registry.source_schema_tab.num3", + "size": 9, + "logicalType": "decimal", + "precision": 20, + "scale": 0 + }, + "null" + ] + }, + { + "name": "num4", + "type": [ + "int", + "null" + ] + }, + { + "name": "num5", + "type": [ + "float", + "null" + ] + }, + { + "name": "num6", + "type": [ + "double", + "null" + ] + }, + { + "name": "bool", + "type": [ + "boolean", + "null" + ] + }, + { + "name": "bin", + "type": [ + "bytes", + "null" + ] + } + ] +} \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_target.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_target.avsc new file mode 100644 index 000000000000..d3d95ed14b47 --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/hive_schema_provider_target.avsc @@ -0,0 +1,103 @@ +/* + * 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. + */ + +{ + "type": "record", + "name": "target_schema_tab", + "namespace": "hoodie.schema_registry", + "fields": [ + { + "name": "id", + "type": [ + "long", + "null" + ] + }, + { + "name": "name", + "type": [ + "string", + "null" + ] + }, + { + "name": "num1", + "type": [ + "int", + "null" + ] + }, + { + "name": "num2", + "type": [ + "long", + "null" + ] + }, + { + "name": "num3", + "type": [ + { + "type": "fixed", + "name": "fixed", + "namespace": "hoodie.schema_registry.target_schema_tab.num3", + "size": 9, + "logicalType": "decimal", + "precision": 20, + "scale": 0 + }, + "null" + ] + }, + { + "name": "num4", + "type": [ + "int", + "null" + ] + }, + { + "name": "num5", + "type": [ + "float", + "null" + ] + }, + { + "name": "num6", + "type": [ + "double", + "null" + ] + }, + { + "name": "bool", + "type": [ + "boolean", + "null" + ] + }, + { + "name": "bin", + "type": [ + "bytes", + "null" + ] + } + ] +} \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.source_schema_tab.sql b/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.source_schema_tab.sql new file mode 100644 index 000000000000..b95ae0f5ee15 --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.source_schema_tab.sql @@ -0,0 +1,12 @@ +CREATE TABLE IF NOT EXISTS `schema_registry`.`source_schema_tab`( + `id` BIGINT, + `name` STRING, + `num1` INT, + `num2` BIGINT, + `num3` DECIMAL(20,0), + `num4` TINYINT, + `num5` FLOAT, + `num6` DOUBLE, + `bool` BOOLEAN, + `bin` BINARY +) \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.target_schema_tab.sql b/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.target_schema_tab.sql new file mode 100644 index 000000000000..07f179f90ed9 --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/schema_registry.target_schema_tab.sql @@ -0,0 +1,12 @@ +CREATE TABLE IF NOT EXISTS `schema_registry`.`target_schema_tab`( + `id` BIGINT, + `name` STRING, + `num1` INT, + `num2` BIGINT, + `num3` DECIMAL(20,0), + `num4` TINYINT, + `num5` FLOAT, + `num6` DOUBLE, + `bool` BOOLEAN, + `bin` BINARY +) \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip b/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip index 48bf278bd6c6..299b070bee34 100644 Binary files a/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip and b/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.COPY_ON_WRITE.zip differ diff --git a/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.MERGE_ON_READ.zip b/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.MERGE_ON_READ.zip index 657f83c2d0a5..d80439d20d3d 100644 Binary files a/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.MERGE_ON_READ.zip and b/hudi-utilities/src/test/resources/fixtures/testUpsertsContinuousModeWithMultipleWriters.MERGE_ON_READ.zip differ diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 868912eea4ff..bbc9620b4bd2 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -81,6 +81,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander @@ -106,6 +107,8 @@ com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.core:jackson-core + com.github.davidmoten:guava-mini + com.github.davidmoten:hilbert-curve com.twitter:bijection-avro_${scala.binary.version} com.twitter:bijection-core_${scala.binary.version} io.dropwizard.metrics:metrics-core @@ -127,7 +130,6 @@ com.esotericsoftware:kryo-shaded org.apache.flink:flink-hadoop-compatibility_${scala.binary.version} - org.apache.flink:flink-avro org.apache.flink:flink-json org.apache.flink:flink-parquet_${scala.binary.version} @@ -204,6 +206,10 @@ org.apache.hadoop.hive.ql.metadata. ${flink.bundle.shade.prefix}org.apache.hadoop.hive.ql.metadata. + + org.apache.hadoop.hive.ql.optimizer. + ${flink.bundle.shade.prefix}org.apache.hadoop.hive.ql.optimizer. + com.codahale.metrics. ${flink.bundle.shade.prefix}com.codahale.metrics. @@ -216,6 +222,15 @@ org.eclipse.jetty. ${flink.bundle.shade.prefix}org.apache.jetty. + + + com.esotericsoftware.kryo. + ${flink.bundle.shade.prefix}com.esotericsoftware.kryo. + + + com.fasterxml.jackson. + ${flink.bundle.shade.prefix}com.fasterxml.jackson. + @@ -322,12 +337,6 @@ ${flink.version} compile - - org.apache.flink - flink-avro - ${flink.version} - compile - org.apache.flink flink-parquet_${scala.binary.version} @@ -435,24 +444,6 @@ ${hive.groupid} hive-exec ${hive.version} - - - javax.mail - mail - - - org.eclipse.jetty.aggregate - * - - - guava - com.google.guava - - - org.pentaho - * - - ${hive.groupid} diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 76ede4738956..a426e74df08d 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index 75d225ab1c9b..967eab469fb7 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 7518e7b44ef0..d90f6acb0189 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -17,7 +17,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -78,6 +78,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws org.apache.hudi:hudi-integ-test org.jetbrains.kotlin:kotlin-stdlib-jdk8 @@ -379,12 +380,6 @@ ${hive.groupid} hive-exec ${hive.version} - - - org.pentaho - * - - compile diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml index debbfa785064..f66bc7f051e4 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -30,6 +30,7 @@ true ${project.parent.basedir} + org.apache.hudi. @@ -67,19 +68,80 @@ - - com.amazonaws.* - org.apache.zookeeper:zookeeper - com.fasterxml.jackson.core:jackson-annotations - commons-httpclient:commons-httpclient - org.apache.htrace:htrace-core - org.jamon:jamon-runtime - org.slf4j:* - log4j:log4j - jdk.tools:jdk.tools - junit:junit - + + org.apache.hudi:hudi-common + org.apache.hudi:hudi-client-common + org.apache.hudi:hudi-java-client + org.apache.hudi:hudi-spark-client + org.apache.hudi:hudi-spark-common_${scala.binary.version} + org.apache.hudi:hudi-kafka-connect + org.apache.hudi:hudi-utilities_${scala.binary.version} + org.apache.hudi:hudi-hive-sync + org.apache.hudi:hudi-sync-common + org.apache.hudi:hudi-hadoop-mr + org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws + + + org.apache.hudi:hudi-flink_${scala.binary.version} + org.apache.hudi:flink-core + org.apache.hudi:hudi-flink-client + org.apache.flink:flink-core + org.apache.flink:flink-hadoop-compatibility_${scala.binary.version} + + com.github.davidmoten:guava-mini + com.github.davidmoten:hilbert-curve + com.yammer.metrics:metrics-core + com.beust:jcommander + io.javalin:javalin + org.jetbrains.kotlin:* + org.eclipse.jetty:* + org.eclipse.jetty.websocket:* + org.rocksdb:rocksdbjni + org.apache.httpcomponents:httpclient + org.apache.httpcomponents:httpcore + org.apache.httpcomponents:fluent-hc + + io.dropwizard.metrics:metrics-core + io.dropwizard.metrics:metrics-graphite + io.dropwizard.metrics:metrics-jmx + io.prometheus:simpleclient + io.prometheus:simpleclient_httpserver + io.prometheus:simpleclient_dropwizard + io.prometheus:simpleclient_pushgateway + io.prometheus:simpleclient_common + com.yammer.metrics:metrics-core + com.google.protobuf:protobuf-java + org.objenesis:objenesis + com.esotericsoftware:kryo-shaded + com.esotericsoftware:minlog + + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core + org.scala-lang:* + + + + com.google.protobuf. + ${kafka.connect.bundle.shade.prefix}com.google.protobuf. + + + com.yammer.metrics. + ${kafka.connect.bundle.shade.prefix}com.yammer.metrics. + + + com.beust.jcommander. + ${kafka.connect.bundle.shade.prefix}com.beust.jcommander. + + + org.eclipse.jetty. + ${kafka.connect.bundle.shade.prefix}org.eclipse.jetty. + + *:* @@ -108,11 +170,18 @@ - + org.apache.hudi hudi-kafka-connect ${project.version} + compile + + + log4j + log4j + + org.apache.hudi @@ -143,6 +212,22 @@ + + org.apache.hudi + hudi-hive-sync + ${project.version} + + + javax.servlet + servlet-api + + + + + org.apache.hudi + hudi-spark-common_${scala.binary.version} + ${project.version} + @@ -170,7 +255,6 @@ org.apache.hadoop hadoop-common ${hadoop.version} - compile org.mortbay.jetty @@ -190,8 +274,61 @@ org.apache.hadoop hadoop-auth ${hadoop.version} + + + + + ${hive.groupid} + hive-service + ${hive.version} + ${utilities.bundle.hive.scope} + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + + + ${hive.groupid} + hive-service-rpc + ${hive.version} + ${utilities.bundle.hive.scope} + + + + ${hive.groupid} + hive-jdbc + ${hive.version} + ${utilities.bundle.hive.scope} + + + + ${hive.groupid} + hive-metastore + ${hive.version} + ${utilities.bundle.hive.scope} + + + + ${hive.groupid} + hive-common + ${hive.version} + ${utilities.bundle.hive.scope} + + + + org.apache.htrace + htrace-core + ${htrace.version} compile + diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 7833bda135aa..284f748c4e22 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -158,11 +158,65 @@ org.apache.hudi hudi-common ${project.version} + + + org.apache.hbase + hbase-server + + + org.apache.hbase + hbase-client + + org.apache.hudi hudi-hadoop-mr-bundle ${project.version} + + + org.apache.hbase + hbase-server + + + org.apache.hbase + hbase-client + + + + + + + org.apache.hbase + hbase-shaded-client + ${hbase.version} + test + + + + org.apache.hbase + hbase-shaded-server + ${hbase.version} + + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index d55b39493dca..7ab8c2170022 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -73,6 +73,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.beust:jcommander io.javalin:javalin @@ -87,6 +88,8 @@ org.antlr:stringtemplate org.apache.parquet:parquet-avro + com.github.davidmoten:guava-mini + com.github.davidmoten:hilbert-curve com.twitter:bijection-avro_${scala.binary.version} com.twitter:bijection-core_${scala.binary.version} io.dropwizard.metrics:metrics-core @@ -100,6 +103,10 @@ com.yammer.metrics:metrics-core com.google.guava:guava + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core + org.apache.spark:spark-avro_${scala.binary.version} org.apache.hive:hive-common org.apache.hive:hive-service diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index d775cb91d737..618d3d212231 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -144,7 +144,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.4 + ${maven-shade-plugin.version} true diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml new file mode 100644 index 000000000000..386369316ae5 --- /dev/null +++ b/packaging/hudi-trino-bundle/pom.xml @@ -0,0 +1,273 @@ + + + + + hudi + org.apache.hudi + 0.11.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + hudi-trino-bundle + jar + + + true + ${project.parent.basedir} + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + ${maven-shade-plugin.version} + + + package + + shade + + + ${shadeSources} + ${project.build.directory}/dependency-reduced-pom.xml + + + + + + true + + + META-INF/LICENSE + target/classes/META-INF/LICENSE + + + + + org.apache.hudi:hudi-common + org.apache.hudi:hudi-hadoop-mr + + org.apache.parquet:parquet-avro + org.apache.avro:avro + org.codehaus.jackson:* + com.esotericsoftware:kryo-shaded + org.objenesis:objenesis + com.esotericsoftware:minlog + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.hbase:hbase-annotations + org.apache.htrace:htrace-core + com.yammer.metrics:metrics-core + com.google.guava:guava + commons-lang:commons-lang + com.google.protobuf:protobuf-java + + + + + + org.apache.avro. + org.apache.hudi.org.apache.avro. + + + org.codehaus.jackson. + org.apache.hudi.org.codehaus.jackson. + + + com.esotericsoftware.kryo. + org.apache.hudi.com.esotericsoftware.kryo. + + + org.objenesis. + org.apache.hudi.org.objenesis. + + + com.esotericsoftware.minlog. + org.apache.hudi.com.esotericsoftware.minlog. + + + com.yammer.metrics. + org.apache.hudi.com.yammer.metrics. + + + com.google.common. + ${trino.bundle.bootstrap.shade.prefix}com.google.common. + + + org.apache.commons.lang. + ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.lang. + + + com.google.protobuf. + ${trino.bundle.bootstrap.shade.prefix}com.google.protobuf. + + + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/services/javax.* + + + + ${project.artifactId}-${project.version} + + + + + + + + src/main/resources + + + src/test/resources + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hbase + hbase-server + + + org.apache.hbase + hbase-client + + + + + org.apache.hudi + hudi-hadoop-mr-bundle + ${project.version} + + + org.apache.hbase + hbase-server + + + org.apache.hbase + hbase-client + + + + + + + org.apache.hbase + hbase-shaded-client + ${hbase.version} + test + + + + org.apache.hbase + hbase-shaded-server + ${hbase.version} + + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + + + + + + org.apache.parquet + parquet-avro + compile + + + + org.apache.avro + avro + compile + + + + + com.google.guava + guava + 12.0.1 + ${trino.bundle.bootstrap.scope} + + + + + commons-lang + commons-lang + 2.6 + ${trino.bundle.bootstrap.scope} + + + + + com.google.protobuf + protobuf-java + 2.5.0 + ${trino.bundle.bootstrap.scope} + + + + + + trino-shade-unbundle-bootstrap + + provided + + + + + diff --git a/packaging/hudi-trino-bundle/src/main/java/org/apache/hudi/trino/bundle/Main.java b/packaging/hudi-trino-bundle/src/main/java/org/apache/hudi/trino/bundle/Main.java new file mode 100644 index 000000000000..eec1ecf88a8d --- /dev/null +++ b/packaging/hudi-trino-bundle/src/main/java/org/apache/hudi/trino/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * 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 org.apache.hudi.trino.bundle; + +import org.apache.hudi.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath + *

+ * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + *

+ * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 63b3b7da8a4b..b9e372ca2c77 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -97,6 +97,7 @@ org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + org.apache.hudi:hudi-aws com.yammer.metrics:metrics-core com.beust:jcommander @@ -112,6 +113,12 @@ org.antlr:stringtemplate org.apache.parquet:parquet-avro + com.amazonaws:dynamodb-lock-client + com.amazonaws:aws-java-sdk-dynamodb + com.amazonaws:aws-java-sdk-core + + com.github.davidmoten:guava-mini + com.github.davidmoten:hilbert-curve com.twitter:bijection-avro_${scala.binary.version} com.twitter:bijection-core_${scala.binary.version} io.confluent:kafka-avro-serializer @@ -122,6 +129,7 @@ org.apache.kafka:kafka-clients io.dropwizard.metrics:metrics-core io.dropwizard.metrics:metrics-graphite + io.dropwizard.metrics:metrics-jmx io.prometheus:simpleclient io.prometheus:simpleclient_httpserver io.prometheus:simpleclient_dropwizard diff --git a/pom.xml b/pom.xml index 888d8d32c6ed..a570c1f7149b 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ org.apache.hudi hudi pom - 0.10.0-SNAPSHOT + 0.11.0-SNAPSHOT Apache Hudi brings stream style processing on big data https://github.com/apache/hudi Hudi @@ -38,6 +38,7 @@ hudi-common hudi-cli hudi-client + hudi-aws hudi-hadoop-mr hudi-spark-datasource hudi-timeline-service @@ -49,6 +50,7 @@ packaging/hudi-presto-bundle packaging/hudi-utilities-bundle packaging/hudi-timeline-server-bundle + packaging/hudi-trino-bundle docker/hoodie/hadoop hudi-integ-test packaging/hudi-integ-test-bundle @@ -73,16 +75,17 @@ - 2.6 + 3.2.0 3.0.0-M4 3.0.0-M4 - 3.1.1 + 3.2.4 3.1.1 3.8.0 2.4 0.15 1.7 3.0.0-M1 + 0.37.0 1.8 2.6.7 @@ -91,9 +94,11 @@ 2.7.4 2.10.0 2.0.0 + 2.8.1 5.3.4 2.17 1.10.1 + 1.12.1 5.7.0-M1 5.7.0-M1 1.7.0-M1 @@ -114,7 +119,7 @@ ${spark2bundle.version} 1.13.1 2.4.4 - 3.1.2 + 3.2.0 3 hudi-spark2 @@ -140,24 +145,30 @@ ${skipTests} ${skipTests} ${skipTests} + ${skipTests} UTF-8 ${project.basedir} provided - + compile org.apache.hudi.spark. provided - + -Xmx2g 0.8.5 compile org.apache.hudi. + compile + org.apache.hudi. true 2.7.1 4.7 1.12.22 3.17.3 3.1.0 + 1.1.0 + 8000 + http://localhost:${dynamodb-local.port} @@ -231,6 +242,8 @@ ${project.build.sourceDirectory} + + basedir=${maven.multiModuleProjectDirectory} **\/generated-sources\/ @@ -304,19 +317,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - ${maven-surefire-plugin.version} - - @{argLine} - - - ${surefire-log4j.file} - - - - org.apache.maven.plugins maven-failsafe-plugin @@ -331,10 +331,32 @@ jacoco-maven-plugin ${jacoco.version} + + io.fabric8 + docker-maven-plugin + ${maven-docker-plugin.version} + + ${skipDocker} + + + + org.apache.maven.plugins + maven-surefire-plugin + ${maven-surefire-plugin.version} + + 3 + @{argLine} + + + ${surefire-log4j.file} + + + + org.apache.maven.plugins maven-jar-plugin @@ -822,6 +844,18 @@ org.pentaho * + + org.apache.logging.log4j + * + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + @@ -914,6 +948,10 @@ org.eclipse.jetty.aggregate * + + org.apache.logging.log4j + * + @@ -935,6 +973,54 @@ org.pentaho * + + org.apache.logging.log4j + * + + + + + org.apache.hive + hive-exec + ${hive.version} + provided + + + commons-lang + commons-lang + + + org.apache.commons + commons-lang3 + + + guava + com.google.guava + + + org.eclipse.jetty.aggregate + * + + + javax.mail + mail + + + org.apache.zookeeper + zookeeper + + + org.pentaho + * + + + com.esotericsoftware + kryo-shaded + + + org.apache.logging.log4j + * + @@ -1036,6 +1122,19 @@ test + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + org.apache.logging.log4j + * + + + + @@ -1147,6 +1246,7 @@ org.apache.maven.plugins maven-surefire-plugin + ${maven-surefire-plugin.version} org.junit.jupiter @@ -1201,6 +1301,7 @@ org.apache.maven.plugins maven-surefire-plugin + ${maven-surefire-plugin.version} org.apache.maven.surefire @@ -1253,6 +1354,7 @@ org.apache.maven.plugins maven-surefire-plugin + ${maven-surefire-plugin.version} ${skipUTs} @@ -1265,6 +1367,9 @@ **/IT*.java + + ${dynamodb-local.endpoint} + @@ -1474,6 +1579,7 @@ hudi-spark3 3.1.0 2.4.1 + ${parquet.spark3.version} ${fasterxml.spark3.version} ${fasterxml.spark3.version} ${fasterxml.spark3.version} @@ -1488,11 +1594,36 @@ + + spark3.1.x + + 3.1.2 + ${spark3.version} + ${spark3bundle.version} + ${scala12.version} + 2.12 + hudi-spark3 + 3.1.0 + 2.4.1 + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + true + true + + + + spark3.1.x + + + + spark3.0.x - 3.0.0 + 3.0.3 ${spark3.version} 3.0.1 diff --git a/rfc/README.md b/rfc/README.md index 5ef97300fcc3..e2fdfbc3d1e1 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -59,4 +59,11 @@ The list of all RFCs can be found here. | 33 | [Hudi supports more comprehensive Schema Evolution](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+33++Hudi+supports+more+comprehensive+Schema+Evolution) | `IN PROGRESS` | | 34 | [Hudi BigQuery Integration (WIP)](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=188745980) | `UNDER REVIEW` | | 35 | [Make Flink MOR table writing streaming friendly](https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly) | `UNDER REVIEW` | -| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | \ No newline at end of file +| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | +| 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | +| 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | +| 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `IN PROGRESS` | +| 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | +| 41 | [Hudi Snowflake Integration] | `UNDER REVIEW` | +| 42 | [Consistent Hashing Index](./rfc-42/rfc-42.md) | `UNDER REVIEW` | +| 43 | [Compaction / Clustering Service](./rfc-43/rfc-43.md) | `UNDER REVIEW` | diff --git a/rfc/rfc-39/arch.png b/rfc/rfc-39/arch.png new file mode 100644 index 000000000000..8864f439daf9 Binary files /dev/null and b/rfc/rfc-39/arch.png differ diff --git a/rfc/rfc-39/rfc-39.md b/rfc/rfc-39/rfc-39.md new file mode 100644 index 000000000000..9dc6335b2b9a --- /dev/null +++ b/rfc/rfc-39/rfc-39.md @@ -0,0 +1,105 @@ + +# RFC-[39]: Deltastreamer avro-based source for Debezium CDC + + + +## Proposers +- @rmahindra +- @vbalaji + +## Approvers + - @vinoth + +## Status + +JIRA: [https://issues.apache.org/jira/browse/HUDI-1290](https://issues.apache.org/jira/browse/HUDI-1290) + +## Abstract + +We intend to implement a source for ingesting Debezium Change Data Capture (CDC) logs into Deltastreamer/ Hudi. With this capability, we can continuously capture row-level changes that insert, update and delete records that were committed to a database. While debezium support multiple databases, we will focus on postgres for the RFC. At the end, we will explain how it can be extended to support Mysql. + +## Background +The architecture of Debezium is shown in figure below. [Debezium](https://debezium.io/documentation/reference/stable/connectors/postgresql.html) is implemented as a Kafka connect source, that reads change logs from databases ([logical decoding](https://www.postgresql.org/docs/current/logicaldecoding-explanation.html) in PostgreSQL and `binlog` in MySQL) and ingests them into a kafka topic. Debezium uses a single kafka topic per table in the source database. + + + +The first time it connects to a PostgreSQL server or cluster, the connector takes a consistent snapshot of all schemas. After that snapshot is complete, the connector continuously captures row-level changes that insert, update, and deletes. The connector generates data change event records and streams them to Kafka topics. For each table, the default behavior is that the connector streams all generated events to a separate Kafka topic for that table. Applications and services consume data change event records from that topic. In addition, Debezium registers the schema of the change events in kafka to a schema registry, such as Confluent schema registry. + + + +The schema of the events for debezium consists of a before, after, source, op and ts\_ms. The `before` field contains the values of the row before the operation took place. And `after` field contains the values of the original database row after the operation took place. The operation is specified in `op` field, which can be either `r` (initial snapshot), `c` (insert), `u` (update) or `d` (delete). In case of insert, the `before` field will be null which for a delete, the `after` field will be null. In the case of update, the `before` field will be the values of the columns in a row before the update was applied, and `after` will contain the values after the update was applied. The `source` field contains a list of key metadata fields. For instance, debezium version, database name, database schema name etc. In the case of PostgresSQL, an important field is `LSN` that represents the log sequence number of the change log, and determines the relative position of the change log. + + + +There are other ways to deploy Debezium, such as Debezium Server, that can write events to other stream systems, such as pulsar, kenisis, google pub/sub etc. However, this RFC focuses on the debezium source in deltastreamer that will assume Kafka as the source for the change log events. + +![](arch.png) + +## Implementation + +As shown in the figure above, in order to ingest the rows from the database into hudi and maintain the change operations done on database in hudi, we need to perform 2 steps: (I) We have to bootstrap the initial data from the database and (ii) incrementally consume the change logs to insert or update the records into hudi + + + +To bootstrap the initial rows from the database, we can either do a full fetch directly from the database using JDBC, or alike and then incrementally pull the change logs from the appropriate checkpoint. The other option is to let the debezium connector perform an initial _consistent snapshot_ of the database up to a specific checkpoint. Subsequently, the debezium publishes change logs over the initial snapshot, that can be read incrementally by the deltastreamer. + + + +To incrementally ingest the changelogs from the debezium connector, we propose to implement a few classes.`DebeziumAvroSource.java` implements the source class that reads the kafka change log events. We reuse `KafkaOffsetGen.java` that helps reading events from Kafka incrementally. The `DebeziumAvroSource.java` pulls the latest schema from the schema registry, applies the schema to a batch of incoming avro records of the change logs, and transforms the records to extract the actual fields of the rows in the database. In case of insert or update records (identified by the `op` field), the field values are picked from the `after` field in the incoming debezium record. In case of delete records (identified by the `op` field), the values are picked from the `before` field since `after` field is null. In addition, we also add the meta fields from both database and debezium. Meta fields such as `LSN` for Postgres DB help us identify the order of the events. + +Since we change the schema of the incoming record in the source class, we have to provide a schema for the target record. We propose to implement DebeziumAvroSource.java as a RowSource and allow spark to infer the schema of the transformed record. An alternative approach is to implement a DebeziumSchemaRegistryProvider.java class that extends the current SchemaRegistryProvider.java, and implements the method getTargetSchema . It constructs the target schema from the original schema by including only the fields nested within the after field of the original record, along with the meta fields that were actually ingested. + + +To ensure proper de-dup, merging, and hard deletes of the records, we implement a custom AvroPayload class for debeizum: `DebeziumAvroPayload.java.` During writes, we check if the `op` field of the record is `d` , we return an empty payload to ensure the record is deleted in storage. In the case of `preCombine` or `combineAndGetUpdateValue` (merge handling of records), we return the existing stored record if the `LSN` (in case of PostgresSQL) of the existing record is higher than the newly inserted record. Else, the new record is written. In the case of MySQL DB, we will either use the `GTID` field or a combination of `bin file id` and `bin pos` to identify the relative ordering of the events when they actually happened in the database. + +###Handling merges with Postgres Toast Columns + +[TOAST](https://www.postgresql.org/docs/current/storage-toast.html) (The Oversized-Attribute Storage Technique) is a mechanism in Postgres which stores large column values in multiple physical rows, circumventing the page size limit of 8 KB. + + + +Typically, TOAST storage is transparent to the user. There’s an exception, though: if a table row has changed, any _unchanged_ values that were stored using the TOAST mechanism are not included in the message that Debezium receives from the database, unless they are part of the table’s [replica identity](https://debezium.io/documentation/reference/0.10/connectors/postgresql.html#replica-identity). Consequently, such unchanged TOAST column value will not be contained in Debezium data change events sent to Kafka. Instead of the actual value, Debezium uses a placeholder `__debezium_unavailable_value` for representing toast columns that have not changed. + + + +During merging, we check for toast columns in the insert records, and if present, we update their value using the values from the current record on disk. + + + + +### Deltastreamer configuration + +To run the deltastreamer, we need to configure the following: + +1. The `source ordering field` should be set to `_source_lsn.` +2. Configure the schema registry server that is used by the Debezium connector. +3. Record Key(s) should be the primary key(s) of the database and can be obtained from the schema registry, since debezium uses the primary key(s) as the key for the kafka topic. +4. Configure the deltastreamer to use the DebeziumSource and DebeziumAvroPayload classes for the source and payload classes respectively. + +### Current Limitations + +With the current constraints within Hudi, we discuss a few limitations of the current implementation for CDC. Consider a case where we have the following change log events for a single row/ record in the following order from kafka: Insert (LSN=1), Delete (LSN=3), Updated (LSN=2). If all these events are ingested in the same batch, then dedup will only pick the second event, since it has the highest LSN. However, if the second and third event are ingested in different batches, then the second event would have deleted the hudi record from the disk. When we receive the third event, we would insert it since the record has been previously deleted. This limitation holds for both CoW and MoR tables. To resolve this limitation, we have to keep the ordering value of each record even after deletion, to ensure we can apply an insert only if the insert event has a ordering value higher than the delete event. This limitation only happens in the case of an out of ordered delete event. Out-of-ordered insert and update events should be applied correctly. + + +## Rollout/Adoption Plan + +This is a new feature specific to Debezium CDC use case, and should not impact existing jobs or tables. + +## Test Plan + +We plan to test the Debezium source by setting up a AWS RDS instance of PostgresSQL, debezium connector using strimzi operator on k8s and a AWS MSK kafka cluster. We will test for correctness by performing SQL based DDL operations, such as insert, update and deletions on multiple records/ rows in the Postgres DB, and query the hudi table to validate that the operations took effect on the records in the hudi table. \ No newline at end of file diff --git a/rfc/rfc-41/rfc-41.md b/rfc/rfc-41/rfc-41.md new file mode 100644 index 000000000000..c9517f5385da --- /dev/null +++ b/rfc/rfc-41/rfc-41.md @@ -0,0 +1,82 @@ + +# RFC-40: Hudi Snowflake Integration + + + +## Proposers + +- @vingov + +## Approvers + - @vinothchandar + - @bhasudha + +## Status + +JIRA: [HUDI-2832](https://issues.apache.org/jira/browse/HUDI-2832) + +> Please keep the status updated in `rfc/README.md`. + +# Hudi Snowflake Integration + +## Abstract + +Snowflake is a fully managed service that's simple to use but can power a near-unlimited number of concurrent workloads. Snowflake is a solution for data warehousing, data lakes, data engineering, data science, data application development, and securely sharing and consuming shared data. Snowflake [doesn't support](https://docs.snowflake.com/en/sql-reference/sql/alter-file-format.html) Apache Hudi file format yet, but it has support for Parquet, ORC and Delta file format. This proposal is to implement a SnowflakeSync similar to HiveSync to sync the Hudi table as the Snowflake External Parquet table, so that users can query the Hudi tables using Snowflake. Many users have expressed interest in Hudi and other support channels asking to integrate Hudi with Snowflake, this will unlock new use cases for Hudi. + +## Background + +Hudi table types define how data is indexed & laid out on the DFS and how the above primitives and timeline activities are implemented on top of such organization (i.e how data is written). In turn, query types define how the underlying data is exposed to the queries (i.e how data is read). + +Hudi supports the following table types: + +- [Copy On Write](https://hudi.apache.org/docs/overview/#copy-on-write-table): Stores data using exclusively columnar file formats (e.g parquet). Updates simply version & rewrite the files by performing a synchronous merge during write. +- [Merge On Read](https://hudi.apache.org/docs/overview/#merge-on-read-table): Stores data using a combination of columnar (e.g parquet) + row based (e.g avro) file formats. Updates are logged to delta files & later compacted to produce new versions of columnar files synchronously or asynchronously. + +Hudi maintains multiple versions of the Parquet files and tracks the latest version using Hudi metadata (Cow), since Snowflake doesn't support Hudi yet, when you sync the Hudi's parquet files to Snowflake and query it without Hudi's metadata layer, it will query all the versions of the parquet files which might cause duplicate rows. + +To avoid the above scenario, this proposal is to implement a Snowflake sync tool which will use the Hudi metadata to know which files are latest and sync only the latest version of parquet files to Snowflake external table so that users can query the Hudi tables without any duplicate records. + +## Implementation + +A Hudi table can be read by Snowflake using a manifest file, which is a text file containing the list of data files to read for querying a Hudi table. This proposal describes how to set up a Snowflake to Hudi integration using manifest files and query Hudi tables. + +This new feature will implement the [AbstractSyncTool](https://github.com/apache/hudi/blob/master/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java) similar to the [HiveSyncTool](https://github.com/apache/hudi/blob/master/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java) named SnowflakeSyncTool with sync methods for CoW tables. The sync implementation will identify the latest parquet files for each .commit file and keep these manifests synced with the Snowflake external table. Spark datasource & DeltaStreamer can already take a list of such classes to keep these manifests synced. + +### Steps to Snowflake Sync + +1. Generate manifests of a Hudi table using Spark Runtime +2. Configure Snowflake to read the generated manifests +3. Update manifests + 1. Update explicitly: After all the data updates, you can run the generate operation to update the manifests. + 2. Update automatically: You can configure a Hudi table so that all write operations on the table automatically update the manifests. To enable this automatic mode, set the corresponding hoodie property. + +## Rollout/Adoption Plan + +There are no impacts to existing users since this is entirely a new feature to support a new use case hence there are no migrations/behavior changes required. + +After the Snowflake sync tool has been implemented, I will reach out to beta customers who have already expressed interest to roll out this feature for their Snowflake ingestion service. + +## Test Plan + +This RFC aims to implement a new SyncTool to sync the Hudi table to Snowflake, to test this feature, there will be some test tables created and updated on to the Snowflake along with unit tests for the code. Since this is an entirely new feature, I am confident that this will not cause any regressions during and after roll out. + +## Future Plan + +After this feature has been rolled out, the same model can be applied to sync the Hudi tables to other external data warehouses like BigQuery. + +After CoW rollout and based on adoption, we can explore supporting MoR table format. diff --git a/scripts/release/create_source_release.sh b/scripts/release/create_source_release.sh index 05aa4b973a59..ff54706be237 100755 --- a/scripts/release/create_source_release.sh +++ b/scripts/release/create_source_release.sh @@ -71,6 +71,7 @@ rsync -a \ --exclude ".github" --exclude "target" \ --exclude ".idea" --exclude "*.iml" --exclude ".DS_Store" --exclude "build-target" \ --exclude "docs/content" --exclude ".rubydeps" \ + --exclude "rfc" \ . hudi-$RELEASE_VERSION tar czf ${RELEASE_DIR}/hudi-${RELEASE_VERSION}.src.tgz hudi-$RELEASE_VERSION diff --git a/scripts/release/validate_staged_release.sh b/scripts/release/validate_staged_release.sh index db6a135a8c2a..681cc2018637 100755 --- a/scripts/release/validate_staged_release.sh +++ b/scripts/release/validate_staged_release.sh @@ -156,10 +156,10 @@ echo -e "\t\tNotice file exists ? [OK]\n" ### Licensing Check echo "Performing custom Licensing Check " -numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.data'| grep -v '.commit' | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'ObjectSizeCalculator.java' | grep -v 'AvroConversionHelper.scala' | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" | wc -l` +numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.data'| grep -v '.commit' | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'ObjectSizeCalculator.java' | grep -v 'AvroConversionHelper.scala' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" | wc -l` if [ "$numfilesWithNoLicense" -gt "0" ]; then echo "There were some source files that did not have Apache License" - find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'ObjectSizeCalculator.java' | grep -v 'AvroConversionHelper.scala' | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" + find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'ObjectSizeCalculator.java' | grep -v 'AvroConversionHelper.scala' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" exit -1 fi echo -e "\t\tLicensing Check Passed [OK]\n" diff --git a/style/checkstyle.xml b/style/checkstyle.xml index 7dbce7973bfd..82325c9535b5 100644 --- a/style/checkstyle.xml +++ b/style/checkstyle.xml @@ -275,7 +275,9 @@ - + + + @@ -305,7 +307,7 @@ - + diff --git a/style/scalastyle.xml b/style/scalastyle.xml index 89306f36e1c3..2ba4042be0ca 100644 --- a/style/scalastyle.xml +++ b/style/scalastyle.xml @@ -27,7 +27,7 @@ - +