wangxianghu commented on a change in pull request #2226:
URL: https://github.com/apache/hudi/pull/2226#discussion_r518740766



##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
##########
@@ -528,8 +528,9 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, 
FileSystem fs, Config
       this.jssc = jssc;
       this.sparkSession = 
SparkSession.builder().config(jssc.getConf()).getOrCreate();
       this.asyncCompactService = Option.empty();
+      this.props = properties.get();
 
-      if (fs.exists(new Path(cfg.targetBasePath))) {
+      if (fs.exists(new Path(cfg.targetBasePath)) && 
!Boolean.valueOf(props.getString("hoodie.deltastreamer.source.dfs.full.overwrite",
 "false"))) {

Review comment:
       since we have introduced a constant for this config, we'd better use the 
constant name. 
   besides, provide a method to acquire it should be better

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java
##########
@@ -98,35 +99,37 @@ public static DFSPathSelector 
createSourceSelector(TypedProperties props,
       long sourceLimit) {
 
     try {
-      // obtain all eligible files under root folder.
-      log.info("Root path => " + props.getString(Config.ROOT_INPUT_PATH_PROP) 
+ " source limit => " + sourceLimit);
-      long lastCheckpointTime = 
lastCheckpointStr.map(Long::parseLong).orElse(Long.MIN_VALUE);
-      List<FileStatus> eligibleFiles = listEligibleFiles(fs, new 
Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), lastCheckpointTime);
-      // sort them by modification time.
-      
eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime));
-      // Filter based on checkpoint & input size, if needed
-      long currentBytes = 0;
+      String pathStr = props.getString(Config.ROOT_INPUT_PATH_PROP);
       long maxModificationTime = Long.MIN_VALUE;
-      List<FileStatus> filteredFiles = new ArrayList<>();
-      for (FileStatus f : eligibleFiles) {
-        if (currentBytes + f.getLen() >= sourceLimit) {
-          // we have enough data, we are done
-          break;
+      if (!Boolean.valueOf(props.getString(Config.FULL_OVERWRITE,"false"))) {
+        // obtain all eligible files under root folder.
+        log.info("Root path => " + 
props.getString(Config.ROOT_INPUT_PATH_PROP) + " source limit => " + 
sourceLimit);
+        long lastCheckpointTime = 
lastCheckpointStr.map(Long::parseLong).orElse(Long.MIN_VALUE);
+        List<FileStatus> eligibleFiles = listEligibleFiles(fs, new 
Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), lastCheckpointTime);
+        // sort them by modification time.
+        
eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime));
+        // Filter based on checkpoint & input size, if needed
+        long currentBytes = 0;
+        List<FileStatus> filteredFiles = new ArrayList<>();
+        for (FileStatus f : eligibleFiles) {
+          if (currentBytes + f.getLen() >= sourceLimit) {
+            // we have enough data, we are done
+            break;
+          }
+
+          maxModificationTime = f.getModificationTime();
+          currentBytes += f.getLen();
+          filteredFiles.add(f);

Review comment:
       IIUC, we can set 'sourceLimit' to `Long.MAX_VALUE` to read all of the 
data at one go. right?
   if I am right we can save lots of changes

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java
##########
@@ -54,6 +54,7 @@
 
     public static final String ROOT_INPUT_PATH_PROP = 
"hoodie.deltastreamer.source.dfs.root";
     public static final String SOURCE_INPUT_SELECTOR = 
"hoodie.deltastreamer.source.input.selector";
+    public static final String FULL_OVERWRITE = 
"hoodie.deltastreamer.source.dfs.full.overwrite";

Review comment:
       @liujinhui1994  Maybe we can move this config to `HoodieWriteConfig` and 
rename it to `hoodie.truncate.before.write` if it can be used in another source 
type. 
   WDYT ?




----------------------------------------------------------------
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]


Reply via email to