zhtaoxiang commented on code in PR #10034: URL: https://github.com/apache/pinot/pull/10034#discussion_r1058115858
########## pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-2.4/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark/SparkSegmentGenerationJobRunner.java: ########## @@ -290,20 +291,26 @@ public void call(String pathAndIdx) long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile); LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName, DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize)); - //move segment to output PinotFS - URI outputSegmentTarURI = - SegmentGenerationUtils.getRelativeOutputPath(finalInputDirURI, inputFileURI, finalOutputDirURI) - .resolve(segmentTarFileName); - LOGGER.info("Trying to move segment tar file from: [{}] to [{}]", localSegmentTarFile, outputSegmentTarURI); - if (!_spec.isOverwriteOutput() && PinotFSFactory.create(outputSegmentTarURI.getScheme()) - .exists(outputSegmentTarURI)) { - LOGGER.warn("Not overwrite existing output segment tar file: {}", - finalOutputDirFS.exists(outputSegmentTarURI)); - } else { - finalOutputDirFS.copyFromLocalFile(localSegmentTarFile, outputSegmentTarURI); + // Move segment to output PinotFS + URI relativeOutputPath = + SegmentGenerationUtils.getRelativeOutputPath(finalInputDirURI, inputFileURI, finalOutputDirURI); + URI outputSegmentTarURI = relativeOutputPath.resolve(segmentTarFileName); + + // Create and upload segment metadata tar file + String metadataTarFileName = URLEncoder.encode(segmentName + Constants.METADATA_TAR_GZ_FILE_EXT, "UTF-8"); Review Comment: the same here. ########## pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentCreationMapper.java: ########## @@ -183,13 +184,24 @@ protected void map(LongWritable key, Text value, Context context) { LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName, DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize)); //move segment to output PinotFS - URI outputSegmentTarURI = SegmentGenerationUtils.getRelativeOutputPath(inputDirURI, inputFileURI, outputDirURI) - .resolve(segmentTarFileName); - LOGGER.info("Copying segment tar file from [{}] to [{}]", localSegmentTarFile, outputSegmentTarURI); - outputDirFS.copyFromLocalFile(localSegmentTarFile, outputSegmentTarURI); + URI relativeOutputPath = SegmentGenerationUtils.getRelativeOutputPath(inputDirURI, inputFileURI, outputDirURI); + URI outputSegmentTarURI = relativeOutputPath.resolve(segmentTarFileName); + SegmentGenerationJobUtils.moveLocalTarFileToRemote(localSegmentTarFile, outputSegmentTarURI, + _spec.isOverwriteOutput()); + + String metadataTarFileName = URLEncoder.encode(segmentName + Constants.METADATA_TAR_GZ_FILE_EXT, "UTF-8"); Review Comment: When `_spec.isOverwriteOutput()` is false and `taskSpec.isCreateMetadataTarGz()` is true, we should not delete `outputMetadataTarURI`. Otherwise, segment tar file will NOT be replaced and metadata tar file will be replaced, which causes inconsistency. ########## pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3.2/src/main/java/org/apache/pinot/plugin/ingestion/batch/spark3/SparkSegmentGenerationJobRunner.java: ########## @@ -289,20 +289,26 @@ public void call(String pathAndIdx) long compressedSegmentSize = FileUtils.sizeOf(localSegmentTarFile); LOGGER.info("Size for segment: {}, uncompressed: {}, compressed: {}", segmentName, DataSizeUtils.fromBytes(uncompressedSegmentSize), DataSizeUtils.fromBytes(compressedSegmentSize)); - //move segment to output PinotFS - URI outputSegmentTarURI = - SegmentGenerationUtils.getRelativeOutputPath(finalInputDirURI, inputFileURI, finalOutputDirURI) - .resolve(segmentTarFileName); - LOGGER.info("Trying to move segment tar file from: [{}] to [{}]", localSegmentTarFile, outputSegmentTarURI); - if (!_spec.isOverwriteOutput() && PinotFSFactory.create(outputSegmentTarURI.getScheme()) - .exists(outputSegmentTarURI)) { - LOGGER.warn("Not overwrite existing output segment tar file: {}", - finalOutputDirFS.exists(outputSegmentTarURI)); - } else { - finalOutputDirFS.copyFromLocalFile(localSegmentTarFile, outputSegmentTarURI); + // Move segment to output PinotFS + URI relativeOutputPath = + SegmentGenerationUtils.getRelativeOutputPath(finalInputDirURI, inputFileURI, finalOutputDirURI); + URI outputSegmentTarURI = relativeOutputPath.resolve(segmentTarFileName); + SegmentGenerationJobUtils.moveLocalTarFileToRemote(localSegmentTarFile, outputSegmentTarURI, + _spec.isOverwriteOutput()); + + // Create and upload segment metadata tar file + String metadataTarFileName = URLEncoder.encode(segmentName + Constants.METADATA_TAR_GZ_FILE_EXT, "UTF-8"); Review Comment: the same here. -- 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: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org