rmahindra123 commented on a change in pull request #4175:
URL: https://github.com/apache/hudi/pull/4175#discussion_r820171767



##########
File path: 
hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java
##########
@@ -46,12 +49,17 @@ public void startLocalHiveServiceIfNeeded(Configuration 
configuration) throws IO
   }
 
   public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) {
+    HiveSyncTool hiveSyncTool;
     if (this.config.isHiveLocal()) {
-      writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync()
-          .syncHive(getLocalHiveServer().getHiveConf());
+      hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(),

Review comment:
       this is just a helper function right?  The flow seems a bit weird to 
call the method from DeltaSync. I had removed the method in DeltaSync as well 
since this was the only place it was used. Can discuss more 1:1 if required.

##########
File path: 
hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java
##########
@@ -167,43 +161,10 @@ private void syncMeta() {
     if (connectConfigs.isMetaSyncEnabled()) {
       Set<String> syncClientToolClasses = new HashSet<>(
           Arrays.asList(connectConfigs.getMetaSyncClasses().split(",")));
+      FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
       for (String impl : syncClientToolClasses) {
-        impl = impl.trim();
-        switch (impl) {
-          case "org.apache.hudi.hive.HiveSyncTool":
-            syncHive();
-            break;
-          default:
-            FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
-            Properties properties = new Properties();
-            properties.putAll(connectConfigs.getProps());
-            properties.put("basePath", tableBasePath);
-            AbstractSyncTool syncTool = (AbstractSyncTool) 
ReflectionUtils.loadClass(impl, new Class[] {Properties.class, 
FileSystem.class}, properties, fs);
-            syncTool.syncHoodieTable();
-        }
+        SyncUtilHelpers.createAndSyncHoodieMeta(impl.trim(), 
connectConfigs.getProps(), hadoopConf, fs, tableBasePath, 
HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue());

Review comment:
       thanks :)

##########
File path: 
hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java
##########
@@ -52,48 +41,31 @@
   @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris")
   public String metastoreUris;
 
-  @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 = {"--use-pre-apache-input-format"},
       description = "Use InputFormat under com.uber.hoodie package "
           + "instead of org.apache.hudi package. Use this when you are in the 
process of migrating from "
           + "com.uber.hoodie to org.apache.hudi. Stop using this after you 
migrated the table definition to "
           + "org.apache.hudi input format.")
-  public Boolean usePreApacheInputFormat = false;
+  public Boolean usePreApacheInputFormat;
 
   @Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in 
metastore", required = false)
   public String bucketSpec;
 
   @Deprecated
   @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url")
-  public Boolean useJdbc = true;
+  public Boolean useJdbc;

Review comment:
       These are not flipped. Instead of having the default dictated by these 
variables, and maintaining 2 sets of defaults, I have moved everything to the 
ConfigProperty: See the variable HIVE_USE_JDBC below

##########
File path: 
hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.sync.common;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import com.beust.jcommander.Parameter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Configs needed to sync data into external meta stores, catalogs, etc.
+ */
+public class HoodieSyncConfig extends HoodieConfig {
+
+  public static final String META_SYNC_BASE_PATH = "meta.sync.base.path";
+
+  @Parameter(names = {"--database"}, description = "name of the target 
database in Hive", required = true)
+  public String databaseName;
+
+  @Parameter(names = {"--table"}, description = "name of the target table in 
Hive", required = true)
+  public String tableName;
+
+  @Parameter(names = {"--base-path"}, description = "Basepath of hoodie table 
to sync", required = true)
+  public String basePath;
+
+  @Parameter(names = {"--base-file-format"}, description = "Format of the base 
files (PARQUET (or) HFILE)")
+  public String baseFileFormat;
+
+  @Parameter(names = "--partitioned-by", description = "Fields in the schema 
partitioned by")
+  public List<String> partitionFields;
+
+  @Parameter(names = "--partition-value-extractor", description = "Class which 
implements PartitionValueExtractor "
+      + "to extract the partition values from HDFS path")
+  public String partitionValueExtractorClass;
+
+  @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;
+
+  @Parameter(names = {"--decode-partition"}, description = "Decode the 
partition value if the partition has encoded during writing")
+  public Boolean decodePartition;
+
+  @Parameter(names = {"--use-file-listing-from-metadata"}, description = 
"Fetch file listing from Hudi's metadata")
+  public Boolean useFileListingFromMetadata;
+
+  @Parameter(names = {"--conditional-sync"}, description = "If true, only sync 
on conditions like schema change or partition change.")
+  public Boolean isConditionalSync;
+
+  @Parameter(names = {"--spark-version"}, description = "The spark version", 
required = false)
+  public String sparkVersion;
+
+  public static final ConfigProperty<String> META_SYNC_ENABLED = ConfigProperty
+      .key("hoodie.datasource.meta.sync.enable")
+      .defaultValue("false")
+      .withDocumentation("Enable Syncing the Hudi Table with an external meta 
store or data catalog.");
+
+  // ToDo change the prefix of the following configs from hive_sync to 
meta_sync
+  public static final ConfigProperty<String> META_SYNC_DATABASE_NAME = 
ConfigProperty
+      .key("hoodie.datasource.hive_sync.database")
+      .defaultValue("default")
+      .withDocumentation("The name of the destination database that we should 
sync the hudi table to.");
+
+  // If the table name for the metastore destination is not provided, pick it 
up from write or table configs.
+  public static final Function<HoodieConfig, Option<String>> 
TABLE_NAME_INFERENCE_FUNCTION = cfg -> {
+    if (cfg.contains(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)) {
+      return 
Option.of(cfg.getString(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY));
+    } else if (cfg.contains(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)) {
+      return Option.of(cfg.getString(HoodieTableConfig.HOODIE_TABLE_NAME_KEY));
+    } else {
+      return Option.empty();
+    }
+  };
+  public static final ConfigProperty<String> META_SYNC_TABLE_NAME = 
ConfigProperty
+      .key("hoodie.datasource.hive_sync.table")
+      .defaultValue("unknown")

Review comment:
       Good catch, i wasnt calling setDefaults

##########
File path: 
hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java
##########
@@ -112,16 +117,21 @@ public static void setUp() throws IOException, 
InterruptedException, HiveExcepti
     }
     fileSystem = FileSystem.get(configuration);
 
-    hiveSyncConfig = new HiveSyncConfig();
-    hiveSyncConfig.jdbcUrl = hiveTestService.getJdbcHive2Url();
-    hiveSyncConfig.hiveUser = "";
-    hiveSyncConfig.hivePass = "";
-    hiveSyncConfig.databaseName = "testdb";
-    hiveSyncConfig.tableName = "test1";
-    hiveSyncConfig.basePath = Files.createTempDirectory("hivesynctest" + 
Instant.now().toEpochMilli()).toUri().toString();
-    hiveSyncConfig.assumeDatePartitioning = true;
-    hiveSyncConfig.usePreApacheInputFormat = false;
-    hiveSyncConfig.partitionFields = Collections.singletonList("datestr");
+    basePath = Files.createTempDirectory("hivesynctest" + 
Instant.now().toEpochMilli()).toUri().toString();
+
+    hiveSyncProps = new TypedProperties();
+    hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), 
hiveTestService.getJdbcHive2Url());
+    hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USER.key(), "");
+    hiveSyncProps.setProperty(HiveSyncConfig.HIVE_PASS.key(), "");
+    hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), 
DB_NAME);
+    hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), 
TABLE_NAME);
+    hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_BASE_PATH, basePath);
+    
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), 
"true");
+    
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(),
 "false");
+    hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), 
"datestr");
+    
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM.key(), 
"3");

