xushiyan commented on a change in pull request #1436: [HUDI-711] Refactor exporter main logic URL: https://github.com/apache/incubator-hudi/pull/1436#discussion_r396039049
########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java ########## @@ -160,37 +174,36 @@ private void exportAsNonHudi(SparkSession spark, Config cfg, List<String> dataFi ? defaultPartitioner : ReflectionUtils.loadClass(cfg.outputPartitioner); - Dataset<Row> sourceDataset = spark.read().parquet(JavaConversions.asScalaIterator(dataFiles.iterator()).toSeq()); + final JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); + final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg); + Iterator<String> exportingFilePaths = jsc + .parallelize(partitions, partitions.size()) + .flatMap(partition -> fsView + .getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp) + .map(HoodieBaseFile::getPath).iterator()) + .toLocalIterator(); + + Dataset<Row> sourceDataset = spark.read().parquet(JavaConversions.asScalaIterator(exportingFilePaths).toSeq()); partitioner.partition(sourceDataset) .format(cfg.outputFormat) .mode(SaveMode.Overwrite) .save(cfg.targetOutputPath); } - private void copySnapshot(JavaSparkContext jsc, - FileSystem fs, - Config cfg, - List<String> partitions, - List<String> dataFiles, - String latestCommitTimestamp, - SerializableConfiguration serConf) throws IOException { - // Make sure the output directory is empty - Path outputPath = new Path(cfg.targetOutputPath); - if (fs.exists(outputPath)) { - LOG.warn(String.format("The output path %s targetBasePath already exists, deleting", outputPath)); - fs.delete(new Path(cfg.targetOutputPath), true); - } - + private void exportAsHudi(JavaSparkContext jsc, Config cfg, List<String> partitions, String latestCommitTimestamp) throws IOException { + final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg); + final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); jsc.parallelize(partitions, partitions.size()).flatMap(partition -> { // Only take latest version files <= latestCommit. - FileSystem fs1 = FSUtils.getFs(cfg.sourceBasePath, serConf.newCopy()); List<Tuple2<String, String>> filePaths = new ArrayList<>(); - dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile))); Review comment: `dataFiles` was passed in being the full list across all partitions. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services