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:
[email protected]


With regards,
Apache Git Services

Reply via email to