[HUDI-875] Abstract hudi-sync-common, and support hudi-hive-sync, hudi-dla-sync (#1810)
- Generalize the hive-sync module for syncing to multiple metastores - Added new options for datasource - Added new command line for delta streamer Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -21,4 +21,4 @@ cp /var/hoodie/ws/docker/demo/config/log4j.properties $SPARK_CONF_DIR/.
|
|||||||
hadoop fs -mkdir -p /var/demo/
|
hadoop fs -mkdir -p /var/demo/
|
||||||
hadoop fs -mkdir -p /tmp/spark-events
|
hadoop fs -mkdir -p /tmp/spark-events
|
||||||
hadoop fs -copyFromLocal -f /var/hoodie/ws/docker/demo/config /var/demo/.
|
hadoop fs -copyFromLocal -f /var/hoodie/ws/docker/demo/config /var/demo/.
|
||||||
chmod +x /var/hoodie/ws/hudi-hive-sync/run_sync_tool.sh
|
chmod +x /var/hoodie/ws/hudi-sync/hudi-hive-sync/run_sync_tool.sh
|
||||||
|
|||||||
@@ -186,6 +186,11 @@
|
|||||||
<artifactId>hudi-hive-sync</artifactId>
|
<artifactId>hudi-hive-sync</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-sync-common</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<!-- Logging -->
|
<!-- Logging -->
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|||||||
@@ -19,6 +19,7 @@ package org.apache.hudi
|
|||||||
|
|
||||||
import org.apache.hudi.common.model.HoodieTableType
|
import org.apache.hudi.common.model.HoodieTableType
|
||||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
|
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
|
||||||
|
import org.apache.hudi.hive.HiveSyncTool
|
||||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
|
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
|
||||||
import org.apache.hudi.keygen.SimpleKeyGenerator
|
import org.apache.hudi.keygen.SimpleKeyGenerator
|
||||||
import org.apache.log4j.LogManager
|
import org.apache.log4j.LogManager
|
||||||
@@ -251,11 +252,14 @@ object DataSourceWriteOptions {
|
|||||||
*/
|
*/
|
||||||
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch"
|
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch"
|
||||||
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true"
|
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true"
|
||||||
|
val META_SYNC_CLIENT_TOOL_CLASS = "hoodie.meta.sync.client.tool.class"
|
||||||
|
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = classOf[HiveSyncTool].getName
|
||||||
|
|
||||||
// HIVE SYNC SPECIFIC CONFIGS
|
// HIVE SYNC SPECIFIC CONFIGS
|
||||||
//NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
|
//NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
|
||||||
// unexpected issues with config getting reset
|
// unexpected issues with config getting reset
|
||||||
val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable"
|
val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable"
|
||||||
|
val META_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.meta.sync.enable"
|
||||||
val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
|
val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
|
||||||
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
|
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
|
||||||
val HIVE_BASE_FILE_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.base_file_format"
|
val HIVE_BASE_FILE_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.base_file_format"
|
||||||
@@ -270,6 +274,7 @@ object DataSourceWriteOptions {
|
|||||||
|
|
||||||
// DEFAULT FOR HIVE SPECIFIC CONFIGS
|
// DEFAULT FOR HIVE SPECIFIC CONFIGS
|
||||||
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
|
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
|
||||||
|
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = "false"
|
||||||
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
|
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
|
||||||
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
|
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
|
||||||
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = "PARQUET"
|
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = "PARQUET"
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
package org.apache.hudi
|
package org.apache.hudi
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
|
import java.util.Properties
|
||||||
|
|
||||||
import org.apache.avro.Schema
|
import org.apache.avro.Schema
|
||||||
import org.apache.avro.generic.GenericRecord
|
import org.apache.avro.generic.GenericRecord
|
||||||
@@ -32,9 +33,11 @@ import org.apache.hudi.common.fs.FSUtils
|
|||||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
|
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
|
||||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||||
|
import org.apache.hudi.common.util.ReflectionUtils
|
||||||
import org.apache.hudi.config.HoodieWriteConfig
|
import org.apache.hudi.config.HoodieWriteConfig
|
||||||
import org.apache.hudi.exception.HoodieException
|
import org.apache.hudi.exception.HoodieException
|
||||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||||
import org.apache.log4j.LogManager
|
import org.apache.log4j.LogManager
|
||||||
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
|
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
|
||||||
import org.apache.spark.rdd.RDD
|
import org.apache.spark.rdd.RDD
|
||||||
@@ -240,7 +243,10 @@ private[hudi] object HoodieSparkSqlWriter {
|
|||||||
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
|
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
|
||||||
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
|
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
|
||||||
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
|
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
|
||||||
|
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
|
||||||
|
//just for backwards compatiblity
|
||||||
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
|
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
|
||||||
|
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
|
||||||
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
|
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
|
||||||
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
|
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
|
||||||
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
|
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
|
||||||
@@ -287,6 +293,44 @@ private[hudi] object HoodieSparkSqlWriter {
|
|||||||
hiveSyncConfig
|
hiveSyncConfig
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def metaSync(parameters: Map[String, String],
|
||||||
|
basePath: Path,
|
||||||
|
hadoopConf: Configuration): Boolean = {
|
||||||
|
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||||
|
var metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||||
|
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||||
|
parameters(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||||
|
|
||||||
|
// for backward compatibility
|
||||||
|
if (hiveSyncEnabled) {
|
||||||
|
metaSyncEnabled = true
|
||||||
|
syncClientToolClassSet += classOf[HiveSyncTool].getName
|
||||||
|
}
|
||||||
|
var metaSyncSuccess = true
|
||||||
|
if (metaSyncEnabled) {
|
||||||
|
val fs = basePath.getFileSystem(hadoopConf)
|
||||||
|
syncClientToolClassSet.foreach(impl => {
|
||||||
|
val syncSuccess = impl.trim match {
|
||||||
|
case "org.apache.hudi.hive.HiveSyncTool" => {
|
||||||
|
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
|
||||||
|
syncHive(basePath, fs, parameters)
|
||||||
|
true
|
||||||
|
}
|
||||||
|
case _ => {
|
||||||
|
val properties = new Properties();
|
||||||
|
properties.putAll(parameters)
|
||||||
|
properties.put("basePath", basePath.toString)
|
||||||
|
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
|
||||||
|
syncHoodie.syncHoodieTable()
|
||||||
|
true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
metaSyncSuccess = metaSyncSuccess && syncSuccess
|
||||||
|
})
|
||||||
|
}
|
||||||
|
metaSyncSuccess
|
||||||
|
}
|
||||||
|
|
||||||
private def commitAndPerformPostOperations(writeStatuses: JavaRDD[WriteStatus],
|
private def commitAndPerformPostOperations(writeStatuses: JavaRDD[WriteStatus],
|
||||||
parameters: Map[String, String],
|
parameters: Map[String, String],
|
||||||
client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
|
client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
|
||||||
@@ -323,20 +367,13 @@ private[hudi] object HoodieSparkSqlWriter {
|
|||||||
}
|
}
|
||||||
|
|
||||||
log.info(s"Compaction Scheduled is $compactionInstant")
|
log.info(s"Compaction Scheduled is $compactionInstant")
|
||||||
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration())
|
||||||
val syncHiveSucess = if (hiveSyncEnabled) {
|
|
||||||
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
|
|
||||||
val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration)
|
|
||||||
syncHive(basePath, fs, parameters)
|
|
||||||
} else {
|
|
||||||
true
|
|
||||||
}
|
|
||||||
|
|
||||||
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
|
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
|
||||||
if (!asyncCompactionEnabled) {
|
if (!asyncCompactionEnabled) {
|
||||||
client.close()
|
client.close()
|
||||||
}
|
}
|
||||||
(commitSuccess && syncHiveSucess, compactionInstant)
|
(commitSuccess && metaSyncSuccess, compactionInstant)
|
||||||
} else {
|
} else {
|
||||||
log.error(s"$operation failed with $errorCount errors :")
|
log.error(s"$operation failed with $errorCount errors :")
|
||||||
if (log.isTraceEnabled) {
|
if (log.isTraceEnabled) {
|
||||||
|
|||||||
171
hudi-sync/hudi-dla-sync/pom.xml
Normal file
171
hudi-sync/hudi-dla-sync/pom.xml
Normal file
@@ -0,0 +1,171 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
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.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<parent>
|
||||||
|
<artifactId>hudi</artifactId>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<version>0.6.0-SNAPSHOT</version>
|
||||||
|
<relativePath>../../pom.xml</relativePath>
|
||||||
|
</parent>
|
||||||
|
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<artifactId>hudi-dla-sync</artifactId>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
|
<properties>
|
||||||
|
<hadoop.aliyun.version>3.2.1</hadoop.aliyun.version>
|
||||||
|
<mysql.connector.java.version>5.1.47</mysql.connector.java.version>
|
||||||
|
<aliyun.sdk.oss.version>3.1.0</aliyun.sdk.oss.version>
|
||||||
|
<aliyun.java.sdk.core.version>3.7.1</aliyun.java.sdk.core.version>
|
||||||
|
</properties>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
<!-- Hoodie -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-common</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-sync-common</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-hive-sync</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
<exclusions>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.apache.hadoop</groupId>
|
||||||
|
<artifactId>hadoop-hdfs</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.apache.hive</groupId>
|
||||||
|
<artifactId>hive-shims</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>commons-lang</groupId>
|
||||||
|
<artifactId>commons-lang</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.apache.zookeeper</groupId>
|
||||||
|
<artifactId>zookeeper</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
</exclusions>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>mysql</groupId>
|
||||||
|
<artifactId>mysql-connector-java</artifactId>
|
||||||
|
<version>${mysql.connector.java.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hadoop</groupId>
|
||||||
|
<artifactId>hadoop-aliyun</artifactId>
|
||||||
|
<version>${hadoop.aliyun.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.aliyun.oss</groupId>
|
||||||
|
<artifactId>aliyun-sdk-oss</artifactId>
|
||||||
|
<version>${aliyun.sdk.oss.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.aliyun</groupId>
|
||||||
|
<artifactId>aliyun-java-sdk-core</artifactId>
|
||||||
|
<version>${aliyun.java.sdk.core.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<!-- Logging -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>log4j</groupId>
|
||||||
|
<artifactId>log4j</artifactId>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.parquet</groupId>
|
||||||
|
<artifactId>parquet-avro</artifactId>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<!-- Hadoop -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hadoop</groupId>
|
||||||
|
<artifactId>hadoop-common</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hive</groupId>
|
||||||
|
<artifactId>hive-exec</artifactId>
|
||||||
|
<version>${hive.version}</version>
|
||||||
|
<exclusions>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>commons-lang</groupId>
|
||||||
|
<artifactId>commons-lang</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.apache.commons</groupId>
|
||||||
|
<artifactId>commons-lang3</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.apache.zookeeper</groupId>
|
||||||
|
<artifactId>zookeeper</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
</exclusions>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.junit.jupiter</groupId>
|
||||||
|
<artifactId>junit-jupiter-api</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<resources>
|
||||||
|
<resource>
|
||||||
|
<directory>src/main/resources</directory>
|
||||||
|
</resource>
|
||||||
|
</resources>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.rat</groupId>
|
||||||
|
<artifactId>apache-rat-plugin</artifactId>
|
||||||
|
</plugin>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
|
<artifactId>maven-jar-plugin</artifactId>
|
||||||
|
<version>${maven-jar-plugin.version}</version>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<goals>
|
||||||
|
<goal>test-jar</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.jacoco</groupId>
|
||||||
|
<artifactId>jacoco-maven-plugin</artifactId>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</build>
|
||||||
|
</project>
|
||||||
@@ -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.dla;
|
||||||
|
|
||||||
|
import com.beust.jcommander.Parameter;
|
||||||
|
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configs needed to sync data into DLA.
|
||||||
|
*/
|
||||||
|
public class DLASyncConfig implements Serializable {
|
||||||
|
|
||||||
|
@Parameter(names = {"--database"}, description = "name of the target database in DLA", required = true)
|
||||||
|
public String databaseName;
|
||||||
|
|
||||||
|
@Parameter(names = {"--table"}, description = "name of the target table in DLA", required = true)
|
||||||
|
public String tableName;
|
||||||
|
|
||||||
|
@Parameter(names = {"--user"}, description = "DLA username", required = true)
|
||||||
|
public String dlaUser;
|
||||||
|
|
||||||
|
@Parameter(names = {"--pass"}, description = "DLA password", required = true)
|
||||||
|
public String dlaPass;
|
||||||
|
|
||||||
|
@Parameter(names = {"--jdbc-url"}, description = "DLA jdbc connect url", required = true)
|
||||||
|
public String jdbcUrl;
|
||||||
|
|
||||||
|
@Parameter(names = {"--base-path"}, description = "Basepath of hoodie table to sync", required = true)
|
||||||
|
public String basePath;
|
||||||
|
|
||||||
|
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by")
|
||||||
|
public List<String> partitionFields = new ArrayList<>();
|
||||||
|
|
||||||
|
@Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor "
|
||||||
|
+ "to extract the partition values from HDFS path")
|
||||||
|
public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class.getName();
|
||||||
|
|
||||||
|
@Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"
|
||||||
|
+ " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter")
|
||||||
|
public Boolean assumeDatePartitioning = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering")
|
||||||
|
public Boolean skipROSuffix = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--hive-style-partitioning"}, description = "Use DLA hive style partitioning, true if like the following style: field1=value1/field2=value2")
|
||||||
|
public Boolean useDLASyncHiveStylePartitioning = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--help", "-h"}, help = true)
|
||||||
|
public Boolean help = false;
|
||||||
|
|
||||||
|
public static DLASyncConfig copy(DLASyncConfig cfg) {
|
||||||
|
DLASyncConfig newConfig = new DLASyncConfig();
|
||||||
|
newConfig.databaseName = cfg.databaseName;
|
||||||
|
newConfig.tableName = cfg.tableName;
|
||||||
|
newConfig.dlaUser = cfg.dlaUser;
|
||||||
|
newConfig.dlaPass = cfg.dlaPass;
|
||||||
|
newConfig.jdbcUrl = cfg.jdbcUrl;
|
||||||
|
newConfig.basePath = cfg.basePath;
|
||||||
|
newConfig.partitionFields = cfg.partitionFields;
|
||||||
|
newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass;
|
||||||
|
newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning;
|
||||||
|
newConfig.skipROSuffix = cfg.skipROSuffix;
|
||||||
|
newConfig.useDLASyncHiveStylePartitioning = cfg.useDLASyncHiveStylePartitioning;
|
||||||
|
return newConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "DLASyncConfig{databaseName='" + databaseName + '\'' + ", tableName='" + tableName + '\''
|
||||||
|
+ ", dlaUser='" + dlaUser + '\'' + ", dlaPass='" + dlaPass + '\'' + ", jdbcUrl='" + jdbcUrl + '\''
|
||||||
|
+ ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='"
|
||||||
|
+ partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning
|
||||||
|
+ ", useDLASyncHiveStylePartitioning=" + useDLASyncHiveStylePartitioning
|
||||||
|
+ ", help=" + help + '}';
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,212 @@
|
|||||||
|
/*
|
||||||
|
* 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.dla;
|
||||||
|
|
||||||
|
import com.beust.jcommander.JCommander;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||||
|
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.dla.util.Utils;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
import org.apache.hudi.exception.InvalidTableException;
|
||||||
|
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||||
|
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||||
|
import org.apache.hudi.hive.SchemaDifference;
|
||||||
|
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.parquet.schema.MessageType;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tool to sync a hoodie table with a dla table. Either use it as a api
|
||||||
|
* DLASyncTool.syncHoodieTable(DLASyncConfig) or as a command line java -cp hoodie-hive.jar DLASyncTool [args]
|
||||||
|
* <p>
|
||||||
|
* This utility will get the schema from the latest commit and will sync dla table schema Also this will sync the
|
||||||
|
* partitions incrementally (all the partitions modified since the last commit)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("WeakerAccess")
|
||||||
|
public class DLASyncTool extends AbstractSyncTool {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(DLASyncTool.class);
|
||||||
|
public static final String SUFFIX_SNAPSHOT_TABLE = "_rt";
|
||||||
|
public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro";
|
||||||
|
|
||||||
|
private final DLASyncConfig cfg;
|
||||||
|
private final HoodieDLAClient hoodieDLAClient;
|
||||||
|
private final String snapshotTableName;
|
||||||
|
private final Option<String> roTableTableName;
|
||||||
|
|
||||||
|
public DLASyncTool(Properties properties, FileSystem fs) {
|
||||||
|
super(properties, fs);
|
||||||
|
this.hoodieDLAClient = new HoodieDLAClient(Utils.propertiesToConfig(properties), fs);
|
||||||
|
this.cfg = Utils.propertiesToConfig(properties);
|
||||||
|
switch (hoodieDLAClient.getTableType()) {
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
this.snapshotTableName = cfg.tableName;
|
||||||
|
this.roTableTableName = Option.empty();
|
||||||
|
break;
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE;
|
||||||
|
this.roTableTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) :
|
||||||
|
Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
LOG.error("Unknown table type " + hoodieDLAClient.getTableType());
|
||||||
|
throw new InvalidTableException(hoodieDLAClient.getBasePath());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void syncHoodieTable() {
|
||||||
|
try {
|
||||||
|
switch (hoodieDLAClient.getTableType()) {
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
syncHoodieTable(snapshotTableName, false);
|
||||||
|
break;
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
// sync a RO table for MOR
|
||||||
|
syncHoodieTable(roTableTableName.get(), false);
|
||||||
|
// sync a RT table for MOR
|
||||||
|
syncHoodieTable(snapshotTableName, true);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
LOG.error("Unknown table type " + hoodieDLAClient.getTableType());
|
||||||
|
throw new InvalidTableException(hoodieDLAClient.getBasePath());
|
||||||
|
}
|
||||||
|
} catch (RuntimeException re) {
|
||||||
|
LOG.error("Got runtime exception when dla syncing", re);
|
||||||
|
} finally {
|
||||||
|
hoodieDLAClient.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat) {
|
||||||
|
LOG.info("Trying to sync hoodie table " + tableName + " with base path " + hoodieDLAClient.getBasePath()
|
||||||
|
+ " of type " + hoodieDLAClient.getTableType());
|
||||||
|
// Check if the necessary table exists
|
||||||
|
boolean tableExists = hoodieDLAClient.doesTableExist(tableName);
|
||||||
|
// Get the parquet schema for this table looking at the latest commit
|
||||||
|
MessageType schema = hoodieDLAClient.getDataSchema();
|
||||||
|
// Sync schema if needed
|
||||||
|
syncSchema(tableName, tableExists, useRealtimeInputFormat, schema);
|
||||||
|
|
||||||
|
LOG.info("Schema sync complete. Syncing partitions for " + tableName);
|
||||||
|
// Get the last time we successfully synced partitions
|
||||||
|
// TODO : once DLA supports alter table properties
|
||||||
|
Option<String> lastCommitTimeSynced = Option.empty();
|
||||||
|
/*if (tableExists) {
|
||||||
|
lastCommitTimeSynced = hoodieDLAClient.getLastCommitTimeSynced(tableName);
|
||||||
|
}*/
|
||||||
|
LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null"));
|
||||||
|
List<String> writtenPartitionsSince = hoodieDLAClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
|
||||||
|
LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size());
|
||||||
|
// Sync the partitions if needed
|
||||||
|
syncPartitions(tableName, writtenPartitionsSince);
|
||||||
|
|
||||||
|
hoodieDLAClient.updateLastCommitTimeSynced(tableName);
|
||||||
|
LOG.info("Sync complete for " + tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the latest schema from the last commit and check if its in sync with the dla table schema. If not, evolves the
|
||||||
|
* table schema.
|
||||||
|
*
|
||||||
|
* @param tableExists - does table exist
|
||||||
|
* @param schema - extracted schema
|
||||||
|
*/
|
||||||
|
private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, MessageType schema) {
|
||||||
|
// Check and sync schema
|
||||||
|
if (!tableExists) {
|
||||||
|
LOG.info("DLA table " + tableName + " is not found. Creating it");
|
||||||
|
if (!useRealTimeInputFormat) {
|
||||||
|
String inputFormatClassName = HoodieParquetInputFormat.class.getName();
|
||||||
|
hoodieDLAClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(),
|
||||||
|
ParquetHiveSerDe.class.getName());
|
||||||
|
} else {
|
||||||
|
// Custom serde will not work with ALTER TABLE REPLACE COLUMNS
|
||||||
|
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive
|
||||||
|
// /ql/exec/DDLTask.java#L3488
|
||||||
|
String inputFormatClassName = HoodieParquetRealtimeInputFormat.class.getName();
|
||||||
|
hoodieDLAClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(),
|
||||||
|
ParquetHiveSerDe.class.getName());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Check if the table schema has evolved
|
||||||
|
Map<String, String> tableSchema = hoodieDLAClient.getTableSchema(tableName);
|
||||||
|
SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
|
||||||
|
if (!schemaDiff.isEmpty()) {
|
||||||
|
LOG.info("Schema difference found for " + tableName);
|
||||||
|
hoodieDLAClient.updateTableDefinition(tableName, schemaDiff);
|
||||||
|
} else {
|
||||||
|
LOG.info("No Schema difference for " + tableName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Syncs the list of storage parititions passed in (checks if the partition is in dla, if not adds it or if the
|
||||||
|
* partition path does not match, it updates the partition path).
|
||||||
|
*/
|
||||||
|
private void syncPartitions(String tableName, List<String> writtenPartitionsSince) {
|
||||||
|
try {
|
||||||
|
if (cfg.partitionFields.isEmpty()) {
|
||||||
|
LOG.info("not a partitioned table.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
Map<List<String>, String> partitions = hoodieDLAClient.scanTablePartitions(tableName);
|
||||||
|
List<AbstractSyncHoodieClient.PartitionEvent> partitionEvents =
|
||||||
|
hoodieDLAClient.getPartitionEvents(partitions, writtenPartitionsSince);
|
||||||
|
List<String> newPartitions = filterPartitions(partitionEvents, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType.ADD);
|
||||||
|
LOG.info("New Partitions " + newPartitions);
|
||||||
|
hoodieDLAClient.addPartitionsToTable(tableName, newPartitions);
|
||||||
|
List<String> updatePartitions = filterPartitions(partitionEvents, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType.UPDATE);
|
||||||
|
LOG.info("Changed Partitions " + updatePartitions);
|
||||||
|
hoodieDLAClient.updatePartitionsToTable(tableName, updatePartitions);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieException("Failed to sync partitions for table " + tableName, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> filterPartitions(List<AbstractSyncHoodieClient.PartitionEvent> events, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType eventType) {
|
||||||
|
return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) {
|
||||||
|
// parse the params
|
||||||
|
final DLASyncConfig cfg = new DLASyncConfig();
|
||||||
|
JCommander cmd = new JCommander(cfg, null, args);
|
||||||
|
if (cfg.help || args.length == 0) {
|
||||||
|
cmd.usage();
|
||||||
|
System.exit(1);
|
||||||
|
}
|
||||||
|
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
|
||||||
|
new DLASyncTool(Utils.configToProperties(cfg), fs).syncHoodieTable();
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,403 @@
|
|||||||
|
/*
|
||||||
|
* 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.dla;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
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.HoodieException;
|
||||||
|
import org.apache.hudi.hive.HiveSyncConfig;
|
||||||
|
import org.apache.hudi.hive.HoodieHiveSyncException;
|
||||||
|
import org.apache.hudi.hive.PartitionValueExtractor;
|
||||||
|
import org.apache.hudi.hive.SchemaDifference;
|
||||||
|
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.parquet.schema.MessageType;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.DriverManager;
|
||||||
|
import java.sql.DatabaseMetaData;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class HoodieDLAClient extends AbstractSyncHoodieClient {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(HoodieDLAClient.class);
|
||||||
|
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "hoodie_last_sync";
|
||||||
|
// Make sure we have the dla JDBC driver in classpath
|
||||||
|
private static final String DRIVER_NAME = "com.mysql.jdbc.Driver";
|
||||||
|
private static final String DLA_ESCAPE_CHARACTER = "";
|
||||||
|
private static final String TBL_PROPERTIES_STR = "TBLPROPERTIES";
|
||||||
|
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
Class.forName(DRIVER_NAME);
|
||||||
|
} catch (ClassNotFoundException e) {
|
||||||
|
throw new IllegalStateException("Could not find " + DRIVER_NAME + " in classpath. ", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Connection connection;
|
||||||
|
private DLASyncConfig dlaConfig;
|
||||||
|
private PartitionValueExtractor partitionValueExtractor;
|
||||||
|
|
||||||
|
public HoodieDLAClient(DLASyncConfig syncConfig, FileSystem fs) {
|
||||||
|
super(syncConfig.basePath, syncConfig.assumeDatePartitioning, fs);
|
||||||
|
this.dlaConfig = syncConfig;
|
||||||
|
try {
|
||||||
|
this.partitionValueExtractor =
|
||||||
|
(PartitionValueExtractor) Class.forName(dlaConfig.partitionValueExtractorClass).newInstance();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieException(
|
||||||
|
"Failed to initialize PartitionValueExtractor class " + dlaConfig.partitionValueExtractorClass, e);
|
||||||
|
}
|
||||||
|
createDLAConnection();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createDLAConnection() {
|
||||||
|
if (connection == null) {
|
||||||
|
try {
|
||||||
|
Class.forName(DRIVER_NAME);
|
||||||
|
} catch (ClassNotFoundException e) {
|
||||||
|
LOG.error("Unable to load DLA driver class", e);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
this.connection = DriverManager.getConnection(dlaConfig.jdbcUrl, dlaConfig.dlaUser, dlaConfig.dlaPass);
|
||||||
|
LOG.info("Successfully established DLA connection to " + dlaConfig.jdbcUrl);
|
||||||
|
} catch (SQLException e) {
|
||||||
|
throw new HoodieException("Cannot create dla connection ", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) {
|
||||||
|
try {
|
||||||
|
String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, toHiveSyncConfig(), inputFormatClass, outputFormatClass, serdeClass);
|
||||||
|
LOG.info("Creating table with " + createSQLQuery);
|
||||||
|
updateDLASQL(createSQLQuery);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieException("Failed to create table " + tableName, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, String> getTableSchema(String tableName) {
|
||||||
|
if (!doesTableExist(tableName)) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Failed to get schema for table " + tableName + " does not exist");
|
||||||
|
}
|
||||||
|
Map<String, String> schema = new HashMap<>();
|
||||||
|
ResultSet result = null;
|
||||||
|
try {
|
||||||
|
DatabaseMetaData databaseMetaData = connection.getMetaData();
|
||||||
|
result = databaseMetaData.getColumns(dlaConfig.databaseName, dlaConfig.databaseName, tableName, null);
|
||||||
|
while (result.next()) {
|
||||||
|
String columnName = result.getString(4);
|
||||||
|
String columnType = result.getString(6);
|
||||||
|
if ("DECIMAL".equals(columnType)) {
|
||||||
|
int columnSize = result.getInt("COLUMN_SIZE");
|
||||||
|
int decimalDigits = result.getInt("DECIMAL_DIGITS");
|
||||||
|
columnType += String.format("(%s,%s)", columnSize, decimalDigits);
|
||||||
|
}
|
||||||
|
schema.put(columnName, columnType);
|
||||||
|
}
|
||||||
|
return schema;
|
||||||
|
} catch (SQLException e) {
|
||||||
|
throw new HoodieException("Failed to get table schema for " + tableName, e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(result, null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addPartitionsToTable(String tableName, List<String> partitionsToAdd) {
|
||||||
|
if (partitionsToAdd.isEmpty()) {
|
||||||
|
LOG.info("No partitions to add for " + tableName);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + tableName);
|
||||||
|
String sql = constructAddPartitions(tableName, partitionsToAdd);
|
||||||
|
updateDLASQL(sql);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String constructAddPartitions(String tableName, List<String> partitions) {
|
||||||
|
return constructDLAAddPartitions(tableName, partitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
String generateAbsolutePathStr(Path path) {
|
||||||
|
String absolutePathStr = path.toString();
|
||||||
|
if (path.toUri().getScheme() == null) {
|
||||||
|
absolutePathStr = getDefaultFs() + absolutePathStr;
|
||||||
|
}
|
||||||
|
return absolutePathStr.endsWith("/") ? absolutePathStr : absolutePathStr + "/";
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> constructChangePartitions(String tableName, List<String> partitions) {
|
||||||
|
List<String> changePartitions = new ArrayList<>();
|
||||||
|
String useDatabase = "USE " + DLA_ESCAPE_CHARACTER + dlaConfig.databaseName + DLA_ESCAPE_CHARACTER;
|
||||||
|
changePartitions.add(useDatabase);
|
||||||
|
String alterTable = "ALTER TABLE " + DLA_ESCAPE_CHARACTER + tableName + DLA_ESCAPE_CHARACTER;
|
||||||
|
for (String partition : partitions) {
|
||||||
|
String partitionClause = getPartitionClause(partition);
|
||||||
|
Path partitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition);
|
||||||
|
String fullPartitionPathStr = generateAbsolutePathStr(partitionPath);
|
||||||
|
String changePartition =
|
||||||
|
alterTable + " ADD IF NOT EXISTS PARTITION (" + partitionClause + ") LOCATION '" + fullPartitionPathStr + "'";
|
||||||
|
changePartitions.add(changePartition);
|
||||||
|
}
|
||||||
|
return changePartitions;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generate Hive Partition from partition values.
|
||||||
|
*
|
||||||
|
* @param partition Partition path
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public String getPartitionClause(String partition) {
|
||||||
|
List<String> partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition);
|
||||||
|
ValidationUtils.checkArgument(dlaConfig.partitionFields.size() == partitionValues.size(),
|
||||||
|
"Partition key parts " + dlaConfig.partitionFields + " does not match with partition values " + partitionValues
|
||||||
|
+ ". Check partition strategy. ");
|
||||||
|
List<String> partBuilder = new ArrayList<>();
|
||||||
|
for (int i = 0; i < dlaConfig.partitionFields.size(); i++) {
|
||||||
|
partBuilder.add(dlaConfig.partitionFields.get(i) + "='" + partitionValues.get(i) + "'");
|
||||||
|
}
|
||||||
|
return partBuilder.stream().collect(Collectors.joining(","));
|
||||||
|
}
|
||||||
|
|
||||||
|
private String constructDLAAddPartitions(String tableName, List<String> partitions) {
|
||||||
|
StringBuilder alterSQL = new StringBuilder("ALTER TABLE ");
|
||||||
|
alterSQL.append(DLA_ESCAPE_CHARACTER).append(dlaConfig.databaseName)
|
||||||
|
.append(DLA_ESCAPE_CHARACTER).append(".").append(DLA_ESCAPE_CHARACTER)
|
||||||
|
.append(tableName).append(DLA_ESCAPE_CHARACTER).append(" ADD IF NOT EXISTS ");
|
||||||
|
for (String partition : partitions) {
|
||||||
|
String partitionClause = getPartitionClause(partition);
|
||||||
|
Path partitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition);
|
||||||
|
String fullPartitionPathStr = generateAbsolutePathStr(partitionPath);
|
||||||
|
alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '").append(fullPartitionPathStr)
|
||||||
|
.append("' ");
|
||||||
|
}
|
||||||
|
return alterSQL.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateDLASQL(String sql) {
|
||||||
|
Statement stmt = null;
|
||||||
|
try {
|
||||||
|
stmt = connection.createStatement();
|
||||||
|
LOG.info("Executing SQL " + sql);
|
||||||
|
stmt.execute(sql);
|
||||||
|
} catch (SQLException e) {
|
||||||
|
throw new HoodieException("Failed in executing SQL " + sql, e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(null, stmt);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean doesTableExist(String tableName) {
|
||||||
|
String sql = consutructShowCreateTableSQL(tableName);
|
||||||
|
Statement stmt = null;
|
||||||
|
ResultSet rs = null;
|
||||||
|
try {
|
||||||
|
stmt = connection.createStatement();
|
||||||
|
rs = stmt.executeQuery(sql);
|
||||||
|
} catch (SQLException e) {
|
||||||
|
return false;
|
||||||
|
} finally {
|
||||||
|
closeQuietly(rs, stmt);
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Option<String> getLastCommitTimeSynced(String tableName) {
|
||||||
|
String sql = consutructShowCreateTableSQL(tableName);
|
||||||
|
Statement stmt = null;
|
||||||
|
ResultSet rs = null;
|
||||||
|
try {
|
||||||
|
stmt = connection.createStatement();
|
||||||
|
rs = stmt.executeQuery(sql);
|
||||||
|
if (rs.next()) {
|
||||||
|
String table = rs.getString(2);
|
||||||
|
Map<String, String> attr = new HashMap<>();
|
||||||
|
int index = table.indexOf(TBL_PROPERTIES_STR);
|
||||||
|
if (index != -1) {
|
||||||
|
String sub = table.substring(index + TBL_PROPERTIES_STR.length());
|
||||||
|
sub = sub.replaceAll("\\(", "").replaceAll("\\)", "").replaceAll("'", "");
|
||||||
|
String[] str = sub.split(",");
|
||||||
|
|
||||||
|
for (int i = 0; i < str.length; i++) {
|
||||||
|
String key = str[i].split("=")[0].trim();
|
||||||
|
String value = str[i].split("=")[1].trim();
|
||||||
|
attr.put(key, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return Option.ofNullable(attr.getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null));
|
||||||
|
}
|
||||||
|
return Option.empty();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieHiveSyncException("Failed to get the last commit time synced from the table", e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(rs, stmt);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateLastCommitTimeSynced(String tableName) {
|
||||||
|
// TODO : dla do not support update tblproperties, so do nothing.
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updatePartitionsToTable(String tableName, List<String> changedPartitions) {
|
||||||
|
if (changedPartitions.isEmpty()) {
|
||||||
|
LOG.info("No partitions to change for " + tableName);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
LOG.info("Changing partitions " + changedPartitions.size() + " on " + tableName);
|
||||||
|
List<String> sqls = constructChangePartitions(tableName, changedPartitions);
|
||||||
|
for (String sql : sqls) {
|
||||||
|
updateDLASQL(sql);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<List<String>, String> scanTablePartitions(String tableName) {
|
||||||
|
String sql = constructShowPartitionSQL(tableName);
|
||||||
|
Statement stmt = null;
|
||||||
|
ResultSet rs = null;
|
||||||
|
Map<List<String>, String> partitions = new HashMap<>();
|
||||||
|
try {
|
||||||
|
stmt = connection.createStatement();
|
||||||
|
LOG.info("Executing SQL " + sql);
|
||||||
|
rs = stmt.executeQuery(sql);
|
||||||
|
while (rs.next()) {
|
||||||
|
if (rs.getMetaData().getColumnCount() > 0) {
|
||||||
|
String str = rs.getString(1);
|
||||||
|
if (!StringUtils.isNullOrEmpty(str)) {
|
||||||
|
List<String> values = partitionValueExtractor.extractPartitionValuesInPath(str);
|
||||||
|
Path storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, String.join("/", values));
|
||||||
|
String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath();
|
||||||
|
partitions.put(values, fullStoragePartitionPath);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return partitions;
|
||||||
|
} catch (SQLException e) {
|
||||||
|
throw new HoodieException("Failed in executing SQL " + sql, e);
|
||||||
|
} finally {
|
||||||
|
closeQuietly(rs, stmt);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PartitionEvent> getPartitionEvents(Map<List<String>, String> tablePartitions, List<String> partitionStoragePartitions) {
|
||||||
|
Map<String, String> paths = new HashMap<>();
|
||||||
|
|
||||||
|
for (Map.Entry<List<String>, String> entry : tablePartitions.entrySet()) {
|
||||||
|
List<String> partitionValues = entry.getKey();
|
||||||
|
Collections.sort(partitionValues);
|
||||||
|
String fullTablePartitionPath = entry.getValue();
|
||||||
|
paths.put(String.join(", ", partitionValues), fullTablePartitionPath);
|
||||||
|
}
|
||||||
|
List<PartitionEvent> events = new ArrayList<>();
|
||||||
|
for (String storagePartition : partitionStoragePartitions) {
|
||||||
|
Path storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, storagePartition);
|
||||||
|
String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath();
|
||||||
|
// Check if the partition values or if hdfs path is the same
|
||||||
|
List<String> storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition);
|
||||||
|
if (dlaConfig.useDLASyncHiveStylePartitioning) {
|
||||||
|
String partition = String.join("/", storagePartitionValues);
|
||||||
|
storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition);
|
||||||
|
fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath();
|
||||||
|
}
|
||||||
|
Collections.sort(storagePartitionValues);
|
||||||
|
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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return events;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateTableDefinition(String tableName, SchemaDifference schemaDiff) {
|
||||||
|
ValidationUtils.checkArgument(schemaDiff.getDeleteColumns().size() == 0, "not support delete columns");
|
||||||
|
ValidationUtils.checkArgument(schemaDiff.getUpdateColumnTypes().size() == 0, "not support alter column type");
|
||||||
|
Map<String, String> columns = schemaDiff.getAddColumnTypes();
|
||||||
|
for (Map.Entry<String, String> entry : columns.entrySet()) {
|
||||||
|
String columnName = entry.getKey();
|
||||||
|
String columnType = entry.getValue();
|
||||||
|
StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append(DLA_ESCAPE_CHARACTER)
|
||||||
|
.append(dlaConfig.databaseName).append(DLA_ESCAPE_CHARACTER).append(".")
|
||||||
|
.append(DLA_ESCAPE_CHARACTER).append(tableName)
|
||||||
|
.append(DLA_ESCAPE_CHARACTER).append(" ADD COLUMNS(")
|
||||||
|
.append(columnName).append(" ").append(columnType).append(" )");
|
||||||
|
LOG.info("Updating table definition with " + sqlBuilder);
|
||||||
|
updateDLASQL(sqlBuilder.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
try {
|
||||||
|
if (connection != null) {
|
||||||
|
connection.close();
|
||||||
|
}
|
||||||
|
} catch (SQLException e) {
|
||||||
|
LOG.error("Could not close connection ", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private String constructShowPartitionSQL(String tableName) {
|
||||||
|
String sql = "show partitions " + dlaConfig.databaseName + "." + tableName;
|
||||||
|
return sql;
|
||||||
|
}
|
||||||
|
|
||||||
|
private String consutructShowCreateTableSQL(String tableName) {
|
||||||
|
String sql = "show create table " + dlaConfig.databaseName + "." + tableName;
|
||||||
|
return sql;
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getDefaultFs() {
|
||||||
|
return fs.getConf().get("fs.defaultFS");
|
||||||
|
}
|
||||||
|
|
||||||
|
private HiveSyncConfig toHiveSyncConfig() {
|
||||||
|
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||||
|
hiveSyncConfig.partitionFields = dlaConfig.partitionFields;
|
||||||
|
hiveSyncConfig.databaseName = dlaConfig.databaseName;
|
||||||
|
Path basePath = new Path(dlaConfig.basePath);
|
||||||
|
hiveSyncConfig.basePath = generateAbsolutePathStr(basePath);
|
||||||
|
return hiveSyncConfig;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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.dla.util;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
|
import org.apache.hudi.dla.DLASyncConfig;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
public class Utils {
|
||||||
|
public static String DLA_DATABASE_OPT_KEY = "hoodie.datasource.dla_sync.database";
|
||||||
|
public static String DLA_TABLE_OPT_KEY = "hoodie.datasource.dla_sync.table";
|
||||||
|
public static String DLA_USER_OPT_KEY = "hoodie.datasource.dla_sync.username";
|
||||||
|
public static String DLA_PASS_OPT_KEY = "hoodie.datasource.dla_sync.password";
|
||||||
|
public static String DLA_URL_OPT_KEY = "hoodie.datasource.dla_sync.jdbcurl";
|
||||||
|
public static String BATH_PATH = "basePath";
|
||||||
|
public static String DLA_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.dla_sync.partition_fields";
|
||||||
|
public static String DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.dla_sync.partition_extractor_class";
|
||||||
|
public static String DLA_ASSUME_DATE_PARTITIONING = "hoodie.datasource.dla_sync.assume_date_partitioning";
|
||||||
|
public static String DLA_SKIP_RO_SUFFIX = "hoodie.datasource.dla_sync.skip_ro_suffix";
|
||||||
|
public static String DLA_SYNC_HIVE_STYLE_PARTITIONING = "hoodie.datasource.dla_sync.hive.style.partitioning";
|
||||||
|
|
||||||
|
public static Properties configToProperties(DLASyncConfig cfg) {
|
||||||
|
Properties properties = new Properties();
|
||||||
|
properties.put(DLA_DATABASE_OPT_KEY, cfg.databaseName);
|
||||||
|
properties.put(DLA_TABLE_OPT_KEY, cfg.tableName);
|
||||||
|
properties.put(DLA_USER_OPT_KEY, cfg.dlaUser);
|
||||||
|
properties.put(DLA_PASS_OPT_KEY, cfg.dlaPass);
|
||||||
|
properties.put(DLA_URL_OPT_KEY, cfg.jdbcUrl);
|
||||||
|
properties.put(BATH_PATH, cfg.basePath);
|
||||||
|
properties.put(DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY, cfg.partitionValueExtractorClass);
|
||||||
|
properties.put(DLA_ASSUME_DATE_PARTITIONING, String.valueOf(cfg.assumeDatePartitioning));
|
||||||
|
properties.put(DLA_SKIP_RO_SUFFIX, String.valueOf(cfg.skipROSuffix));
|
||||||
|
properties.put(DLA_SYNC_HIVE_STYLE_PARTITIONING, String.valueOf(cfg.useDLASyncHiveStylePartitioning));
|
||||||
|
return properties;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DLASyncConfig propertiesToConfig(Properties properties) {
|
||||||
|
DLASyncConfig config = new DLASyncConfig();
|
||||||
|
config.databaseName = properties.getProperty(DLA_DATABASE_OPT_KEY);
|
||||||
|
config.tableName = properties.getProperty(DLA_TABLE_OPT_KEY);
|
||||||
|
config.dlaUser = properties.getProperty(DLA_USER_OPT_KEY);
|
||||||
|
config.dlaPass = properties.getProperty(DLA_PASS_OPT_KEY);
|
||||||
|
config.jdbcUrl = properties.getProperty(DLA_URL_OPT_KEY);
|
||||||
|
config.basePath = properties.getProperty(BATH_PATH);
|
||||||
|
if (StringUtils.isNullOrEmpty(properties.getProperty(DLA_PARTITION_FIELDS_OPT_KEY))) {
|
||||||
|
config.partitionFields = new ArrayList<>();
|
||||||
|
} else {
|
||||||
|
config.partitionFields = Arrays.asList(properties.getProperty(DLA_PARTITION_FIELDS_OPT_KEY).split(","));
|
||||||
|
}
|
||||||
|
config.partitionValueExtractorClass = properties.getProperty(DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY);
|
||||||
|
config.assumeDatePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_ASSUME_DATE_PARTITIONING, "false"));
|
||||||
|
config.skipROSuffix = Boolean.parseBoolean(properties.getProperty(DLA_SKIP_RO_SUFFIX, "false"));
|
||||||
|
config.useDLASyncHiveStylePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_SYNC_HIVE_STYLE_PARTITIONING, "false"));
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,54 @@
|
|||||||
|
/*
|
||||||
|
* 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.dla;
|
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
public class TestDLASyncConfig {
|
||||||
|
@Test
|
||||||
|
public void testCopy() {
|
||||||
|
DLASyncConfig dlaSyncConfig = new DLASyncConfig();
|
||||||
|
List<String> partitions = Arrays.asList("a", "b");
|
||||||
|
dlaSyncConfig.partitionFields = partitions;
|
||||||
|
dlaSyncConfig.basePath = "/tmp";
|
||||||
|
dlaSyncConfig.assumeDatePartitioning = true;
|
||||||
|
dlaSyncConfig.databaseName = "test";
|
||||||
|
dlaSyncConfig.tableName = "test";
|
||||||
|
dlaSyncConfig.dlaUser = "dla";
|
||||||
|
dlaSyncConfig.dlaPass = "dla";
|
||||||
|
dlaSyncConfig.jdbcUrl = "jdbc:mysql://localhost:3306";
|
||||||
|
dlaSyncConfig.skipROSuffix = false;
|
||||||
|
|
||||||
|
DLASyncConfig copied = DLASyncConfig.copy(dlaSyncConfig);
|
||||||
|
|
||||||
|
assertEquals(copied.partitionFields, dlaSyncConfig.partitionFields);
|
||||||
|
assertEquals(copied.basePath, dlaSyncConfig.basePath);
|
||||||
|
assertEquals(copied.assumeDatePartitioning, dlaSyncConfig.assumeDatePartitioning);
|
||||||
|
assertEquals(copied.databaseName, dlaSyncConfig.databaseName);
|
||||||
|
assertEquals(copied.tableName, dlaSyncConfig.tableName);
|
||||||
|
assertEquals(copied.dlaUser, dlaSyncConfig.dlaUser);
|
||||||
|
assertEquals(copied.dlaPass, dlaSyncConfig.dlaPass);
|
||||||
|
assertEquals(copied.basePath, dlaSyncConfig.basePath);
|
||||||
|
assertEquals(copied.jdbcUrl, dlaSyncConfig.jdbcUrl);
|
||||||
|
assertEquals(copied.skipROSuffix, dlaSyncConfig.skipROSuffix);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -20,7 +20,9 @@
|
|||||||
<artifactId>hudi</artifactId>
|
<artifactId>hudi</artifactId>
|
||||||
<groupId>org.apache.hudi</groupId>
|
<groupId>org.apache.hudi</groupId>
|
||||||
<version>0.6.0-SNAPSHOT</version>
|
<version>0.6.0-SNAPSHOT</version>
|
||||||
|
<relativePath>../../pom.xml</relativePath>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
<artifactId>hudi-hive-sync</artifactId>
|
<artifactId>hudi-hive-sync</artifactId>
|
||||||
@@ -43,6 +45,11 @@
|
|||||||
<artifactId>hudi-hadoop-mr</artifactId>
|
<artifactId>hudi-hadoop-mr</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-sync-common</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<!-- Logging -->
|
<!-- Logging -->
|
||||||
<dependency>
|
<dependency>
|
||||||
46
hudi-sync/hudi-hive-sync/src/assembly/src.xml
Normal file
46
hudi-sync/hudi-hive-sync/src/assembly/src.xml
Normal file
@@ -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.
|
||||||
|
-->
|
||||||
|
|
||||||
|
<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
|
xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
|
||||||
|
<id>jar-with-dependencies</id>
|
||||||
|
<formats>
|
||||||
|
<format>jar</format>
|
||||||
|
</formats>
|
||||||
|
|
||||||
|
<includeBaseDirectory>false</includeBaseDirectory>
|
||||||
|
<dependencySets>
|
||||||
|
|
||||||
|
<dependencySet>
|
||||||
|
<outputDirectory>/</outputDirectory>
|
||||||
|
<unpack>true</unpack>
|
||||||
|
<scope>runtime</scope>
|
||||||
|
<excludes>
|
||||||
|
<exclude>junit:junit</exclude>
|
||||||
|
<exclude>com.google.code.findbugs:*</exclude>
|
||||||
|
<exclude>org.apache.hbase:*</exclude>
|
||||||
|
</excludes>
|
||||||
|
</dependencySet>
|
||||||
|
|
||||||
|
<dependencySet>
|
||||||
|
<unpack>true</unpack>
|
||||||
|
<scope>provided</scope>
|
||||||
|
</dependencySet>
|
||||||
|
</dependencySets>
|
||||||
|
</assembly>
|
||||||
@@ -23,8 +23,8 @@ import org.apache.hudi.common.model.HoodieFileFormat;
|
|||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.exception.InvalidTableException;
|
import org.apache.hudi.exception.InvalidTableException;
|
||||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent;
|
||||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType;
|
||||||
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||||
|
|
||||||
import com.beust.jcommander.JCommander;
|
import com.beust.jcommander.JCommander;
|
||||||
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.hive.conf.HiveConf;
|
import org.apache.hadoop.hive.conf.HiveConf;
|
||||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.parquet.schema.MessageType;
|
import org.apache.parquet.schema.MessageType;
|
||||||
@@ -49,7 +50,7 @@ import java.util.stream.Collectors;
|
|||||||
* partitions incrementally (all the partitions modified since the last commit)
|
* partitions incrementally (all the partitions modified since the last commit)
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("WeakerAccess")
|
@SuppressWarnings("WeakerAccess")
|
||||||
public class HiveSyncTool {
|
public class HiveSyncTool extends AbstractSyncTool {
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class);
|
private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class);
|
||||||
public static final String SUFFIX_SNAPSHOT_TABLE = "_rt";
|
public static final String SUFFIX_SNAPSHOT_TABLE = "_rt";
|
||||||
@@ -61,6 +62,7 @@ public class HiveSyncTool {
|
|||||||
private final Option<String> roTableTableName;
|
private final Option<String> roTableTableName;
|
||||||
|
|
||||||
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||||
|
super(configuration.getAllProperties(), fs);
|
||||||
this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs);
|
this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs);
|
||||||
this.cfg = cfg;
|
this.cfg = cfg;
|
||||||
// Set partitionFields to empty, when the NonPartitionedExtractor is used
|
// Set partitionFields to empty, when the NonPartitionedExtractor is used
|
||||||
@@ -84,6 +86,7 @@ public class HiveSyncTool {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public void syncHoodieTable() {
|
public void syncHoodieTable() {
|
||||||
try {
|
try {
|
||||||
switch (hoodieHiveClient.getTableType()) {
|
switch (hoodieHiveClient.getTableType()) {
|
||||||
@@ -20,14 +20,9 @@ package org.apache.hudi.hive;
|
|||||||
|
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.fs.StorageSchemes;
|
import org.apache.hudi.common.fs.StorageSchemes;
|
||||||
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.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
|
||||||
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -43,6 +38,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
|
|||||||
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
|
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
|
||||||
import org.apache.hadoop.hive.ql.session.SessionState;
|
import org.apache.hadoop.hive.ql.session.SessionState;
|
||||||
import org.apache.hive.jdbc.HiveDriver;
|
import org.apache.hive.jdbc.HiveDriver;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.parquet.schema.MessageType;
|
import org.apache.parquet.schema.MessageType;
|
||||||
@@ -62,7 +58,7 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class HoodieHiveClient {
|
public class HoodieHiveClient extends AbstractSyncHoodieClient {
|
||||||
|
|
||||||
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync";
|
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync";
|
||||||
// Make sure we have the hive JDBC driver in classpath
|
// Make sure we have the hive JDBC driver in classpath
|
||||||
@@ -78,8 +74,6 @@ public class HoodieHiveClient {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HoodieHiveClient.class);
|
private static final Logger LOG = LogManager.getLogger(HoodieHiveClient.class);
|
||||||
private final HoodieTableMetaClient metaClient;
|
|
||||||
private final HoodieTableType tableType;
|
|
||||||
private final PartitionValueExtractor partitionValueExtractor;
|
private final PartitionValueExtractor partitionValueExtractor;
|
||||||
private IMetaStoreClient client;
|
private IMetaStoreClient client;
|
||||||
private HiveSyncConfig syncConfig;
|
private HiveSyncConfig syncConfig;
|
||||||
@@ -89,10 +83,9 @@ public class HoodieHiveClient {
|
|||||||
private HiveConf configuration;
|
private HiveConf configuration;
|
||||||
|
|
||||||
public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||||
|
super(cfg.basePath, cfg.assumeDatePartitioning, fs);
|
||||||
this.syncConfig = cfg;
|
this.syncConfig = cfg;
|
||||||
this.fs = fs;
|
this.fs = fs;
|
||||||
this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true);
|
|
||||||
this.tableType = metaClient.getTableType();
|
|
||||||
|
|
||||||
this.configuration = configuration;
|
this.configuration = configuration;
|
||||||
// Support both JDBC and metastore based implementations for backwards compatiblity. Future users should
|
// Support both JDBC and metastore based implementations for backwards compatiblity. Future users should
|
||||||
@@ -125,7 +118,8 @@ public class HoodieHiveClient {
|
|||||||
/**
|
/**
|
||||||
* Add the (NEW) partitions to the table.
|
* Add the (NEW) partitions to the table.
|
||||||
*/
|
*/
|
||||||
void addPartitionsToTable(String tableName, List<String> partitionsToAdd) {
|
@Override
|
||||||
|
public void addPartitionsToTable(String tableName, List<String> partitionsToAdd) {
|
||||||
if (partitionsToAdd.isEmpty()) {
|
if (partitionsToAdd.isEmpty()) {
|
||||||
LOG.info("No partitions to add for " + tableName);
|
LOG.info("No partitions to add for " + tableName);
|
||||||
return;
|
return;
|
||||||
@@ -138,7 +132,8 @@ public class HoodieHiveClient {
|
|||||||
/**
|
/**
|
||||||
* Partition path has changed - update the path for te following partitions.
|
* Partition path has changed - update the path for te following partitions.
|
||||||
*/
|
*/
|
||||||
void updatePartitionsToTable(String tableName, List<String> changedPartitions) {
|
@Override
|
||||||
|
public void updatePartitionsToTable(String tableName, List<String> changedPartitions) {
|
||||||
if (changedPartitions.isEmpty()) {
|
if (changedPartitions.isEmpty()) {
|
||||||
LOG.info("No partitions to change for " + tableName);
|
LOG.info("No partitions to change for " + tableName);
|
||||||
return;
|
return;
|
||||||
@@ -258,7 +253,8 @@ public class HoodieHiveClient {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) {
|
@Override
|
||||||
|
public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) {
|
||||||
try {
|
try {
|
||||||
String createSQLQuery =
|
String createSQLQuery =
|
||||||
HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass);
|
HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass);
|
||||||
@@ -272,6 +268,7 @@ public class HoodieHiveClient {
|
|||||||
/**
|
/**
|
||||||
* Get the table schema.
|
* Get the table schema.
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public Map<String, String> getTableSchema(String tableName) {
|
public Map<String, String> getTableSchema(String tableName) {
|
||||||
if (syncConfig.useJdbc) {
|
if (syncConfig.useJdbc) {
|
||||||
if (!doesTableExist(tableName)) {
|
if (!doesTableExist(tableName)) {
|
||||||
@@ -327,24 +324,10 @@ public class HoodieHiveClient {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the schema for a hoodie table. Depending on the type of table, try to read schema from commit metadata if
|
|
||||||
* present, else fallback to reading from any file written in the latest commit. We will assume that the schema has
|
|
||||||
* not changed within a single atomic write.
|
|
||||||
*
|
|
||||||
* @return Parquet schema for this table
|
|
||||||
*/
|
|
||||||
public MessageType getDataSchema() {
|
|
||||||
try {
|
|
||||||
return new TableSchemaResolver(metaClient).getTableParquetSchema();
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new HoodieHiveSyncException("Failed to read data schema", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return true if the configured table exists
|
* @return true if the configured table exists
|
||||||
*/
|
*/
|
||||||
|
@Override
|
||||||
public boolean doesTableExist(String tableName) {
|
public boolean doesTableExist(String tableName) {
|
||||||
try {
|
try {
|
||||||
return client.tableExists(syncConfig.databaseName, tableName);
|
return client.tableExists(syncConfig.databaseName, tableName);
|
||||||
@@ -455,36 +438,7 @@ public class HoodieHiveClient {
|
|||||||
return hiveJdbcUrl + (urlAppend == null ? "" : urlAppend);
|
return hiveJdbcUrl + (urlAppend == null ? "" : urlAppend);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void closeQuietly(ResultSet resultSet, Statement stmt) {
|
@Override
|
||||||
try {
|
|
||||||
if (stmt != null) {
|
|
||||||
stmt.close();
|
|
||||||
}
|
|
||||||
} catch (SQLException e) {
|
|
||||||
LOG.error("Could not close the statement opened ", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
if (resultSet != null) {
|
|
||||||
resultSet.close();
|
|
||||||
}
|
|
||||||
} catch (SQLException e) {
|
|
||||||
LOG.error("Could not close the resultset opened ", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getBasePath() {
|
|
||||||
return metaClient.getBasePath();
|
|
||||||
}
|
|
||||||
|
|
||||||
HoodieTableType getTableType() {
|
|
||||||
return tableType;
|
|
||||||
}
|
|
||||||
|
|
||||||
public FileSystem getFs() {
|
|
||||||
return fs;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Option<String> getLastCommitTimeSynced(String tableName) {
|
public Option<String> getLastCommitTimeSynced(String tableName) {
|
||||||
// Get the last commit time from the TBLproperties
|
// Get the last commit time from the TBLproperties
|
||||||
try {
|
try {
|
||||||
@@ -509,33 +463,12 @@ public class HoodieHiveClient {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
|
|
||||||
if (!lastCommitTimeSynced.isPresent()) {
|
|
||||||
LOG.info("Last commit time synced is not known, listing all partitions in " + syncConfig.basePath + ",FS :" + fs);
|
|
||||||
try {
|
|
||||||
return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then");
|
|
||||||
|
|
||||||
HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE);
|
|
||||||
return timelineToSync.getInstants().map(s -> {
|
|
||||||
try {
|
|
||||||
return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get(), HoodieCommitMetadata.class);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIOException("Failed to get partitions written since " + lastCommitTimeSynced, e);
|
|
||||||
}
|
|
||||||
}).flatMap(s -> s.getPartitionToWriteStats().keySet().stream()).distinct().collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
List<String> getAllTables(String db) throws Exception {
|
List<String> getAllTables(String db) throws Exception {
|
||||||
return client.getAllTables(db);
|
return client.getAllTables(db);
|
||||||
}
|
}
|
||||||
|
|
||||||
void updateLastCommitTimeSynced(String tableName) {
|
@Override
|
||||||
|
public void updateLastCommitTimeSynced(String tableName) {
|
||||||
// Set the last commit time from the TBLproperties
|
// Set the last commit time from the TBLproperties
|
||||||
String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp();
|
String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp();
|
||||||
try {
|
try {
|
||||||
@@ -546,34 +479,4 @@ public class HoodieHiveClient {
|
|||||||
throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e);
|
throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Partition Event captures any partition that needs to be added or updated.
|
|
||||||
*/
|
|
||||||
static class PartitionEvent {
|
|
||||||
|
|
||||||
public enum PartitionEventType {
|
|
||||||
ADD, UPDATE
|
|
||||||
}
|
|
||||||
|
|
||||||
PartitionEventType eventType;
|
|
||||||
String storagePartition;
|
|
||||||
|
|
||||||
PartitionEvent(PartitionEventType eventType, String storagePartition) {
|
|
||||||
this.eventType = eventType;
|
|
||||||
this.storagePartition = storagePartition;
|
|
||||||
}
|
|
||||||
|
|
||||||
static PartitionEvent newPartitionAddEvent(String storagePartition) {
|
|
||||||
return new PartitionEvent(PartitionEventType.ADD, storagePartition);
|
|
||||||
}
|
|
||||||
|
|
||||||
static PartitionEvent newPartitionUpdateEvent(String storagePartition) {
|
|
||||||
return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public IMetaStoreClient getClient() {
|
|
||||||
return client;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
@@ -23,10 +23,12 @@ import org.apache.parquet.schema.MessageType;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.StringJoiner;
|
import java.util.StringJoiner;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents the schema difference between the storage schema and hive table schema.
|
* Represents the schema difference between the storage schema and hive table schema.
|
||||||
*/
|
*/
|
||||||
@@ -91,7 +93,7 @@ public class SchemaDifference {
|
|||||||
this.tableSchema = tableSchema;
|
this.tableSchema = tableSchema;
|
||||||
deleteColumns = new ArrayList<>();
|
deleteColumns = new ArrayList<>();
|
||||||
updateColumnTypes = new HashMap<>();
|
updateColumnTypes = new HashMap<>();
|
||||||
addColumnTypes = new HashMap<>();
|
addColumnTypes = new LinkedHashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder deleteTableColumn(String column) {
|
public Builder deleteTableColumn(String column) {
|
||||||
@@ -21,8 +21,8 @@ package org.apache.hudi.hive;
|
|||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
import org.apache.hudi.common.testutils.SchemaTestUtil;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent;
|
||||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType;
|
||||||
import org.apache.hudi.hive.testutils.HiveTestUtil;
|
import org.apache.hudi.hive.testutils.HiveTestUtil;
|
||||||
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||||
|
|
||||||
@@ -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.
|
||||||
|
###
|
||||||
|
log4j.rootLogger=WARN, CONSOLE
|
||||||
|
log4j.logger.org.apache.hudi=DEBUG
|
||||||
|
|
||||||
|
# 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
|
||||||
@@ -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=INFO
|
||||||
|
log4j.logger.org.apache.hudi=DEBUG
|
||||||
|
|
||||||
|
# A1 is set to be a ConsoleAppender.
|
||||||
|
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||||
|
# A1 uses PatternLayout.
|
||||||
|
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||||
|
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %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
|
||||||
71
hudi-sync/hudi-sync-common/pom.xml
Normal file
71
hudi-sync/hudi-sync-common/pom.xml
Normal file
@@ -0,0 +1,71 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
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.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<parent>
|
||||||
|
<artifactId>hudi</artifactId>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<version>0.6.0-SNAPSHOT</version>
|
||||||
|
<relativePath>../../pom.xml</relativePath>
|
||||||
|
</parent>
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<artifactId>hudi-sync-common</artifactId>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
|
<properties>
|
||||||
|
<main.basedir>${project.parent.basedir}</main.basedir>
|
||||||
|
</properties>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-common</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.parquet</groupId>
|
||||||
|
<artifactId>parquet-avro</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hadoop</groupId>
|
||||||
|
<artifactId>hadoop-common</artifactId>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<resources>
|
||||||
|
<resource>
|
||||||
|
<directory>src/main/resources</directory>
|
||||||
|
</resource>
|
||||||
|
</resources>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
|
<artifactId>maven-jar-plugin</artifactId>
|
||||||
|
<version>${maven-jar-plugin.version}</version>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<goals>
|
||||||
|
<goal>test-jar</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</build>
|
||||||
|
</project>
|
||||||
@@ -0,0 +1,189 @@
|
|||||||
|
/*
|
||||||
|
* 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.sync.common;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
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.table.TableSchemaResolver;
|
||||||
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.parquet.schema.MessageType;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public abstract class AbstractSyncHoodieClient {
|
||||||
|
private static final Logger LOG = LogManager.getLogger(AbstractSyncHoodieClient.class);
|
||||||
|
protected final HoodieTableMetaClient metaClient;
|
||||||
|
protected HoodieTimeline activeTimeline;
|
||||||
|
protected final HoodieTableType tableType;
|
||||||
|
protected final FileSystem fs;
|
||||||
|
private String basePath;
|
||||||
|
private boolean assumeDatePartitioning;
|
||||||
|
|
||||||
|
public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, FileSystem fs) {
|
||||||
|
this.metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||||
|
this.tableType = metaClient.getTableType();
|
||||||
|
this.basePath = basePath;
|
||||||
|
this.assumeDatePartitioning = assumeDatePartitioning;
|
||||||
|
this.fs = fs;
|
||||||
|
this.activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract void createTable(String tableName, MessageType storageSchema,
|
||||||
|
String inputFormatClass, String outputFormatClass, String serdeClass);
|
||||||
|
|
||||||
|
public abstract boolean doesTableExist(String tableName);
|
||||||
|
|
||||||
|
public abstract Option<String> getLastCommitTimeSynced(String tableName);
|
||||||
|
|
||||||
|
public abstract void updateLastCommitTimeSynced(String tableName);
|
||||||
|
|
||||||
|
public abstract void addPartitionsToTable(String tableName, List<String> partitionsToAdd);
|
||||||
|
|
||||||
|
public abstract void updatePartitionsToTable(String tableName, List<String> changedPartitions);
|
||||||
|
|
||||||
|
public abstract Map<String, String> getTableSchema(String tableName);
|
||||||
|
|
||||||
|
public HoodieTimeline getActiveTimeline() {
|
||||||
|
return activeTimeline;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieTableType getTableType() {
|
||||||
|
return tableType;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getBasePath() {
|
||||||
|
return metaClient.getBasePath();
|
||||||
|
}
|
||||||
|
|
||||||
|
public FileSystem getFs() {
|
||||||
|
return fs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void closeQuietly(ResultSet resultSet, Statement stmt) {
|
||||||
|
try {
|
||||||
|
if (stmt != null) {
|
||||||
|
stmt.close();
|
||||||
|
}
|
||||||
|
} catch (SQLException e) {
|
||||||
|
LOG.warn("Could not close the statement opened ", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (resultSet != null) {
|
||||||
|
resultSet.close();
|
||||||
|
}
|
||||||
|
} catch (SQLException e) {
|
||||||
|
LOG.warn("Could not close the resultset opened ", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the schema for a hoodie table. Depending on the type of table, try to read schema from commit metadata if
|
||||||
|
* present, else fallback to reading from any file written in the latest commit. We will assume that the schema has
|
||||||
|
* not changed within a single atomic write.
|
||||||
|
*
|
||||||
|
* @return Parquet schema for this table
|
||||||
|
*/
|
||||||
|
public MessageType getDataSchema() {
|
||||||
|
try {
|
||||||
|
return new TableSchemaResolver(metaClient).getTableParquetSchema();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieSyncException("Failed to read data schema", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||||
|
public List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
|
||||||
|
if (!lastCommitTimeSynced.isPresent()) {
|
||||||
|
LOG.info("Last commit time synced is not known, listing all partitions in " + basePath + ",FS :" + fs);
|
||||||
|
try {
|
||||||
|
return FSUtils.getAllPartitionPaths(fs, basePath, assumeDatePartitioning);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to list all partitions in " + basePath, e);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then");
|
||||||
|
|
||||||
|
HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE);
|
||||||
|
return timelineToSync.getInstants().map(s -> {
|
||||||
|
try {
|
||||||
|
return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get(), HoodieCommitMetadata.class);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to get partitions written since " + lastCommitTimeSynced, e);
|
||||||
|
}
|
||||||
|
}).flatMap(s -> s.getPartitionToWriteStats().keySet().stream()).distinct().collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read the schema from the log file on path.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||||
|
private MessageType readSchemaFromLogFile(Option<HoodieInstant> lastCompactionCommitOpt, Path path) throws Exception {
|
||||||
|
MessageType messageType = TableSchemaResolver.readSchemaFromLogFile(fs, path);
|
||||||
|
// Fall back to read the schema from last compaction
|
||||||
|
if (messageType == null) {
|
||||||
|
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
|
||||||
|
return new TableSchemaResolver(this.metaClient).readSchemaFromLastCompaction(lastCompactionCommitOpt);
|
||||||
|
}
|
||||||
|
return messageType;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Partition Event captures any partition that needs to be added or updated.
|
||||||
|
*/
|
||||||
|
public static class PartitionEvent {
|
||||||
|
|
||||||
|
public enum PartitionEventType {
|
||||||
|
ADD, UPDATE
|
||||||
|
}
|
||||||
|
|
||||||
|
public PartitionEventType eventType;
|
||||||
|
public String storagePartition;
|
||||||
|
|
||||||
|
PartitionEvent(PartitionEventType eventType, String storagePartition) {
|
||||||
|
this.eventType = eventType;
|
||||||
|
this.storagePartition = storagePartition;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static PartitionEvent newPartitionAddEvent(String storagePartition) {
|
||||||
|
return new PartitionEvent(PartitionEventType.ADD, storagePartition);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static PartitionEvent newPartitionUpdateEvent(String storagePartition) {
|
||||||
|
return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,35 @@
|
|||||||
|
/*
|
||||||
|
* 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.sync.common;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
public abstract class AbstractSyncTool {
|
||||||
|
protected Properties props;
|
||||||
|
protected FileSystem fileSystem;
|
||||||
|
|
||||||
|
public AbstractSyncTool(Properties props, FileSystem fileSystem) {
|
||||||
|
this.props = props;
|
||||||
|
this.fileSystem = fileSystem;
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract void syncHoodieTable();
|
||||||
|
|
||||||
|
}
|
||||||
@@ -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.sync.common;
|
||||||
|
|
||||||
|
public class HoodieSyncException extends RuntimeException {
|
||||||
|
|
||||||
|
public HoodieSyncException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieSyncException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieSyncException(String message, Throwable t) {
|
||||||
|
super(message, t);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieSyncException(Throwable t) {
|
||||||
|
super(t);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static String format(String message, Object... args) {
|
||||||
|
return String.format(String.valueOf(message), (Object[]) args);
|
||||||
|
}
|
||||||
|
}
|
||||||
38
hudi-sync/pom.xml
Normal file
38
hudi-sync/pom.xml
Normal file
@@ -0,0 +1,38 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
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.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<parent>
|
||||||
|
<artifactId>hudi</artifactId>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<version>0.6.0-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<artifactId>hudi-sync</artifactId>
|
||||||
|
<packaging>pom</packaging>
|
||||||
|
|
||||||
|
<properties>
|
||||||
|
<main.basedir>${project.parent.basedir}</main.basedir>
|
||||||
|
</properties>
|
||||||
|
|
||||||
|
<modules>
|
||||||
|
<module>hudi-sync-common</module>
|
||||||
|
<module>hudi-hive-sync</module>
|
||||||
|
<module>hudi-dla-sync</module>
|
||||||
|
</modules>
|
||||||
|
</project>
|
||||||
@@ -23,6 +23,7 @@ import org.apache.hudi.DataSourceUtils;
|
|||||||
import org.apache.hudi.client.HoodieWriteClient;
|
import org.apache.hudi.client.HoodieWriteClient;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.common.config.TypedProperties;
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
@@ -31,6 +32,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.ReflectionUtils;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||||
@@ -39,6 +41,7 @@ import org.apache.hudi.exception.HoodieException;
|
|||||||
import org.apache.hudi.hive.HiveSyncConfig;
|
import org.apache.hudi.hive.HiveSyncConfig;
|
||||||
import org.apache.hudi.hive.HiveSyncTool;
|
import org.apache.hudi.hive.HiveSyncTool;
|
||||||
import org.apache.hudi.keygen.KeyGenerator;
|
import org.apache.hudi.keygen.KeyGenerator;
|
||||||
|
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||||
import org.apache.hudi.utilities.UtilHelpers;
|
import org.apache.hudi.utilities.UtilHelpers;
|
||||||
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
import org.apache.hudi.exception.HoodieDeltaStreamerException;
|
||||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
|
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
|
||||||
@@ -66,10 +69,15 @@ import org.apache.spark.sql.SparkSession;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.function.Function;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.function.Function;
|
import java.util.Properties;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import scala.collection.JavaConversions;
|
import scala.collection.JavaConversions;
|
||||||
@@ -391,6 +399,7 @@ public class DeltaSync implements Serializable {
|
|||||||
long totalRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalRecords).sum().longValue();
|
long totalRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalRecords).sum().longValue();
|
||||||
boolean hasErrors = totalErrorRecords > 0;
|
boolean hasErrors = totalErrorRecords > 0;
|
||||||
long hiveSyncTimeMs = 0;
|
long hiveSyncTimeMs = 0;
|
||||||
|
long metaSyncTimeMs = 0;
|
||||||
if (!hasErrors || cfg.commitOnErrors) {
|
if (!hasErrors || cfg.commitOnErrors) {
|
||||||
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
||||||
checkpointCommitMetadata.put(CHECKPOINT_KEY, checkpointStr);
|
checkpointCommitMetadata.put(CHECKPOINT_KEY, checkpointStr);
|
||||||
@@ -413,10 +422,7 @@ public class DeltaSync implements Serializable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (!isEmpty) {
|
if (!isEmpty) {
|
||||||
// Sync to hive if enabled
|
syncMeta(metrics);
|
||||||
Timer.Context hiveSyncContext = metrics.getHiveSyncTimerContext();
|
|
||||||
syncHiveIfNeeded();
|
|
||||||
hiveSyncTimeMs = hiveSyncContext != null ? hiveSyncContext.stop() : 0;
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
LOG.info("Commit " + instantTime + " failed!");
|
LOG.info("Commit " + instantTime + " failed!");
|
||||||
@@ -438,8 +444,7 @@ public class DeltaSync implements Serializable {
|
|||||||
long overallTimeMs = overallTimerContext != null ? overallTimerContext.stop() : 0;
|
long overallTimeMs = overallTimerContext != null ? overallTimerContext.stop() : 0;
|
||||||
|
|
||||||
// Send DeltaStreamer Metrics
|
// Send DeltaStreamer Metrics
|
||||||
metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs);
|
metrics.updateDeltaStreamerMetrics(overallTimeMs);
|
||||||
|
|
||||||
return Pair.of(scheduledCompactionInstant, writeStatusRDD);
|
return Pair.of(scheduledCompactionInstant, writeStatusRDD);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -471,12 +476,41 @@ public class DeltaSync implements Serializable {
|
|||||||
throw lastException;
|
throw lastException;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private String getSyncClassShortName(String syncClassName) {
|
||||||
* Sync to Hive.
|
return syncClassName.substring(syncClassName.lastIndexOf(".") + 1);
|
||||||
*/
|
}
|
||||||
public void syncHiveIfNeeded() {
|
|
||||||
|
private void syncMeta(HoodieDeltaStreamerMetrics metrics) {
|
||||||
|
Set<String> syncClientToolClasses = new HashSet<>(Arrays.asList(cfg.syncClientToolClass.split(",")));
|
||||||
|
// for backward compatibility
|
||||||
if (cfg.enableHiveSync) {
|
if (cfg.enableHiveSync) {
|
||||||
syncHive();
|
cfg.enableMetaSync = true;
|
||||||
|
syncClientToolClasses.add(HiveSyncTool.class.getName());
|
||||||
|
LOG.info("When set --enable-hive-sync will use HiveSyncTool for backward compatibility");
|
||||||
|
}
|
||||||
|
if (cfg.enableMetaSync) {
|
||||||
|
for (String impl : syncClientToolClasses) {
|
||||||
|
Timer.Context syncContext = metrics.getMetaSyncTimerContext();
|
||||||
|
impl = impl.trim();
|
||||||
|
AbstractSyncTool syncTool = null;
|
||||||
|
switch (impl) {
|
||||||
|
case "org.apache.hudi.hive.HiveSyncTool":
|
||||||
|
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
|
||||||
|
LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
|
||||||
|
+ hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
|
||||||
|
syncTool = new HiveSyncTool(hiveSyncConfig, new HiveConf(conf, HiveConf.class), fs);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
|
||||||
|
Properties properties = new Properties();
|
||||||
|
properties.putAll(props);
|
||||||
|
properties.put("basePath", cfg.targetBasePath);
|
||||||
|
syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[]{Properties.class, FileSystem.class}, properties, fs);
|
||||||
|
}
|
||||||
|
syncTool.syncHoodieTable();
|
||||||
|
long metaSyncTimeMs = syncContext != null ? syncContext.stop() : 0;
|
||||||
|
metrics.updateDeltaStreamerMetaSyncMetrics(getSyncClassShortName(impl), metaSyncTimeMs);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -38,6 +38,7 @@ import org.apache.hudi.utilities.IdentitySplitter;
|
|||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
|
import org.apache.hudi.hive.HiveSyncTool;
|
||||||
import org.apache.hudi.utilities.HiveIncrementalPuller;
|
import org.apache.hudi.utilities.HiveIncrementalPuller;
|
||||||
import org.apache.hudi.utilities.UtilHelpers;
|
import org.apache.hudi.utilities.UtilHelpers;
|
||||||
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
|
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
|
||||||
@@ -268,9 +269,16 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
description = "Should duplicate records from source be dropped/filtered out before insert/bulk-insert")
|
description = "Should duplicate records from source be dropped/filtered out before insert/bulk-insert")
|
||||||
public Boolean filterDupes = false;
|
public Boolean filterDupes = false;
|
||||||
|
|
||||||
|
//will abandon in the future version, recommended use --enable-sync
|
||||||
@Parameter(names = {"--enable-hive-sync"}, description = "Enable syncing to hive")
|
@Parameter(names = {"--enable-hive-sync"}, description = "Enable syncing to hive")
|
||||||
public Boolean enableHiveSync = false;
|
public Boolean enableHiveSync = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--enable-sync"}, description = "Enable syncing meta")
|
||||||
|
public Boolean enableMetaSync = false;
|
||||||
|
|
||||||
|
@Parameter(names = {"--sync-tool-classes"}, description = "Meta sync client tool, using comma to separate multi tools")
|
||||||
|
public String syncClientToolClass = HiveSyncTool.class.getName();
|
||||||
|
|
||||||
@Parameter(names = {"--max-pending-compactions"},
|
@Parameter(names = {"--max-pending-compactions"},
|
||||||
description = "Maximum number of outstanding inflight/requested compactions. Delta Sync will not happen unless"
|
description = "Maximum number of outstanding inflight/requested compactions. Delta Sync will not happen unless"
|
||||||
+ "outstanding compactions is less than this number")
|
+ "outstanding compactions is less than this number")
|
||||||
@@ -447,6 +455,11 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
Map<String, String> additionalSparkConfigs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
|
Map<String, String> additionalSparkConfigs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
|
||||||
JavaSparkContext jssc =
|
JavaSparkContext jssc =
|
||||||
UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster, additionalSparkConfigs);
|
UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName, cfg.sparkMaster, additionalSparkConfigs);
|
||||||
|
|
||||||
|
if (cfg.enableHiveSync) {
|
||||||
|
LOG.warn("--enable-hive-sync will be deprecated in a future release; please use --enable-sync instead for Hive syncing");
|
||||||
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
new HoodieDeltaStreamer(cfg, jssc).sync();
|
new HoodieDeltaStreamer(cfg, jssc).sync();
|
||||||
} finally {
|
} finally {
|
||||||
|
|||||||
@@ -32,8 +32,10 @@ public class HoodieDeltaStreamerMetrics implements Serializable {
|
|||||||
|
|
||||||
public String overallTimerName = null;
|
public String overallTimerName = null;
|
||||||
public String hiveSyncTimerName = null;
|
public String hiveSyncTimerName = null;
|
||||||
private transient Timer overallTimer = null;
|
public String metaSyncTimerName = null;
|
||||||
public transient Timer hiveSyncTimer = null;
|
private Timer overallTimer = null;
|
||||||
|
public Timer hiveSyncTimer = null;
|
||||||
|
public Timer metaSyncTimer = null;
|
||||||
|
|
||||||
public HoodieDeltaStreamerMetrics(HoodieWriteConfig config) {
|
public HoodieDeltaStreamerMetrics(HoodieWriteConfig config) {
|
||||||
this.config = config;
|
this.config = config;
|
||||||
@@ -42,6 +44,7 @@ public class HoodieDeltaStreamerMetrics implements Serializable {
|
|||||||
Metrics.init(config);
|
Metrics.init(config);
|
||||||
this.overallTimerName = getMetricsName("timer", "deltastreamer");
|
this.overallTimerName = getMetricsName("timer", "deltastreamer");
|
||||||
this.hiveSyncTimerName = getMetricsName("timer", "deltastreamerHiveSync");
|
this.hiveSyncTimerName = getMetricsName("timer", "deltastreamerHiveSync");
|
||||||
|
this.metaSyncTimerName = getMetricsName("timer", "deltastreamerMetaSync");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -59,6 +62,13 @@ public class HoodieDeltaStreamerMetrics implements Serializable {
|
|||||||
return hiveSyncTimer == null ? null : hiveSyncTimer.time();
|
return hiveSyncTimer == null ? null : hiveSyncTimer.time();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Timer.Context getMetaSyncTimerContext() {
|
||||||
|
if (config.isMetricsOn() && metaSyncTimer == null) {
|
||||||
|
metaSyncTimer = createTimer(metaSyncTimerName);
|
||||||
|
}
|
||||||
|
return metaSyncTimer == null ? null : metaSyncTimer.time();
|
||||||
|
}
|
||||||
|
|
||||||
private Timer createTimer(String name) {
|
private Timer createTimer(String name) {
|
||||||
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
||||||
}
|
}
|
||||||
@@ -67,10 +77,15 @@ public class HoodieDeltaStreamerMetrics implements Serializable {
|
|||||||
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void updateDeltaStreamerMetrics(long durationInNs, long hiveSyncNs) {
|
public void updateDeltaStreamerMetrics(long durationInNs) {
|
||||||
if (config.isMetricsOn()) {
|
if (config.isMetricsOn()) {
|
||||||
Metrics.registerGauge(getMetricsName("deltastreamer", "duration"), getDurationInMs(durationInNs));
|
Metrics.registerGauge(getMetricsName("deltastreamer", "duration"), getDurationInMs(durationInNs));
|
||||||
Metrics.registerGauge(getMetricsName("deltastreamer", "hiveSyncDuration"), getDurationInMs(hiveSyncNs));
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateDeltaStreamerMetaSyncMetrics(String syncClassShortName, long syncNs) {
|
||||||
|
if (config.isMetricsOn()) {
|
||||||
|
Metrics.registerGauge(getMetricsName("deltastreamer", syncClassShortName), getDurationInMs(syncNs));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -66,7 +66,8 @@
|
|||||||
<includes>
|
<includes>
|
||||||
<include>org.apache.hudi:hudi-common</include>
|
<include>org.apache.hudi:hudi-common</include>
|
||||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
<include>org.apache.hudi:hudi-sync-common</include>
|
||||||
|
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||||
|
|
||||||
<include>com.beust:jcommander</include>
|
<include>com.beust:jcommander</include>
|
||||||
<include>org.apache.avro:avro</include>
|
<include>org.apache.avro:avro</include>
|
||||||
|
|||||||
@@ -68,6 +68,7 @@
|
|||||||
<include>org.apache.hudi:hudi-client</include>
|
<include>org.apache.hudi:hudi-client</include>
|
||||||
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
||||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||||
|
<include>org.apache.hudi:hudi-sync-common</include>
|
||||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||||
<include>org.apache.hudi:hudi-timeline-service</include>
|
<include>org.apache.hudi:hudi-timeline-service</include>
|
||||||
|
|
||||||
|
|||||||
@@ -70,6 +70,7 @@
|
|||||||
<include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
|
<include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
|
||||||
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
|
||||||
<include>org.apache.hudi:hudi-hive-sync</include>
|
<include>org.apache.hudi:hudi-hive-sync</include>
|
||||||
|
<include>org.apache.hudi:hudi-sync-common</include>
|
||||||
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
<include>org.apache.hudi:hudi-hadoop-mr</include>
|
||||||
<include>org.apache.hudi:hudi-timeline-service</include>
|
<include>org.apache.hudi:hudi-timeline-service</include>
|
||||||
|
|
||||||
|
|||||||
2
pom.xml
2
pom.xml
@@ -39,10 +39,10 @@
|
|||||||
<module>hudi-cli</module>
|
<module>hudi-cli</module>
|
||||||
<module>hudi-client</module>
|
<module>hudi-client</module>
|
||||||
<module>hudi-hadoop-mr</module>
|
<module>hudi-hadoop-mr</module>
|
||||||
<module>hudi-hive-sync</module>
|
|
||||||
<module>hudi-spark</module>
|
<module>hudi-spark</module>
|
||||||
<module>hudi-timeline-service</module>
|
<module>hudi-timeline-service</module>
|
||||||
<module>hudi-utilities</module>
|
<module>hudi-utilities</module>
|
||||||
|
<module>hudi-sync</module>
|
||||||
<module>packaging/hudi-hadoop-mr-bundle</module>
|
<module>packaging/hudi-hadoop-mr-bundle</module>
|
||||||
<module>packaging/hudi-hive-sync-bundle</module>
|
<module>packaging/hudi-hive-sync-bundle</module>
|
||||||
<module>packaging/hudi-spark-bundle</module>
|
<module>packaging/hudi-spark-bundle</module>
|
||||||
|
|||||||
Reference in New Issue
Block a user