Review comment:
       Good catch. Thanks for the thorough review. 
   1. I haven't removed it, its set in line 123. 
       hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), 
hiveTestService.getJdbcHive2Url());
   2 For batch sync partition num, literally every test was setting it manually 
to HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; 

##########
File path: 
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
##########
@@ -102,15 +103,15 @@ case class AlterHoodieTableDropPartitionCommand(
         RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
         PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
         PARTITIONPATH_FIELD.key -> partitionFields,
-        HIVE_SYNC_ENABLED.key -> enableHive.toString,
-        META_SYNC_ENABLED.key -> enableHive.toString,
-        HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
-        HIVE_USE_JDBC.key -> "false",
-        HIVE_DATABASE.key -> 
hoodieCatalogTable.table.identifier.database.getOrElse("default"),
-        HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
-        HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
-        HIVE_PARTITION_FIELDS.key -> partitionFields,
-        HIVE_PARTITION_EXTRACTOR_CLASS.key -> 
classOf[MultiPartKeysValueExtractor].getCanonicalName
+        HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,

Review comment:
       So HIVE_SYNC_ENABLED is depracated and META_SYNC_ENABLED should be used 
instead and the meta sync class should be provided as HiveSyncTool. However, 
for backward compatibility, we keep HIVE_SYNC_ENABLED, which by default picks 
the sync class as HiveSyncTool.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to