nsivabalan commented on code in PR #8086:
URL: https://github.com/apache/hudi/pull/8086#discussion_r1126777232


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java:
##########
@@ -883,9 +877,19 @@ public SparkSession getSparkSession() {
     public TypedProperties getProps() {
       return props;
     }
+
+    /**
+     * This API is for testing only.
+     */
+    public DeltaSync getDeltaSync() {
+      return deltaSync;
+    }
   }
 
-  public DeltaSyncService getDeltaSyncService() {
-    return deltaSyncService.get();
+  /**
+   * This API is for testing only.
+   */

Review Comment:
   I am not a fan of exposing internal variables just for testing purpose. lets 
see  if we can avoid this.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/ingestion/HoodieIngestionService.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.ingestion;
+
+import org.apache.hudi.async.HoodieAsyncService;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static 
org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_IS_CONTINUOUS;
+import static 
org.apache.hudi.utilities.ingestion.HoodieIngestionService.HoodieIngestionConfig.INGESTION_MIN_SYNC_INTERNAL_SECONDS;
+
+/**
+ * A generic service to facilitate running data ingestion.
+ */
+public abstract class HoodieIngestionService extends HoodieAsyncService {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieIngestionService.class);
+
+  protected HoodieIngestionConfig ingestionConfig;
+
+  public HoodieIngestionService(HoodieIngestionConfig ingestionConfig) {
+    this.ingestionConfig = ingestionConfig;
+  }
+
+  /**
+   * Entrypoint to start ingestion.

Review Comment:
   lets add some documentation on the flow of calls for syncOnce and continuous 
mode. esply continuous mode is not very apparent. 
   for eg, startIngestion -> start -> startService { while { ingestOnce() } } 



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to