You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2019/12/28 13:55:59 UTC

[GitHub] [incubator-hudi] pratyakshsharma opened a new pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

pratyakshsharma opened a new pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400021647
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
 ##########
 @@ -152,20 +151,14 @@
    */
   private transient HoodieWriteClient writeClient;
 
-  /**
-   * Table Type.
-   */
-  private final HoodieTableType tableType;
-
   public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider,
-                   HoodieTableType tableType, TypedProperties props, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf,
+                   TypedProperties props, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf,
                    Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
 
     this.cfg = cfg;
     this.jssc = jssc;
     this.sparkSession = sparkSession;
     this.fs = fs;
-    this.tableType = tableType;
 
 Review comment:
   thanks for the cleanup.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399504930
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/TableExecutionObject.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.common.util.TypedProperties;
+
+import java.util.Objects;
+
+/**
+ * Wrapper over TableConfig objects.
+ * Useful for incrementally syncing multiple tables one by one via HoodieMultiTableDeltaStreamer.java class.
+ */
+public class TableExecutionObject {
 
 Review comment:
   Done.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-601717380
 
 
   @bvaradar got chance to take a pass?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362697199
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hive.MultiPartKeysValueExtractor;
+import org.apache.hudi.utilities.deltastreamer.HoodieMultiTableDeltaStreamer;
+import org.apache.hudi.utilities.deltastreamer.TableExecutionObject;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.JsonKafkaSource;
+import org.apache.hudi.utilities.sources.TestDataSource;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.streaming.kafka.KafkaTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestHoodieMultiTableDeltaStreamer extends UtilitiesTestBase {
+
+  private static final String PROPS_FILENAME_TEST_SOURCE = "test-source1.properties";
+  private static volatile Logger log = LogManager.getLogger(TestHoodieMultiTableDeltaStreamer.class);
+  private static KafkaTestUtils testUtils;
+
+  @BeforeClass
+  public static void initClass() throws Exception {
 
 Review comment:
   Can you see if you can reduce duplicate code by extending TestHoodieDeltaStreamer directly ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399541806
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396122286
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static String TRIP_UBER_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripuberrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+  public static String TRIP_FG_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripfgrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+
   public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
   public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,"
-                                                  + "struct<amount:double,currency:string>,boolean";
-
+      + "struct<amount:double,currency:string>,boolean";
   public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
   public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
       HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
+  public static Schema avroFgSchema = new Schema.Parser().parse(TRIP_FG_EXAMPLE_SCHEMA);
 
 Review comment:
   nit: Curious what is FG ? Can you describe in code ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396122037
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -124,6 +138,18 @@ public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths
     }
   }
 
+  public TestRawTripPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   nit: Can you consider defining enum for supported schema instead of comparing schema ?  I would leave it to you to make the final call.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-605297172
 
 
   @gdineshbabu88 Yeah I will update documentation for this tool. That is going to be my next task after taking care of code review comments. You can expect this in 0.6.0 release (which is going to be the next). 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362693772
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   @pratyakshsharma : Since this is used in the context of deltastreamer, it would be better if we move to utilities package

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399503228
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -342,6 +361,32 @@ public static void main(String[] args) throws Exception {
      */
     private transient DeltaSync deltaSync;
 
+    public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf,
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-587113088
 
 
   @pratyakshsharma : Let me know if you need any clarification.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362275299
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   Could be better to re-use the same schema? if you really want two different schema, then its time to modularize this class better?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396128870
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -342,6 +361,32 @@ public static void main(String[] args) throws Exception {
      */
     private transient DeltaSync deltaSync;
 
+    public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf,
 
 Review comment:
   minor : Looks like most of the code in this constructor is same as that of existing constructor. Can you have one constructor call other.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399542370
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
 
 Review comment:
   This is taken care of now. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r404990360
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -169,12 +180,16 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    // TODO: How to obtain hive configs to register?
     @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
     public String targetTableName;
 
     @Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ", required = true)
     public String tableType;
 
+    @Parameter(names = {"--config-folder"}, description = "Path to folder which contains all the properties file", required = true)
 
 Review comment:
   Can we remove this ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399500147
 
 

 ##########
 File path: hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
 ##########
 @@ -121,6 +121,20 @@ public HiveServer2 start() throws IOException {
     return hiveServer;
   }
 
+  public void stop() {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-602030510
 
 
   @bvaradar done. Please have a look. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403746266
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -155,6 +181,19 @@ public static TestRawTripPayload generateRandomValue(
     return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
   }
 
+  /**
+   * Generates a new avro record with TRIP_UBER_EXAMPLE_SCHEMA, retaining the key if optionally provided.
+   */
+  public TestRawTripPayload generatePayloadForUberSchema(HoodieKey key, String commitTime) throws IOException {
+    GenericRecord rec = generateRecordForUberSchema(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
+    return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_UBER_SCHEMA);
+  }
+
+  public TestRawTripPayload generatePayloadForFgSchema(HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   Done

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r391206162
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   So as per the design of our updates and inserts generation in HoodieTestDataGenerator class, we are keeping track of number of keys generated corresponding to a particular schema/topic. So the best that I can do is to have multiple topics actually consume the same schema, but the schema variable needs to be different for every topic. This way I will be able to track the number of keys generated or updated for every topic in a better way. You can have a look at HoodieTestDataGenerator class to get a better idea of what I am trying to say. I have defined the following variables now - 
   
   Map<String, Map<Integer, KeyPartition>> existingKeysBySchema;
   Map<String, Integer> numKeysBySchema;
   
    Let me know your thoughts on this @vinothchandar @bvaradar 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362695705
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -171,6 +186,10 @@ public Operation convert(String value) throws ParameterException {
     public String propsFilePath =
         "file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties";
 
+    @Parameter(names = {"--custom-props"}, description = "path to properties file on localfs or dfs, with configurations for "
 
 Review comment:
   Is this being used ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400006445
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -155,6 +181,19 @@ public static TestRawTripPayload generateRandomValue(
     return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
   }
 
+  /**
+   * Generates a new avro record with TRIP_UBER_EXAMPLE_SCHEMA, retaining the key if optionally provided.
+   */
+  public TestRawTripPayload generatePayloadForUberSchema(HoodieKey key, String commitTime) throws IOException {
+    GenericRecord rec = generateRecordForUberSchema(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
+    return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_UBER_SCHEMA);
+  }
+
+  public TestRawTripPayload generatePayloadForFgSchema(HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   Please remove Fg from this method name 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r379215926
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   That last sentence, can happen on a separate JIRA.. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403746232
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -208,6 +247,31 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam
     return rec;
   }
 
+  /*
+  Generate random record using TRIP_UBER_EXAMPLE_SCHEMA
+   */
+  public GenericRecord generateRecordForUberSchema(String rowKey, String riderName, String driverName, double timestamp) {
+    GenericRecord rec = new GenericData.Record(AVRO_TRIP_SCHEMA);
+    rec.put("_row_key", rowKey);
+    rec.put("timestamp", timestamp);
+    rec.put("rider", riderName);
+    rec.put("driver", driverName);
+    rec.put("fare", RAND.nextDouble() * 100);
+    rec.put("_hoodie_is_deleted", false);
+    return rec;
+  }
+
+  public GenericRecord generateRecordForFgSchema(String rowKey, String riderName, String driverName, double timestamp) {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396133493
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
 
 Review comment:
   Instead of tampering with command line arguments, can you directly create HoodieDeltaStreamer.Config object. This is harder to understand. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399494359
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -286,24 +350,46 @@ public static void createSavepointFile(String basePath, String commitTime, Confi
    * @return  List of {@link HoodieRecord}s
    */
   public List<HoodieRecord> generateInserts(String commitTime, Integer n, boolean isFlattened) {
-    return generateInsertsStream(commitTime, n, isFlattened).collect(Collectors.toList());
+    return generateInsertsStream(commitTime, n, isFlattened, TRIP_EXAMPLE_SCHEMA).collect(Collectors.toList());
+  }
+
+  /**
+   * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
+   */
+  public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, String schemaStr) {
+    int currSize = getNumExistingKeys(schemaStr);
+
+    return IntStream.range(0, n).boxed().map(i -> {
+      String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
+      HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
+      KeyPartition kp = new KeyPartition();
+      kp.key = key;
+      kp.partitionPath = partitionPath;
+      populateKeysBySchema(schemaStr, currSize + i, kp);
+      incrementNumExistingKeysBySchema(schemaStr);
+      try {
+        return new HoodieRecord(key, generateRandomValueAsPerSchema(schemaStr, key, commitTime));
+      } catch (IOException e) {
+        throw new HoodieIOException(e.getMessage(), e);
+      }
+    });
   }
 
   /**
    * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
    */
   public Stream<HoodieRecord> generateInsertsStream(
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r363437991
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -156,6 +167,10 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   From the comment you cited, I think by default we can use the organization where tables share  same prefix but we need to support overrides where we can allow tables to have different paths

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367491239
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/util/DFSTablePropertiesConfiguration.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.util;
+
+import org.apache.hudi.common.model.TableConfig;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Used for parsing custom files having TableConfig objects.
+ */
+public class DFSTablePropertiesConfiguration {
 
 Review comment:
   I did not get this completely, let me try to put forward my understanding. Basically you do not want to have DFSTablePropertiesConfiguration class, rather we can have a separate folder for every table that we want to ingest and every such folder will have the source sink properties needed for ingesting that particular table. In essence, this means I need to redefine the way configs are maintained. Basically I should have every table config similar to how we have configs in DFSPropertiesConfiguration (key value pair). 
   
   Please let me know if I understood it correctly. @bvaradar 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399542571
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
+        successTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      } catch (Exception e) {
+        logger.error("error while running MultiTableDeltaStreamer for table: " + object.getTableName(), e);
+        failedTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      }
+    }
+
+    logger.info("Ingestion was successful for topics: " + successTables);
+    if (!failedTables.isEmpty()) {
+      logger.info("Ingestion failed for topics: " + failedTables);
+    }
+  }
+
+  public static class Constants {
 
 Review comment:
   yeah will take it up next once we are done with code review of this PR. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396132796
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
 
 Review comment:
   this should be replaced with Config class with JCommander parsing.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362696693
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.TableConfig;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTopics;
+  private Set<String> failedTopics;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTopics = new HashSet<>();
+    this.failedTopics = new HashSet<>();
+    this.jssc = jssc;
+    String tableConfigFile = getCustomPropsFileName(args);
+    FileSystem fs = FSUtils.getFs(tableConfigFile, jssc.hadoopConfiguration());
+    List<TableConfig> configList = UtilHelpers.readTableConfig(fs, new Path(tableConfigFile)).getConfigs();
+
+    for (TableConfig config : configList) {
+      validateTableConfigObject(config);
+      populateTableExecutionObjectList(config, args);
+    }
+  }
+
+  /*
+  validate if given object has all the necessary fields.
+  Throws IllegalArgumentException if any of the required fields are missing
+   */
+  private void validateTableConfigObject(TableConfig config) {
+    if (Strings.isNullOrEmpty(config.getDatabase()) || Strings.isNullOrEmpty(config.getTableName()) || Strings.isNullOrEmpty(config.getPrimaryKeyField())
+        || Strings.isNullOrEmpty(config.getTopic())) {
+      throw new IllegalArgumentException("Please provide valid table config arguments!");
+    }
+  }
+
+  private void populateTableExecutionObjectList(TableConfig config, String[] args) {
+    TableExecutionObject executionObject;
+    try {
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, config.getDatabase(), config.getTableName());
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      cfg.targetBasePath = targetBasePath;
+      FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
+      TypedProperties typedProperties = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+      populateIngestionProps(typedProperties, config);
+      populateSchemaProviderProps(cfg, typedProperties, config);
+      populateHiveSyncProps(cfg, typedProperties, config);
+      executionObject = new TableExecutionObject();
+      executionObject.setConfig(cfg);
+      executionObject.setProperties(typedProperties);
+      executionObject.setTableConfig(config);
+      this.tableExecutionObjects.add(executionObject);
+    } catch (Exception e) {
+      logger.error("Error while creating execution object for topic: " + config.getTopic(), e);
+      throw e;
+    }
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+    }
+  }
+
+  private void populateHiveSyncProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.enableHiveSync && Strings.isNullOrEmpty(config.getHiveSyncTable())) {
+      throw new HoodieException("Hive sync table field not provided!");
+    }
+    typedProperties.setProperty(Constants.HIVE_SYNC_TABLE_PROP, config.getHiveSyncTable());
+    typedProperties.setProperty(Constants.HIVE_SYNC_DATABASE_NAME_PROP, Strings.isNullOrEmpty(config.getHiveSyncDatabase())
+        ? typedProperties.getString(Constants.HIVE_SYNC_DATABASE_NAME_PROP, DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL())
+        : config.getHiveSyncDatabase());
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), String.valueOf(config.getAssumeDatePartitioningForHiveSync()));
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(), String.valueOf(config.getUsePreApacheInputFormatForHiveSync()));
+  }
+
+  private void populateIngestionProps(TypedProperties typedProperties, TableConfig config) {
+    typedProperties.setProperty(Constants.KAFKA_TOPIC_PROP, config.getTopic());
+    typedProperties.setProperty(Constants.PARTITION_TIMESTAMP_TYPE_PROP, config.getPartitionTimestampType());
+    typedProperties.setProperty(Constants.PARTITION_FIELD_INPUT_FORMAT_PROP, config.getPartitionInputFormat());
+    typedProperties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), config.getPrimaryKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), config.getPartitionKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), Strings.isNullOrEmpty(config.getKeyGeneratorClassName())
+        ? typedProperties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL())
+        : config.getKeyGeneratorClassName());
+  }
+
+  public static void main(String[] args) {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.YARN_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  /**
+   * Gets customPropsFileName from given args.
+   * @param args
+   * @return
+   */
+  private static String getCustomPropsFileName(String[] args) {
+    String customPropsFileName = "custom_config.json";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CUSTOM_PROPS_FILE_PROP)) {
+        customPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return customPropsFileName;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
 
 Review comment:
   Wouldn't this force serial execution for ingestion ? In that case, we should at-least have it configurable to allow different ingestion jobs to be run in parallel ? Let me know your thoughts.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362799180
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -156,6 +167,10 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   It was done with the idea that the complete path will consist of this base-path-prefix. In essence, complete path is getting created like <basePathPrefix>/<database>/<table_name>.This was initially discussed with @vinothchandar here (https://issues.apache.org/jira/browse/HUDI-288?focusedCommentId=16977695&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16977695).
   I am open to more suggestions @bvaradar . 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r382758933
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   @bvaradar yes since we are going ahead with DFSPropertiesConfiguration, I am going to remove TableConfig class completely. 
   
   Regarding comment having same source, yes I meant same source type. But since source class gets passed as command line parameter (--source-class), it is better to put this restriction. Because we cannot override command line parameter in source <-> sink config key value pairs. If you really want to have this flexibility of overriding source type also for every pair, I will have to define one new property in DataSourceWriteOptions and use it to configure source class. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399646360
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396129463
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/TableExecutionObject.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.common.util.TypedProperties;
+
+import java.util.Objects;
+
+/**
+ * Wrapper over TableConfig objects.
+ * Useful for incrementally syncing multiple tables one by one via HoodieMultiTableDeltaStreamer.java class.
+ */
+public class TableExecutionObject {
 
 Review comment:
   Rename  TableExecutionObject => TableExecutionContext ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400016912
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -155,12 +166,20 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   @pratyakshsharma : Instead of reusing HoodieDeltaStreamer.Config, Can you introduce an inner class HoodieMultiTableDeltaStreamer.Config and have all these configs  (duplicated) along with basePathPrefix. When we instantiate HoodieDeltaStreamer instance in HoodieMultiTableDeltaStreamer, we can create new instance of HoodieDeltaStreamer.Config  by copying all the parameters from HoodieMultiTableDeltaStreamer.Config. This will avoid introducing a config parameter in HoodieDeltaStreamer.Config which has only relevance in HoodieMultiTableDeltaStreamer and keep the CLI parameters for both of them decoupled.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362694773
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/util/DFSTablePropertiesConfiguration.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.util;
+
+import org.apache.hudi.common.model.TableConfig;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Used for parsing custom files having TableConfig objects.
+ */
+public class DFSTablePropertiesConfiguration {
 
 Review comment:
   With HoodieDeltaStreamer, we have 2 kinds of configurations :
   
   1. DFSPropertiesConfiguration passed as config files containing hoodie write and other configurations.
   2. Command-Line parameters (HoodieDeltaStreamer.Config) which contains some source and target specific configurations and some common configurations (like Spark, Hive Server configs...)
   
   With MultiHoodieDeltaStreamer, we essentially need a composite set of above configuration pairs -one per deltastreamer instantiation except for some common configurations in (2). right ? 
   
   Instead of defining a new class at this level (DFSPropertiesConfiguration, can we instead have  a simple convention which is
   
   (1) Keep each pair of source<->sink configuration in separate config folder named by a logical name (e.g: rawtrips in the case of testRawTripPayload) and pass these folder names as command line parameter to MultiDeltaStreamer ? You have to introduce new properties to take in some of source/sink specific command line parameters in HoodieDeltaStreamer.Config. 
   
   The intention is to make the configuration setup reusable when running in MultiDeltaStreamer or as single instance. We also need to be careful with the change so that existing delta-streamer installations do not see the change.  Let me know if this makes sense. 
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r404992549
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
+        successTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      } catch (Exception e) {
+        logger.error("error while running MultiTableDeltaStreamer for table: " + object.getTableName(), e);
+        failedTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      }
+    }
+
+    logger.info("Ingestion was successful for topics: " + successTables);
+    if (!failedTables.isEmpty()) {
+      logger.info("Ingestion failed for topics: " + failedTables);
+    }
+  }
+
+  public static class Constants {
 
 Review comment:
   Filed jira HUDI-766 and assigned to you.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r379215764
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   I think what @bvaradar is trying to get at it is that the current payload is built around the trip schema.. Overall, can we just have multiple topics with the same trips schema? I am not sure what real benefits we get from introducing new schema here.. if we do, I would like to spend more time structuring this class well , since it will make this class even more unclean than what it is today.. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r380322128
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   In TableConfig.java, there are kafka related configs and there are configs like Keygeneratorclass. which are related to upstream source whereas configs related to hive-sync and others apply to the written target dataset.  My understanding is TableConfig is coming from DFSProperties. IMO, decoupling source and table config separately will be cleaner. Let me know if there is any implementation difficulties around it 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-597641119
 
 
   > @pratyakshsharma : Let us know if you need any help on getting this through :) ?
   
   @bvaradar The fixes in this PR depend on https://github.com/apache/incubator-hudi/pull/1395. Let us try to close it as soon as possible to get this through. :) Also I have tried to address most of the things. Please take a pass. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r382758933
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   @bvaradar since we are going ahead with DFSPropertiesConfiguration, I am going to remove TableConfig class completely. 
   
   Regarding comment having same source, yes I meant same source type. But since source class gets passed as command line parameter (--source-class), it is better to put this restriction. Because we cannot override command line parameter in source <-> sink config key value pairs. If you really want to have this flexibility of overriding source type also for every pair, I will have to define one new property in DataSourceWriteOptions and use it to configure source class. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396126493
 
 

 ##########
 File path: hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
 ##########
 @@ -121,6 +121,20 @@ public HiveServer2 start() throws IOException {
     return hiveServer;
   }
 
+  public void stop() {
 
 Review comment:
   Thanks for fixing this. Can you make TestUtil.stop() call this method to prevent leaks. Also, check for other places to see this method needs to be called.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367363989
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   Get your point, will move the class to utilities package. Can you please elaborate what do you mean by TargetTableConfig. All the properties in this class only apply for Source. I guess with few changes in validations part (validateTableConfigObject function in HoodieMultiTableDeltaStreamer.java), the same class can be used for non-kafka sources too? Let me know your thoughts on this. @bvaradar 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396135251
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
 
 Review comment:
   nit: See if you can introduce helper methods for constructing  config file paths.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403746299
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -155,6 +181,19 @@ public static TestRawTripPayload generateRandomValue(
     return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
   }
 
+  /**
+   * Generates a new avro record with TRIP_UBER_EXAMPLE_SCHEMA, retaining the key if optionally provided.
+   */
+  public TestRawTripPayload generatePayloadForUberSchema(HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   Done

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] codecov-io edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-605617268
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=h1) Report
   > Merging [#1150](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-hudi/commit/04449f33feb300b99750c52ec37f2561aa644456&el=desc) will **increase** coverage by `0.19%`.
   > The diff coverage is `74.21%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-hudi/pull/1150/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1150      +/-   ##
   ============================================
   + Coverage     67.48%   67.67%   +0.19%     
   - Complexity      261      290      +29     
   ============================================
     Files           343      350       +7     
     Lines         16565    16825     +260     
     Branches       1693     1707      +14     
   ============================================
   + Hits          11179    11387     +208     
   - Misses         4648     4687      +39     
   - Partials        738      751      +13     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `72.13% <ø> (-0.14%)` | `38.00 <0.00> (ø)` | |
   | [...utilities/deltastreamer/TableExecutionContext.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvVGFibGVFeGVjdXRpb25Db250ZXh0LmphdmE=) | `65.00% <65.00%> (ø)` | `9.00 <9.00> (?)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `73.01% <73.01%> (ø)` | `18.00 <18.00> (?)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `93.25% <100.00%> (+0.07%)` | `0.00 <0.00> (ø)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `81.06% <100.00%> (+0.86%)` | `10.00 <2.00> (+2.00)` | |
   | [...rg/apache/hudi/io/storage/HoodieParquetWriter.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW8vc3RvcmFnZS9Ib29kaWVQYXJxdWV0V3JpdGVyLmphdmE=) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...he/hudi/io/storage/HoodieStorageWriterFactory.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW8vc3RvcmFnZS9Ib29kaWVTdG9yYWdlV3JpdGVyRmFjdG9yeS5qYXZh) | `93.75% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [.../hudi/execution/MergeOnReadLazyInsertIterable.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZXhlY3V0aW9uL01lcmdlT25SZWFkTGF6eUluc2VydEl0ZXJhYmxlLmphdmE=) | `64.70% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...ache/hudi/common/inline/fs/InMemoryFileSystem.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2lubGluZS9mcy9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...hudi/common/inline/fs/InLineFsDataInputStream.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2lubGluZS9mcy9JbkxpbmVGc0RhdGFJbnB1dFN0cmVhbS5qYXZh) | `53.84% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | ... and [10 more](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=footer). Last update [04449f3...e0eb241](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396121811
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static String TRIP_UBER_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripuberrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+  public static String TRIP_FG_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripfgrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+
   public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
   public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,"
-                                                  + "struct<amount:double,currency:string>,boolean";
-
+      + "struct<amount:double,currency:string>,boolean";
   public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
   public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
       HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
+  public static Schema avroFgSchema = new Schema.Parser().parse(TRIP_FG_EXAMPLE_SCHEMA);
+  public static Schema avroUberSchema = new Schema.Parser().parse(TRIP_UBER_EXAMPLE_SCHEMA);
   public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
 
-  private static final Random RAND = new Random(46474747);
+  private static Random rand = new Random(46474747);
 
 Review comment:
   Why is this no longer final ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r380318911
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/util/DFSTablePropertiesConfiguration.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.util;
+
+import org.apache.hudi.common.model.TableConfig;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Used for parsing custom files having TableConfig objects.
+ */
+public class DFSTablePropertiesConfiguration {
 
 Review comment:
   Yes

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-605618890
 
 
   @bvaradar Please take a pass. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-586060460
 
 
   @pratyakshsharma  lets revive this ? :) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400006741
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -208,6 +247,31 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam
     return rec;
   }
 
+  /*
+  Generate random record using TRIP_UBER_EXAMPLE_SCHEMA
+   */
+  public GenericRecord generateRecordForUberSchema(String rowKey, String riderName, String driverName, double timestamp) {
 
 Review comment:
   Same here

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400025762
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionContext> tableExecutionContexts;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(Config config, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionContexts = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = config.propsFilePath;
+    String configFolder = config.configFolder;
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionContextList(properties, configFolder, fs, config);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionContextList(TypedProperties properties, String configFolder, FileSystem fs, Config config) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    logger.info("tables to be ingested via MultiTableDeltaStreamer : " + tablesToBeIngested);
+    TableExecutionContext executionContext;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable));
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      //copy all the values from config to cfgF
+      String targetBasePath = resetTarget(config, database, currentTable);
+      Helpers.deepCopyConfigs(config, cfg);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = StringUtils.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && StringUtils.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionContext = new TableExecutionContext();
+      executionContext.setProperties(tableProperties);
+      executionContext.setConfig(cfg);
+      executionContext.setDatabase(database);
+      executionContext.setTableName(currentTable);
+      this.tableExecutionContexts.add(executionContext);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(Constants.COMMA_SEPARATOR);
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static class Helpers {
+
+    static String getDefaultConfigFilePath(String configFolder, String database, String currentTable) {
+      return configFolder + Constants.FILEDELIMITER + database + Constants.UNDERSCORE + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX;
+    }
+
+    static String getTableWithDatabase(TableExecutionContext context) {
+      return context.getDatabase() + Constants.DELIMITER + context.getTableName();
+    }
+
+    static void deepCopyConfigs(Config globalConfig, Config tableConfig) {
+      tableConfig.enableHiveSync = globalConfig.enableHiveSync;
+      tableConfig.schemaProviderClassName = globalConfig.schemaProviderClassName;
+      tableConfig.sourceOrderingField = globalConfig.sourceOrderingField;
+      tableConfig.sourceClassName = globalConfig.sourceClassName;
+      tableConfig.tableType = globalConfig.tableType;
+      tableConfig.propsFilePath = globalConfig.propsFilePath;
+      tableConfig.basePathPrefix = globalConfig.basePathPrefix;
+      tableConfig.targetTableName = globalConfig.targetTableName;
+      tableConfig.configFolder = globalConfig.configFolder;
+      tableConfig.operation = globalConfig.operation;
+      tableConfig.sourceLimit = globalConfig.sourceLimit;
+      tableConfig.checkpoint = globalConfig.checkpoint;
+      tableConfig.continuousMode = globalConfig.continuousMode;
+      tableConfig.filterDupes = globalConfig.filterDupes;
+      tableConfig.payloadClassName = globalConfig.payloadClassName;
+      tableConfig.configs = globalConfig.configs;
+      tableConfig.forceDisableCompaction = globalConfig.forceDisableCompaction;
+      tableConfig.maxPendingCompactions = globalConfig.maxPendingCompactions;
+      tableConfig.minSyncIntervalSeconds = globalConfig.minSyncIntervalSeconds;
+      tableConfig.transformerClassName = globalConfig.transformerClassName;
+      tableConfig.commitOnErrors = globalConfig.commitOnErrors;
+      tableConfig.compactSchedulingMinShare = globalConfig.compactSchedulingMinShare;
+      tableConfig.compactSchedulingWeight = globalConfig.compactSchedulingWeight;
+      tableConfig.deltaSyncSchedulingMinShare = globalConfig.deltaSyncSchedulingMinShare;
+      tableConfig.deltaSyncSchedulingWeight = globalConfig.deltaSyncSchedulingWeight;
+      tableConfig.sparkMaster = globalConfig.sparkMaster;
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    final Config config = new Config();
 
 Review comment:
   Just to reiterate my comment related to configs, Can you create a new inner Config class for HoodieMultiTableDeltaStreamer with JCommander arguments to decouple from HoodieDeltaStreamer.Config. This should avoid keeping basePathPrefix in HoodieDeltaStreamer.Config 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400005860
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static final String TRIP_UBER_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":["
 
 Review comment:
   Lets not add the name uber in these example schema as they are generic. Just keep TRIP_SCHEMA and SHORT_TRIP_SCHEMA

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r380555306
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   @bvaradar If we assume TableConfig is coming from DFSProperties (I am doing that change anyways :) ), then what extra benefit are we getting by decoupling source and target configs?  Because implementation wise I will be merging source and target configs into single TypedProperties instance after reading them separately since everywhere we are passing only single TypedProperties instance for reading relevant configs. 
   
   If TableConfig is read as DFSProperties instance (key-value pair), then non-kafka sources are automatically handled. I understand maintaining source and target configs separately is cleaner, but then it will be an extra overhead for users to maintain 2 separate files. 
   
   Please help me understand. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403746251
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -208,6 +247,31 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam
     return rec;
   }
 
+  /*
+  Generate random record using TRIP_UBER_EXAMPLE_SCHEMA
+   */
+  public GenericRecord generateRecordForUberSchema(String rowKey, String riderName, String driverName, double timestamp) {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r384706441
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   Sounds good on removing TableConfig. Since, HoodieMultiTableDeltaStreamer is going to be a new main class, is it possible to avoid source-class to be a command-line-parameter and instead be part of the DFSProperties ? Anyways, lets not get too bogged down by it for now. Once you have the changes ready, will do another pass. If minor, we can try to do those as follow-ups. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r379215764
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   I think what @bvaradar is trying to get at it is that the current payload is built around the trip schema.. Overall, can we just have multiple topics with the same trips schema? I am not sure what real benefits we get from introducing new schema here.. if you feel strongly still, I would like to spend more time structuring this class well , since it will make this class even more unclean than what it is today.. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-574505738
 
 
   @vinothchandar @bvaradar Sorry was busy with some other stuff. Today I will try to address most of the comments here. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367363989
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   Get your point, will move the class to utilities package. Can you please elaborate what do you mean by TargetTableConfig. All the properties in this class only apply for Source. I guess with few changes in validations part, the same class can be used for non-kafka sources too? Let me know your thoughts on this. @bvaradar 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399646598
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -124,6 +138,18 @@ public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths
     }
   }
 
+  public TestRawTripPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   I feel we do not gain much by defining enums here :) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r381549268
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   For example, for DFS source we need to specify hoodie.deltastreamer.source.dfs.root property. This can be specified in corresponding table config coming as DFSProperties instance. 
   
   Again I am assuming in a single installation of MultiTableDeltaStreamer, all the tables will use same source (which I believe is a valid assumption to make). Let me know your thoughts on this. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403740013
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionContext> tableExecutionContexts;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(Config config, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionContexts = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = config.propsFilePath;
+    String configFolder = config.configFolder;
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionContextList(properties, configFolder, fs, config);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionContextList(TypedProperties properties, String configFolder, FileSystem fs, Config config) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    logger.info("tables to be ingested via MultiTableDeltaStreamer : " + tablesToBeIngested);
+    TableExecutionContext executionContext;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable));
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      //copy all the values from config to cfgF
+      String targetBasePath = resetTarget(config, database, currentTable);
+      Helpers.deepCopyConfigs(config, cfg);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = StringUtils.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && StringUtils.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionContext = new TableExecutionContext();
+      executionContext.setProperties(tableProperties);
+      executionContext.setConfig(cfg);
+      executionContext.setDatabase(database);
+      executionContext.setTableName(currentTable);
+      this.tableExecutionContexts.add(executionContext);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(Constants.COMMA_SEPARATOR);
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static class Helpers {
+
+    static String getDefaultConfigFilePath(String configFolder, String database, String currentTable) {
+      return configFolder + Constants.FILEDELIMITER + database + Constants.UNDERSCORE + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX;
+    }
+
+    static String getTableWithDatabase(TableExecutionContext context) {
+      return context.getDatabase() + Constants.DELIMITER + context.getTableName();
+    }
+
+    static void deepCopyConfigs(Config globalConfig, Config tableConfig) {
+      tableConfig.enableHiveSync = globalConfig.enableHiveSync;
+      tableConfig.schemaProviderClassName = globalConfig.schemaProviderClassName;
+      tableConfig.sourceOrderingField = globalConfig.sourceOrderingField;
+      tableConfig.sourceClassName = globalConfig.sourceClassName;
+      tableConfig.tableType = globalConfig.tableType;
+      tableConfig.propsFilePath = globalConfig.propsFilePath;
+      tableConfig.basePathPrefix = globalConfig.basePathPrefix;
+      tableConfig.targetTableName = globalConfig.targetTableName;
+      tableConfig.configFolder = globalConfig.configFolder;
+      tableConfig.operation = globalConfig.operation;
+      tableConfig.sourceLimit = globalConfig.sourceLimit;
+      tableConfig.checkpoint = globalConfig.checkpoint;
+      tableConfig.continuousMode = globalConfig.continuousMode;
+      tableConfig.filterDupes = globalConfig.filterDupes;
+      tableConfig.payloadClassName = globalConfig.payloadClassName;
+      tableConfig.configs = globalConfig.configs;
+      tableConfig.forceDisableCompaction = globalConfig.forceDisableCompaction;
+      tableConfig.maxPendingCompactions = globalConfig.maxPendingCompactions;
+      tableConfig.minSyncIntervalSeconds = globalConfig.minSyncIntervalSeconds;
+      tableConfig.transformerClassName = globalConfig.transformerClassName;
+      tableConfig.commitOnErrors = globalConfig.commitOnErrors;
+      tableConfig.compactSchedulingMinShare = globalConfig.compactSchedulingMinShare;
+      tableConfig.compactSchedulingWeight = globalConfig.compactSchedulingWeight;
+      tableConfig.deltaSyncSchedulingMinShare = globalConfig.deltaSyncSchedulingMinShare;
+      tableConfig.deltaSyncSchedulingWeight = globalConfig.deltaSyncSchedulingWeight;
+      tableConfig.sparkMaster = globalConfig.sparkMaster;
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    final Config config = new Config();
 
 Review comment:
   Sure. Initially I was thinking of not creating the Config class in HoodieMultiTableDeltaStreamer as I thought it is going to be a duplicate of HoodieDeltaStreamer.Config class and hence leads to code duplication. But since you want to decouple the 2 classes, will do that in my next commit. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399542132
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-575461848
 
 
   @bvaradar Have asked for clarification on 2-3 comments. Otherwise give me some time to fix the failing test cases. Things broke after rebasing with master.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] codecov-io edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-605617268
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=h1) Report
   > Merging [#1150](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-hudi/commit/eaf6cc2d90bf27c0d9414a4ea18dbd1b61f58e50&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `78.06%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-hudi/pull/1150/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1150      +/-   ##
   ============================================
   + Coverage     71.54%   71.64%   +0.09%     
   - Complexity      261      290      +29     
   ============================================
     Files           336      338       +2     
     Lines         15744    15931     +187     
     Branches       1610     1625      +15     
   ============================================
   + Hits          11264    11413     +149     
   - Misses         3759     3785      +26     
   - Partials        721      733      +12     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../org/apache/hudi/table/HoodieCommitArchiveLog.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdGFibGUvSG9vZGllQ29tbWl0QXJjaGl2ZUxvZy5qYXZh) | `77.48% <ø> (+2.48%)` | `0.00 <0.00> (ø)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `71.85% <ø> (-0.15%)` | `38.00 <0.00> (ø)` | |
   | [...utilities/deltastreamer/TableExecutionContext.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvVGFibGVFeGVjdXRpb25Db250ZXh0LmphdmE=) | `65.00% <65.00%> (ø)` | `9.00 <9.00> (?)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `78.88% <78.88%> (ø)` | `18.00 <18.00> (?)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `78.77% <85.71%> (+0.16%)` | `10.00 <2.00> (+2.00)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `93.25% <100.00%> (+0.07%)` | `0.00 <0.00> (ø)` | |
   | [...n/java/org/apache/hudi/common/model/HoodieKey.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZUtleS5qYXZh) | `88.88% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=footer). Last update [eaf6cc2...1eece0d](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396132561
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
 
 Review comment:
   Instead of passing String[] args, Can you define a  inner Config class for HoodieMultiTableDeltaStreamer in the same way done for HoodieDeltaStreamer ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396135161
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
 
 Review comment:
   minor: Log the tables to be ingested ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362386402
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   @vinothchandar I wanted to depict a real-life scenario in test-cases. Hence preferred to have different schemas for every topic. By modularizing the class, you mean I should write a class which extends this class?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399372702
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static String TRIP_UBER_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripuberrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+  public static String TRIP_FG_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripfgrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+
   public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
   public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,"
-                                                  + "struct<amount:double,currency:string>,boolean";
-
+      + "struct<amount:double,currency:string>,boolean";
   public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
   public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
       HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
+  public static Schema avroFgSchema = new Schema.Parser().parse(TRIP_FG_EXAMPLE_SCHEMA);
 
 Review comment:
   Actually that is the initials of my current organisation. While testing I put it like that and forgot to modify later. Will change the name. :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-596629299
 
 
   @pratyakshsharma : Let us know if you need any help on getting this through :) ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r379216693
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/util/DFSTablePropertiesConfiguration.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.util;
+
+import org.apache.hudi.common.model.TableConfig;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Used for parsing custom files having TableConfig objects.
+ */
+public class DFSTablePropertiesConfiguration {
 
 Review comment:
   I think your understanding is correct.. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-574501169
 
 
   @pratyakshsharma  any updates? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-571280176
 
 
   @pratyakshsharma : Let me know once you addressed the comments. 
   
   Thanks,
   Balaji.V

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r403740566
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static final String TRIP_UBER_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":["
 
 Review comment:
   Sure. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400006959
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -208,6 +247,31 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam
     return rec;
   }
 
+  /*
+  Generate random record using TRIP_UBER_EXAMPLE_SCHEMA
+   */
+  public GenericRecord generateRecordForUberSchema(String rowKey, String riderName, String driverName, double timestamp) {
+    GenericRecord rec = new GenericData.Record(AVRO_TRIP_SCHEMA);
+    rec.put("_row_key", rowKey);
+    rec.put("timestamp", timestamp);
+    rec.put("rider", riderName);
+    rec.put("driver", driverName);
+    rec.put("fare", RAND.nextDouble() * 100);
+    rec.put("_hoodie_is_deleted", false);
+    return rec;
+  }
+
+  public GenericRecord generateRecordForFgSchema(String rowKey, String riderName, String driverName, double timestamp) {
 
 Review comment:
   Same rename here. Please check for other method names as well. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362693191
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   @pratyakshsharma : I would also vote to keep one schema. If you want to add different schema, you would need to encapsulate the schemas separately, have its own RecordPayload or use generic one (not TestRawTripPayload). Maybe add a nullable column in TestRawtripPayload to distinguish the topic from which the record got ingested ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367371389
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -156,6 +167,10 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   Makes sense, will add the support.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399541700
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362695652
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -156,6 +167,10 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   why do we need to enforce that add destination base-paths needs to share same prefix ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-609647685
 
 
   @bvaradar Have addressed the comments. Please take a pass. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r405053101
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
+        successTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      } catch (Exception e) {
+        logger.error("error while running MultiTableDeltaStreamer for table: " + object.getTableName(), e);
+        failedTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      }
+    }
+
+    logger.info("Ingestion was successful for topics: " + successTables);
+    if (!failedTables.isEmpty()) {
+      logger.info("Ingestion failed for topics: " + failedTables);
+    }
+  }
+
+  public static class Constants {
 
 Review comment:
   Will take care of that :) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399518706
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -155,12 +166,20 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   Yes, this is needed for HoodieMultiTableDeltaStreamer. Now I am defining a separate config class for MultiTableStreamer. All unused configs can be removed from HoodieDeltaStreamer.Config class. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r363437075
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.TableConfig;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTopics;
+  private Set<String> failedTopics;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTopics = new HashSet<>();
+    this.failedTopics = new HashSet<>();
+    this.jssc = jssc;
+    String tableConfigFile = getCustomPropsFileName(args);
+    FileSystem fs = FSUtils.getFs(tableConfigFile, jssc.hadoopConfiguration());
+    List<TableConfig> configList = UtilHelpers.readTableConfig(fs, new Path(tableConfigFile)).getConfigs();
+
+    for (TableConfig config : configList) {
+      validateTableConfigObject(config);
+      populateTableExecutionObjectList(config, args);
+    }
+  }
+
+  /*
+  validate if given object has all the necessary fields.
+  Throws IllegalArgumentException if any of the required fields are missing
+   */
+  private void validateTableConfigObject(TableConfig config) {
+    if (Strings.isNullOrEmpty(config.getDatabase()) || Strings.isNullOrEmpty(config.getTableName()) || Strings.isNullOrEmpty(config.getPrimaryKeyField())
+        || Strings.isNullOrEmpty(config.getTopic())) {
+      throw new IllegalArgumentException("Please provide valid table config arguments!");
+    }
+  }
+
+  private void populateTableExecutionObjectList(TableConfig config, String[] args) {
+    TableExecutionObject executionObject;
+    try {
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, config.getDatabase(), config.getTableName());
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      cfg.targetBasePath = targetBasePath;
+      FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
+      TypedProperties typedProperties = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+      populateIngestionProps(typedProperties, config);
+      populateSchemaProviderProps(cfg, typedProperties, config);
+      populateHiveSyncProps(cfg, typedProperties, config);
+      executionObject = new TableExecutionObject();
+      executionObject.setConfig(cfg);
+      executionObject.setProperties(typedProperties);
+      executionObject.setTableConfig(config);
+      this.tableExecutionObjects.add(executionObject);
+    } catch (Exception e) {
+      logger.error("Error while creating execution object for topic: " + config.getTopic(), e);
+      throw e;
+    }
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+    }
+  }
+
+  private void populateHiveSyncProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.enableHiveSync && Strings.isNullOrEmpty(config.getHiveSyncTable())) {
+      throw new HoodieException("Hive sync table field not provided!");
+    }
+    typedProperties.setProperty(Constants.HIVE_SYNC_TABLE_PROP, config.getHiveSyncTable());
+    typedProperties.setProperty(Constants.HIVE_SYNC_DATABASE_NAME_PROP, Strings.isNullOrEmpty(config.getHiveSyncDatabase())
+        ? typedProperties.getString(Constants.HIVE_SYNC_DATABASE_NAME_PROP, DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL())
+        : config.getHiveSyncDatabase());
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), String.valueOf(config.getAssumeDatePartitioningForHiveSync()));
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(), String.valueOf(config.getUsePreApacheInputFormatForHiveSync()));
+  }
+
+  private void populateIngestionProps(TypedProperties typedProperties, TableConfig config) {
+    typedProperties.setProperty(Constants.KAFKA_TOPIC_PROP, config.getTopic());
+    typedProperties.setProperty(Constants.PARTITION_TIMESTAMP_TYPE_PROP, config.getPartitionTimestampType());
+    typedProperties.setProperty(Constants.PARTITION_FIELD_INPUT_FORMAT_PROP, config.getPartitionInputFormat());
+    typedProperties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), config.getPrimaryKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), config.getPartitionKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), Strings.isNullOrEmpty(config.getKeyGeneratorClassName())
+        ? typedProperties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL())
+        : config.getKeyGeneratorClassName());
+  }
+
+  public static void main(String[] args) {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.YARN_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  /**
+   * Gets customPropsFileName from given args.
+   * @param args
+   * @return
+   */
+  private static String getCustomPropsFileName(String[] args) {
+    String customPropsFileName = "custom_config.json";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CUSTOM_PROPS_FILE_PROP)) {
+        customPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return customPropsFileName;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
 
 Review comment:
   ok. Should be fine for first round.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399519630
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
 
 Review comment:
   Done

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] codecov-io commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-605617268
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=h1) Report
   > Merging [#1150](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-hudi/commit/04449f33feb300b99750c52ec37f2561aa644456&el=desc) will **increase** coverage by `0.19%`.
   > The diff coverage is `74.21%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-hudi/pull/1150/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1150      +/-   ##
   ============================================
   + Coverage     67.48%   67.67%   +0.19%     
   - Complexity      261      290      +29     
   ============================================
     Files           343      350       +7     
     Lines         16565    16825     +260     
     Branches       1693     1707      +14     
   ============================================
   + Hits          11179    11387     +208     
   - Misses         4648     4687      +39     
   - Partials        738      751      +13     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `72.13% <ø> (-0.14%)` | `38.00 <0.00> (ø)` | |
   | [...utilities/deltastreamer/TableExecutionContext.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvVGFibGVFeGVjdXRpb25Db250ZXh0LmphdmE=) | `65.00% <65.00%> (ø)` | `9.00 <9.00> (?)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `73.01% <73.01%> (ø)` | `18.00 <18.00> (?)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `93.25% <100.00%> (+0.07%)` | `0.00 <0.00> (ø)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `81.06% <100.00%> (+0.86%)` | `10.00 <2.00> (+2.00)` | |
   | [...rg/apache/hudi/io/storage/HoodieParquetWriter.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW8vc3RvcmFnZS9Ib29kaWVQYXJxdWV0V3JpdGVyLmphdmE=) | `100.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...he/hudi/io/storage/HoodieStorageWriterFactory.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW8vc3RvcmFnZS9Ib29kaWVTdG9yYWdlV3JpdGVyRmFjdG9yeS5qYXZh) | `93.75% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [.../hudi/execution/MergeOnReadLazyInsertIterable.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jbGllbnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZXhlY3V0aW9uL01lcmdlT25SZWFkTGF6eUluc2VydEl0ZXJhYmxlLmphdmE=) | `64.70% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...ache/hudi/common/inline/fs/InMemoryFileSystem.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2lubGluZS9mcy9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...hudi/common/inline/fs/InLineFsDataInputStream.java](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2lubGluZS9mcy9JbkxpbmVGc0RhdGFJbnB1dFN0cmVhbS5qYXZh) | `53.84% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | ... and [10 more](https://codecov.io/gh/apache/incubator-hudi/pull/1150/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=footer). Last update [04449f3...e0eb241](https://codecov.io/gh/apache/incubator-hudi/pull/1150?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396135755
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
 
 Review comment:
   Instead of cloning raw args here, can we copy necessary parameters from HoodieMultiDeltaStreamer.Config (to be introduced) to HodieDeltaStreamer.Config ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r380490349
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   Ok, got the point. I will get back to you on this @vinothchandar .

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar merged pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar merged pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396132803
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
 
 Review comment:
   Same here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396135966
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTables = new HashSet<>();
+    this.failedTables = new HashSet<>();
+    this.jssc = jssc;
+    String commonPropsFile = getCommonPropsFileName(args);
+    String configFolder = getConfigFolder(args);
+    FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
+    configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
+    checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
+    TypedProperties properties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getConfig();
+    //get the tables to be ingested and their corresponding config files from this properties instance
+    populateTableExecutionObjectList(properties, configFolder, fs, args);
+  }
+
+  private void checkIfPropsFileAndConfigFolderExist(String commonPropsFile, String configFolder, FileSystem fs) throws IOException {
+    if (!fs.exists(new Path(commonPropsFile))) {
+      throw new IllegalArgumentException("Please provide valid common config file path!");
+    }
+
+    if (!fs.exists(new Path(configFolder))) {
+      fs.mkdirs(new Path(configFolder));
+    }
+  }
+
+  private void checkIfTableConfigFileExists(String configFolder, FileSystem fs, String configFilePath) throws IOException {
+    if (!fs.exists(new Path(configFilePath)) || !fs.isFile(new Path(configFilePath))) {
+      throw new IllegalArgumentException("Please provide valid table config file path!");
+    }
+
+    Path path = new Path(configFilePath);
+    Path filePathInConfigFolder = new Path(configFolder, path.getName());
+    if (!fs.exists(filePathInConfigFolder)) {
+      FileUtil.copy(fs, path, fs, filePathInConfigFolder, false, fs.getConf());
+    }
+  }
+
+  //commonProps are passed as parameter which contain table to config file mapping
+  private void populateTableExecutionObjectList(TypedProperties properties, String configFolder, FileSystem fs, String[] args) throws IOException {
+    List<String> tablesToBeIngested = getTablesToBeIngested(properties);
+    TableExecutionObject executionObject;
+    for (String table : tablesToBeIngested) {
+      String[] tableWithDatabase = table.split("\\.");
+      String database = tableWithDatabase.length > 1 ? tableWithDatabase[0] : "default";
+      String currentTable = tableWithDatabase.length > 1 ? tableWithDatabase[1] : table;
+      String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
+      String configFilePath = properties.getString(configProp, configFolder + "/" + database + "_" + currentTable + Constants.DEFAULT_CONFIG_FILE_NAME_SUFFIX);
+      checkIfTableConfigFileExists(configFolder, fs, configFilePath);
+      TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getConfig();
+      properties.forEach((k,v) -> {
+        tableProperties.setProperty(k.toString(), v.toString());
+      });
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, database, currentTable);
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
+      cfg.targetBasePath = Strings.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
+      if (cfg.enableHiveSync && Strings.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), ""))) {
+        throw new HoodieException("Hive sync table field not provided!");
+      }
+      populateSchemaProviderProps(cfg, tableProperties);
+      executionObject = new TableExecutionObject();
+      executionObject.setProperties(tableProperties);
+      executionObject.setConfig(cfg);
+      executionObject.setDatabase(database);
+      executionObject.setTableName(currentTable);
+      this.tableExecutionObjects.add(executionObject);
+    }
+  }
+
+  private List<String> getTablesToBeIngested(TypedProperties properties) {
+    String combinedTablesString = properties.getString(Constants.TABLES_TO_BE_INGESTED_PROP);
+    if (combinedTablesString == null) {
+      return new ArrayList<>();
+    }
+    String[] tablesArray = combinedTablesString.split(",");
+    return Arrays.asList(tablesArray);
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + typedProperties.getString(Constants.KAFKA_TOPIC_PROP) + schemaRegistrySuffix);
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.LOCAL_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  private static String getCommonPropsFileName(String[] args) {
+    String commonPropsFileName = "common_props.properties";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.PROPS_FILE_PROP)) {
+        commonPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return commonPropsFileName;
+  }
+
+  private static String getConfigFolder(String[] args) {
+    String configFolder = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CONFIG_FOLDER_PROP)) {
+        configFolder = args[i + 1];
+        break;
+      }
+    }
+    return configFolder;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
+        successTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      } catch (Exception e) {
+        logger.error("error while running MultiTableDeltaStreamer for table: " + object.getTableName(), e);
+        failedTables.add(object.getDatabase() + Constants.DELIMITER + object.getTableName());
+      }
+    }
+
+    logger.info("Ingestion was successful for topics: " + successTables);
+    if (!failedTables.isEmpty()) {
+      logger.info("Ingestion failed for topics: " + failedTables);
+    }
+  }
+
+  public static class Constants {
 
 Review comment:
   We would need to document these  properties and add a section for HoodieMultiDeltaStreamer just like https://hudi.incubator.apache.org/docs/writing_data.html#deltastreamer. This can be done as a followup.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362799979
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.TableConfig;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTopics;
+  private Set<String> failedTopics;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTopics = new HashSet<>();
+    this.failedTopics = new HashSet<>();
+    this.jssc = jssc;
+    String tableConfigFile = getCustomPropsFileName(args);
+    FileSystem fs = FSUtils.getFs(tableConfigFile, jssc.hadoopConfiguration());
+    List<TableConfig> configList = UtilHelpers.readTableConfig(fs, new Path(tableConfigFile)).getConfigs();
+
+    for (TableConfig config : configList) {
+      validateTableConfigObject(config);
+      populateTableExecutionObjectList(config, args);
+    }
+  }
+
+  /*
+  validate if given object has all the necessary fields.
+  Throws IllegalArgumentException if any of the required fields are missing
+   */
+  private void validateTableConfigObject(TableConfig config) {
+    if (Strings.isNullOrEmpty(config.getDatabase()) || Strings.isNullOrEmpty(config.getTableName()) || Strings.isNullOrEmpty(config.getPrimaryKeyField())
+        || Strings.isNullOrEmpty(config.getTopic())) {
+      throw new IllegalArgumentException("Please provide valid table config arguments!");
+    }
+  }
+
+  private void populateTableExecutionObjectList(TableConfig config, String[] args) {
+    TableExecutionObject executionObject;
+    try {
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, config.getDatabase(), config.getTableName());
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      cfg.targetBasePath = targetBasePath;
+      FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
+      TypedProperties typedProperties = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+      populateIngestionProps(typedProperties, config);
+      populateSchemaProviderProps(cfg, typedProperties, config);
+      populateHiveSyncProps(cfg, typedProperties, config);
+      executionObject = new TableExecutionObject();
+      executionObject.setConfig(cfg);
+      executionObject.setProperties(typedProperties);
+      executionObject.setTableConfig(config);
+      this.tableExecutionObjects.add(executionObject);
+    } catch (Exception e) {
+      logger.error("Error while creating execution object for topic: " + config.getTopic(), e);
+      throw e;
+    }
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+    }
+  }
+
+  private void populateHiveSyncProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.enableHiveSync && Strings.isNullOrEmpty(config.getHiveSyncTable())) {
+      throw new HoodieException("Hive sync table field not provided!");
+    }
+    typedProperties.setProperty(Constants.HIVE_SYNC_TABLE_PROP, config.getHiveSyncTable());
+    typedProperties.setProperty(Constants.HIVE_SYNC_DATABASE_NAME_PROP, Strings.isNullOrEmpty(config.getHiveSyncDatabase())
+        ? typedProperties.getString(Constants.HIVE_SYNC_DATABASE_NAME_PROP, DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL())
+        : config.getHiveSyncDatabase());
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), String.valueOf(config.getAssumeDatePartitioningForHiveSync()));
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(), String.valueOf(config.getUsePreApacheInputFormatForHiveSync()));
+  }
+
+  private void populateIngestionProps(TypedProperties typedProperties, TableConfig config) {
+    typedProperties.setProperty(Constants.KAFKA_TOPIC_PROP, config.getTopic());
+    typedProperties.setProperty(Constants.PARTITION_TIMESTAMP_TYPE_PROP, config.getPartitionTimestampType());
+    typedProperties.setProperty(Constants.PARTITION_FIELD_INPUT_FORMAT_PROP, config.getPartitionInputFormat());
+    typedProperties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), config.getPrimaryKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), config.getPartitionKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), Strings.isNullOrEmpty(config.getKeyGeneratorClassName())
+        ? typedProperties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL())
+        : config.getKeyGeneratorClassName());
+  }
+
+  public static void main(String[] args) {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.YARN_SPARK_MASTER);
+    try {
+      new HoodieMultiTableDeltaStreamer(args, jssc).sync();
+    } finally {
+      jssc.stop();
+    }
+  }
+
+  /**
+   * Gets customPropsFileName from given args.
+   * @param args
+   * @return
+   */
+  private static String getCustomPropsFileName(String[] args) {
+    String customPropsFileName = "custom_config.json";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.CUSTOM_PROPS_FILE_PROP)) {
+        customPropsFileName = args[i + 1];
+        break;
+      }
+    }
+    return customPropsFileName;
+  }
+
+  /**
+   * Resets target table name and target path using base-path-prefix.
+   * @param args
+   * @param database
+   * @param tableName
+   * @return
+   */
+  private static String resetTarget(String[] args, String database, String tableName) {
+    int counter = 0;
+    String targetBasePath = "";
+    for (int i = 0; i < args.length; i++) {
+      if (args[i].equals(Constants.BASE_PATH_PREFIX_PROP)) {
+        args[i + 1] = args[i + 1].charAt(args[i + 1].length() - 1) == '/' ? args[i + 1].substring(0, args[i + 1].length() - 1) : args[i + 1];
+        targetBasePath = args[i + 1] + Constants.FILEDELIMITER + database + Constants.FILEDELIMITER + tableName;
+        counter += 1;
+      } else if (args[i].equals(Constants.TARGET_TABLE_ARG)) {
+        args[i + 1] = database + Constants.DELIMITER + tableName;
+        counter += 1;
+      }
+      if (counter == 2) {
+        break;
+      }
+    }
+
+    return targetBasePath;
+  }
+
+  /*
+  Creates actual HoodieDeltaStreamer objects for every table/topic and does incremental sync
+   */
+  public void sync() {
+    for (TableExecutionObject object : tableExecutionObjects) {
+      try {
+        new HoodieDeltaStreamer(object.getConfig(), jssc, object.getProperties()).sync();
 
 Review comment:
   Actually we are planning to handle only COPY_ON_WRITE storage type in phase-1. This was discussed here (https://issues.apache.org/jira/browse/HUDI-288?focusedCommentId=16986264&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16986264).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362799446
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -171,6 +186,10 @@ public Operation convert(String value) throws ParameterException {
     public String propsFilePath =
         "file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties";
 
+    @Parameter(names = {"--custom-props"}, description = "path to properties file on localfs or dfs, with configurations for "
 
 Review comment:
   Yes, This props file holds the table config objects needed for multi table execution. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-586849583
 
 
   Working on it, @vinothchandar 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] gdineshbabu88 commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
gdineshbabu88 commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-602577700
 
 
   @pratyakshsharma Can you update the wiki for HoodieMultiDeltaStreamer similar to https://hudi.incubator.apache.org/docs/writing_data.html#deltastreamer?
   
   Can you advise in which release this tool will be available?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362275428
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   any reason this is in the `hudi-common` package? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r382175651
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   @pratyakshsharma : Sounds fair. Let's proceed with having TableConfig containing both Source and Sink config but rename the class to reflect it. The name TableConfig seems misleading to me.  I will go through the change once you address other comments.
   
   Regarding your comment about having same source, I think you meant source type (Kafka, DFS,...).  As the current TableConfig takes care of 1 pair of Source<->Sink configs (with configs in separate folders) anyways, there is no need to force that restriction. Let me know if I am missing something ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399542054
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTables;
+  private Set<String> failedTables;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) throws IOException {
 
 Review comment:
   HoodieDeltaStreamer.Config can be used at all places. Have made the changes. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-601773835
 
 
   @pratyakshsharma : Per your previous comment, was waiting for you to give a green signal :)  I will look at it sometime today. There are conflicts in this PR. If you can resolve them, that will be great. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362696795
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.TableConfig;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTopics;
+  private Set<String> failedTopics;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTopics = new HashSet<>();
+    this.failedTopics = new HashSet<>();
+    this.jssc = jssc;
+    String tableConfigFile = getCustomPropsFileName(args);
+    FileSystem fs = FSUtils.getFs(tableConfigFile, jssc.hadoopConfiguration());
+    List<TableConfig> configList = UtilHelpers.readTableConfig(fs, new Path(tableConfigFile)).getConfigs();
+
+    for (TableConfig config : configList) {
+      validateTableConfigObject(config);
+      populateTableExecutionObjectList(config, args);
+    }
+  }
+
+  /*
+  validate if given object has all the necessary fields.
+  Throws IllegalArgumentException if any of the required fields are missing
+   */
+  private void validateTableConfigObject(TableConfig config) {
+    if (Strings.isNullOrEmpty(config.getDatabase()) || Strings.isNullOrEmpty(config.getTableName()) || Strings.isNullOrEmpty(config.getPrimaryKeyField())
+        || Strings.isNullOrEmpty(config.getTopic())) {
+      throw new IllegalArgumentException("Please provide valid table config arguments!");
+    }
+  }
+
+  private void populateTableExecutionObjectList(TableConfig config, String[] args) {
+    TableExecutionObject executionObject;
+    try {
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, config.getDatabase(), config.getTableName());
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      cfg.targetBasePath = targetBasePath;
+      FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
+      TypedProperties typedProperties = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+      populateIngestionProps(typedProperties, config);
+      populateSchemaProviderProps(cfg, typedProperties, config);
+      populateHiveSyncProps(cfg, typedProperties, config);
+      executionObject = new TableExecutionObject();
+      executionObject.setConfig(cfg);
+      executionObject.setProperties(typedProperties);
+      executionObject.setTableConfig(config);
+      this.tableExecutionObjects.add(executionObject);
+    } catch (Exception e) {
+      logger.error("Error while creating execution object for topic: " + config.getTopic(), e);
+      throw e;
+    }
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+    }
+  }
+
+  private void populateHiveSyncProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.enableHiveSync && Strings.isNullOrEmpty(config.getHiveSyncTable())) {
+      throw new HoodieException("Hive sync table field not provided!");
+    }
+    typedProperties.setProperty(Constants.HIVE_SYNC_TABLE_PROP, config.getHiveSyncTable());
+    typedProperties.setProperty(Constants.HIVE_SYNC_DATABASE_NAME_PROP, Strings.isNullOrEmpty(config.getHiveSyncDatabase())
+        ? typedProperties.getString(Constants.HIVE_SYNC_DATABASE_NAME_PROP, DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL())
+        : config.getHiveSyncDatabase());
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), String.valueOf(config.getAssumeDatePartitioningForHiveSync()));
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(), String.valueOf(config.getUsePreApacheInputFormatForHiveSync()));
+  }
+
+  private void populateIngestionProps(TypedProperties typedProperties, TableConfig config) {
+    typedProperties.setProperty(Constants.KAFKA_TOPIC_PROP, config.getTopic());
+    typedProperties.setProperty(Constants.PARTITION_TIMESTAMP_TYPE_PROP, config.getPartitionTimestampType());
+    typedProperties.setProperty(Constants.PARTITION_FIELD_INPUT_FORMAT_PROP, config.getPartitionInputFormat());
+    typedProperties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), config.getPrimaryKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), config.getPartitionKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), Strings.isNullOrEmpty(config.getKeyGeneratorClassName())
+        ? typedProperties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL())
+        : config.getKeyGeneratorClassName());
+  }
+
+  public static void main(String[] args) {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.YARN_SPARK_MASTER);
 
 Review comment:
   Do we need to force yarn spark master ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362694211
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   Also, I think it would make sense to split the configuration to SourceConfig and TargetTableConfig. This way the configuration will be reusable for multi delta-streamer reading from non-kafka sources too ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r400006238
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -155,6 +181,19 @@ public static TestRawTripPayload generateRandomValue(
     return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
   }
 
+  /**
+   * Generates a new avro record with TRIP_UBER_EXAMPLE_SCHEMA, retaining the key if optionally provided.
+   */
+  public TestRawTripPayload generatePayloadForUberSchema(HoodieKey key, String commitTime) throws IOException {
 
 Review comment:
   Please remove Uber from this method name

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396130404
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -155,12 +166,20 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    @Parameter(names = {"--base-path-prefix"},
 
 Review comment:
   Is this parameter still needed for DeltaStreamer ? I see one in HoodieMultiDeltaStreamer 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
bvaradar commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r396124806
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -286,24 +350,46 @@ public static void createSavepointFile(String basePath, String commitTime, Confi
    * @return  List of {@link HoodieRecord}s
    */
   public List<HoodieRecord> generateInserts(String commitTime, Integer n, boolean isFlattened) {
-    return generateInsertsStream(commitTime, n, isFlattened).collect(Collectors.toList());
+    return generateInsertsStream(commitTime, n, isFlattened, TRIP_EXAMPLE_SCHEMA).collect(Collectors.toList());
+  }
+
+  /**
+   * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
+   */
+  public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, String schemaStr) {
+    int currSize = getNumExistingKeys(schemaStr);
+
+    return IntStream.range(0, n).boxed().map(i -> {
+      String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
+      HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
+      KeyPartition kp = new KeyPartition();
+      kp.key = key;
+      kp.partitionPath = partitionPath;
+      populateKeysBySchema(schemaStr, currSize + i, kp);
+      incrementNumExistingKeysBySchema(schemaStr);
+      try {
+        return new HoodieRecord(key, generateRandomValueAsPerSchema(schemaStr, key, commitTime));
+      } catch (IOException e) {
+        throw new HoodieIOException(e.getMessage(), e);
+      }
+    });
   }
 
   /**
    * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
    */
   public Stream<HoodieRecord> generateInsertsStream(
 
 Review comment:
   Can you refactor this and the previous method to reuse code ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367492682
 
 

 ##########
 File path: hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hive.MultiPartKeysValueExtractor;
+import org.apache.hudi.utilities.deltastreamer.HoodieMultiTableDeltaStreamer;
+import org.apache.hudi.utilities.deltastreamer.TableExecutionObject;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.JsonKafkaSource;
+import org.apache.hudi.utilities.sources.TestDataSource;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.streaming.kafka.KafkaTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestHoodieMultiTableDeltaStreamer extends UtilitiesTestBase {
+
+  private static final String PROPS_FILENAME_TEST_SOURCE = "test-source1.properties";
+  private static volatile Logger log = LogManager.getLogger(TestHoodieMultiTableDeltaStreamer.class);
+  private static KafkaTestUtils testUtils;
+
+  @BeforeClass
+  public static void initClass() throws Exception {
 
 Review comment:
   Will extend TestHoodieDeltaStreamer directly here. In the process I fixed a bug as well. :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on issue #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#issuecomment-597641119
 
 
   > @pratyakshsharma : Let us know if you need any help on getting this through :) ?
   
   @bvaradar The fixes in this PR depend on https://github.com/apache/incubator-hudi/pull/1395. Let us try to close it as soon as possible to get this through. :) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362385735
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/model/TableConfig.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Objects;
+
+/*
+Represents object with all the topic level overrides for multi table delta streamer execution
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
 
 Review comment:
   Basically it is a POJO and hudi-common is having most of the POJOs. Also since hudi-common is added as a dependency in most of the modules, it will be easier this way to use this class anywhere in future if need be. :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r405005385
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
 ##########
 @@ -169,12 +180,16 @@ public Operation convert(String value) throws ParameterException {
         required = true)
     public String targetBasePath;
 
+    // TODO: How to obtain hive configs to register?
     @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
     public String targetTableName;
 
     @Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ", required = true)
     public String tableType;
 
+    @Parameter(names = {"--config-folder"}, description = "Path to folder which contains all the properties file", required = true)
 
 Review comment:
   Done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r362797626
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -80,16 +79,27 @@
       + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
       + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
       + "{\"name\":\"fare\",\"type\": \"double\"}]}";
+  public static String GROCERY_PURCHASE_SCHEMA = "{\"type\":\"record\",\"name\":\"purchaserec\",\"fields\":["
 
 Review comment:
   @bvaradar I am still trying to understand why do you want to distinguish the topic from which the record with TestRawTripPayload got ingested? In the current setup also, all the test cases are passing. Could you please make your intention more clear?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r399371617
 
 

 ##########
 File path: hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
 ##########
 @@ -84,26 +87,35 @@
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-
   public static final String TRIP_EXAMPLE_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
   public static final String TRIP_FLATTENED_SCHEMA =
       TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
 
+  public static String TRIP_UBER_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripuberrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+  public static String TRIP_FG_EXAMPLE_SCHEMA = "{\"type\":\"record\",\"name\":\"tripfgrec\",\"fields\":["
+      + "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+      + "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
+
   public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
   public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,"
-                                                  + "struct<amount:double,currency:string>,boolean";
-
+      + "struct<amount:double,currency:string>,boolean";
   public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
   public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
       HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
+  public static Schema avroFgSchema = new Schema.Parser().parse(TRIP_FG_EXAMPLE_SCHEMA);
+  public static Schema avroUberSchema = new Schema.Parser().parse(TRIP_UBER_EXAMPLE_SCHEMA);
   public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
 
-  private static final Random RAND = new Random(46474747);
+  private static Random rand = new Random(46474747);
 
 Review comment:
   Fixed. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment

Posted by GitBox <gi...@apache.org>.
pratyakshsharma commented on a change in pull request #1150: [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
URL: https://github.com/apache/incubator-hudi/pull/1150#discussion_r367366905
 
 

 ##########
 File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
 ##########
 @@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.deltastreamer;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.TableConfig;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
+import org.apache.hudi.utilities.schema.SchemaRegistryProvider;
+
+import com.beust.jcommander.JCommander;
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Wrapper over HoodieDeltaStreamer.java class.
+ * Helps with ingesting incremental data into hoodie datasets for multiple tables.
+ * Currently supports only COPY_ON_WRITE storage type.
+ */
+public class HoodieMultiTableDeltaStreamer {
+
+  private static Logger logger = LogManager.getLogger(HoodieMultiTableDeltaStreamer.class);
+
+  private List<TableExecutionObject> tableExecutionObjects;
+  private transient JavaSparkContext jssc;
+  private Set<String> successTopics;
+  private Set<String> failedTopics;
+
+  public HoodieMultiTableDeltaStreamer(String[] args, JavaSparkContext jssc) {
+    this.tableExecutionObjects = new ArrayList<>();
+    this.successTopics = new HashSet<>();
+    this.failedTopics = new HashSet<>();
+    this.jssc = jssc;
+    String tableConfigFile = getCustomPropsFileName(args);
+    FileSystem fs = FSUtils.getFs(tableConfigFile, jssc.hadoopConfiguration());
+    List<TableConfig> configList = UtilHelpers.readTableConfig(fs, new Path(tableConfigFile)).getConfigs();
+
+    for (TableConfig config : configList) {
+      validateTableConfigObject(config);
+      populateTableExecutionObjectList(config, args);
+    }
+  }
+
+  /*
+  validate if given object has all the necessary fields.
+  Throws IllegalArgumentException if any of the required fields are missing
+   */
+  private void validateTableConfigObject(TableConfig config) {
+    if (Strings.isNullOrEmpty(config.getDatabase()) || Strings.isNullOrEmpty(config.getTableName()) || Strings.isNullOrEmpty(config.getPrimaryKeyField())
+        || Strings.isNullOrEmpty(config.getTopic())) {
+      throw new IllegalArgumentException("Please provide valid table config arguments!");
+    }
+  }
+
+  private void populateTableExecutionObjectList(TableConfig config, String[] args) {
+    TableExecutionObject executionObject;
+    try {
+      final Config cfg = new Config();
+      String[] tableArgs = args.clone();
+      String targetBasePath = resetTarget(tableArgs, config.getDatabase(), config.getTableName());
+      JCommander cmd = new JCommander(cfg);
+      cmd.parse(tableArgs);
+      cfg.targetBasePath = targetBasePath;
+      FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
+      TypedProperties typedProperties = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+      populateIngestionProps(typedProperties, config);
+      populateSchemaProviderProps(cfg, typedProperties, config);
+      populateHiveSyncProps(cfg, typedProperties, config);
+      executionObject = new TableExecutionObject();
+      executionObject.setConfig(cfg);
+      executionObject.setProperties(typedProperties);
+      executionObject.setTableConfig(config);
+      this.tableExecutionObjects.add(executionObject);
+    } catch (Exception e) {
+      logger.error("Error while creating execution object for topic: " + config.getTopic(), e);
+      throw e;
+    }
+  }
+
+  private void populateSchemaProviderProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.schemaProviderClassName.equals(SchemaRegistryProvider.class.getName())) {
+      String schemaRegistryBaseUrl = typedProperties.getString(Constants.SCHEMA_REGISTRY_BASE_URL_PROP);
+      String schemaRegistrySuffix = typedProperties.getString(Constants.SCHEMA_REGISTRY_URL_SUFFIX_PROP);
+      typedProperties.setProperty(Constants.SOURCE_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+      typedProperties.setProperty(Constants.TARGET_SCHEMA_REGISTRY_URL_PROP, schemaRegistryBaseUrl + config.getTopic() + schemaRegistrySuffix);
+    }
+  }
+
+  private void populateHiveSyncProps(Config cfg, TypedProperties typedProperties, TableConfig config) {
+    if (cfg.enableHiveSync && Strings.isNullOrEmpty(config.getHiveSyncTable())) {
+      throw new HoodieException("Hive sync table field not provided!");
+    }
+    typedProperties.setProperty(Constants.HIVE_SYNC_TABLE_PROP, config.getHiveSyncTable());
+    typedProperties.setProperty(Constants.HIVE_SYNC_DATABASE_NAME_PROP, Strings.isNullOrEmpty(config.getHiveSyncDatabase())
+        ? typedProperties.getString(Constants.HIVE_SYNC_DATABASE_NAME_PROP, DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL())
+        : config.getHiveSyncDatabase());
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), String.valueOf(config.getAssumeDatePartitioningForHiveSync()));
+    typedProperties.setProperty(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(), String.valueOf(config.getUsePreApacheInputFormatForHiveSync()));
+  }
+
+  private void populateIngestionProps(TypedProperties typedProperties, TableConfig config) {
+    typedProperties.setProperty(Constants.KAFKA_TOPIC_PROP, config.getTopic());
+    typedProperties.setProperty(Constants.PARTITION_TIMESTAMP_TYPE_PROP, config.getPartitionTimestampType());
+    typedProperties.setProperty(Constants.PARTITION_FIELD_INPUT_FORMAT_PROP, config.getPartitionInputFormat());
+    typedProperties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), config.getPrimaryKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), config.getPartitionKeyField());
+    typedProperties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), Strings.isNullOrEmpty(config.getKeyGeneratorClassName())
+        ? typedProperties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL())
+        : config.getKeyGeneratorClassName());
+  }
+
+  public static void main(String[] args) {
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("multi-table-delta-streamer", Constants.YARN_SPARK_MASTER);
 
 Review comment:
   Will change it. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services