@@ -114,16 +114,18 @@ public static class Config implements Serializable {
114114 }
115115
116116 public void export (JavaSparkContext jsc , Config cfg ) throws IOException {
117- FileSystem fs = FSUtils .getFs (cfg .sourceBasePath , jsc .hadoopConfiguration ());
117+ FileSystem outputFs = FSUtils .getFs (cfg .targetOutputPath , jsc .hadoopConfiguration ());
118118 HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext (jsc );
119119
120- if (outputPathExists (fs , cfg )) {
120+ if (outputPathExists (outputFs , cfg )) {
121121 throw new HoodieSnapshotExporterException ("The target output path already exists." );
122122 }
123123
124- final String latestCommitTimestamp = getLatestCommitTimestamp (fs , cfg ).<HoodieSnapshotExporterException >orElseThrow (() -> {
125- throw new HoodieSnapshotExporterException ("No commits present. Nothing to snapshot." );
126- });
124+ FileSystem sourceFs = FSUtils .getFs (cfg .sourceBasePath , jsc .hadoopConfiguration ());
125+ final String latestCommitTimestamp = getLatestCommitTimestamp (sourceFs , cfg )
126+ .<HoodieSnapshotExporterException >orElseThrow (() -> {
127+ throw new HoodieSnapshotExporterException ("No commits present. Nothing to snapshot." );
128+ });
127129 LOG .info (String .format ("Starting to snapshot latest version files which are also no-late-than %s." ,
128130 latestCommitTimestamp ));
129131
@@ -134,11 +136,11 @@ public void export(JavaSparkContext jsc, Config cfg) throws IOException {
134136 LOG .info (String .format ("The job needs to export %d partitions." , partitions .size ()));
135137
136138 if (cfg .outputFormat .equals (OutputFormatValidator .HUDI )) {
137- exportAsHudi (jsc , cfg , partitions , latestCommitTimestamp );
139+ exportAsHudi (jsc , sourceFs , cfg , partitions , latestCommitTimestamp );
138140 } else {
139- exportAsNonHudi (jsc , cfg , partitions , latestCommitTimestamp );
141+ exportAsNonHudi (jsc , sourceFs , cfg , partitions , latestCommitTimestamp );
140142 }
141- createSuccessTag (fs , cfg );
143+ createSuccessTag (outputFs , cfg );
142144 }
143145
144146 private boolean outputPathExists (FileSystem fs , Config cfg ) throws IOException {
@@ -164,7 +166,8 @@ private void createSuccessTag(FileSystem fs, Config cfg) throws IOException {
164166 }
165167 }
166168
167- private void exportAsNonHudi (JavaSparkContext jsc , Config cfg , List <String > partitions , String latestCommitTimestamp ) {
169+ private void exportAsNonHudi (JavaSparkContext jsc , FileSystem sourceFs ,
170+ Config cfg , List <String > partitions , String latestCommitTimestamp ) {
168171 Partitioner defaultPartitioner = dataset -> {
169172 Dataset <Row > hoodieDroppedDataset = dataset .drop (JavaConversions .asScalaIterator (HoodieRecord .HOODIE_META_COLUMNS .iterator ()).toSeq ());
170173 return StringUtils .isNullOrEmpty (cfg .outputPartitionField )
@@ -178,7 +181,7 @@ private void exportAsNonHudi(JavaSparkContext jsc, Config cfg, List<String> part
178181
179182 HoodieEngineContext context = new HoodieSparkEngineContext (jsc );
180183 context .setJobStatus (this .getClass ().getSimpleName (), "Exporting as non-HUDI dataset: " + cfg .targetOutputPath );
181- final BaseFileOnlyView fsView = getBaseFileOnlyView (jsc , cfg );
184+ final BaseFileOnlyView fsView = getBaseFileOnlyView (sourceFs , cfg );
182185 Iterator <String > exportingFilePaths = jsc
183186 .parallelize (partitions , partitions .size ())
184187 .flatMap (partition -> fsView
@@ -193,8 +196,9 @@ private void exportAsNonHudi(JavaSparkContext jsc, Config cfg, List<String> part
193196 .save (cfg .targetOutputPath );
194197 }
195198
196- private void exportAsHudi (JavaSparkContext jsc , Config cfg , List <String > partitions , String latestCommitTimestamp ) throws IOException {
197- final BaseFileOnlyView fsView = getBaseFileOnlyView (jsc , cfg );
199+ private void exportAsHudi (JavaSparkContext jsc , FileSystem sourceFs ,
200+ Config cfg , List <String > partitions , String latestCommitTimestamp ) throws IOException {
201+ final BaseFileOnlyView fsView = getBaseFileOnlyView (sourceFs , cfg );
198202
199203 final HoodieEngineContext context = new HoodieSparkEngineContext (jsc );
200204 final SerializableConfiguration serConf = context .getHadoopConf ();
@@ -219,20 +223,26 @@ private void exportAsHudi(JavaSparkContext jsc, Config cfg, List<String> partiti
219223 String partition = tuple ._1 ();
220224 Path sourceFilePath = new Path (tuple ._2 ());
221225 Path toPartitionPath = FSUtils .getPartitionPath (cfg .targetOutputPath , partition );
222- FileSystem fs = FSUtils .getFs (cfg .targetOutputPath , serConf .newCopy ());
226+ FileSystem executorSourceFs = FSUtils .getFs (cfg .sourceBasePath , serConf .newCopy ());
227+ FileSystem executorOutputFs = FSUtils .getFs (cfg .targetOutputPath , serConf .newCopy ());
223228
224- if (!fs .exists (toPartitionPath )) {
225- fs .mkdirs (toPartitionPath );
229+ if (!executorOutputFs .exists (toPartitionPath )) {
230+ executorOutputFs .mkdirs (toPartitionPath );
226231 }
227- FileUtil .copy (fs , sourceFilePath , fs , new Path (toPartitionPath , sourceFilePath .getName ()), false ,
228- fs .getConf ());
232+ FileUtil .copy (
233+ executorSourceFs ,
234+ sourceFilePath ,
235+ executorOutputFs ,
236+ new Path (toPartitionPath , sourceFilePath .getName ()),
237+ false ,
238+ executorOutputFs .getConf ());
229239 }, files .size ());
230240
231241 // Also copy the .commit files
232242 LOG .info (String .format ("Copying .commit files which are no-late-than %s." , latestCommitTimestamp ));
233- final FileSystem fileSystem = FSUtils .getFs (cfg .sourceBasePath , jsc .hadoopConfiguration ());
243+ FileSystem outputFs = FSUtils .getFs (cfg .targetOutputPath , jsc .hadoopConfiguration ());
234244 FileStatus [] commitFilesToCopy =
235- fileSystem .listStatus (new Path (cfg .sourceBasePath + "/" + HoodieTableMetaClient .METAFOLDER_NAME ), (commitFilePath ) -> {
245+ sourceFs .listStatus (new Path (cfg .sourceBasePath + "/" + HoodieTableMetaClient .METAFOLDER_NAME ), (commitFilePath ) -> {
236246 if (commitFilePath .getName ().equals (HoodieTableConfig .HOODIE_PROPERTIES_FILE )) {
237247 return true ;
238248 } else {
@@ -244,20 +254,22 @@ private void exportAsHudi(JavaSparkContext jsc, Config cfg, List<String> partiti
244254 for (FileStatus commitStatus : commitFilesToCopy ) {
245255 Path targetFilePath =
246256 new Path (cfg .targetOutputPath + "/" + HoodieTableMetaClient .METAFOLDER_NAME + "/" + commitStatus .getPath ().getName ());
247- if (!fileSystem .exists (targetFilePath .getParent ())) {
248- fileSystem .mkdirs (targetFilePath .getParent ());
257+ if (!outputFs .exists (targetFilePath .getParent ())) {
258+ outputFs .mkdirs (targetFilePath .getParent ());
249259 }
250- if (fileSystem .exists (targetFilePath )) {
260+ if (outputFs .exists (targetFilePath )) {
251261 LOG .error (
252262 String .format ("The target output commit file (%s targetBasePath) already exists." , targetFilePath ));
253263 }
254- FileUtil .copy (fileSystem , commitStatus .getPath (), fileSystem , targetFilePath , false , fileSystem .getConf ());
264+ FileUtil .copy (sourceFs , commitStatus .getPath (), outputFs , targetFilePath , false , outputFs .getConf ());
255265 }
256266 }
257267
258- private BaseFileOnlyView getBaseFileOnlyView (JavaSparkContext jsc , Config cfg ) {
259- FileSystem fs = FSUtils .getFs (cfg .sourceBasePath , jsc .hadoopConfiguration ());
260- HoodieTableMetaClient tableMetadata = HoodieTableMetaClient .builder ().setConf (fs .getConf ()).setBasePath (cfg .sourceBasePath ).build ();
268+ private BaseFileOnlyView getBaseFileOnlyView (FileSystem sourceFs , Config cfg ) {
269+ HoodieTableMetaClient tableMetadata = HoodieTableMetaClient .builder ()
270+ .setConf (sourceFs .getConf ())
271+ .setBasePath (cfg .sourceBasePath )
272+ .build ();
261273 return new HoodieTableFileSystemView (tableMetadata , tableMetadata
262274 .getActiveTimeline ().getWriteTimeline ().filterCompletedInstants ());
263275 }
0 commit comments