-
Notifications
You must be signed in to change notification settings - Fork 5
ice: Refactored multiple files processing to a separate function. #24
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 4 commits
ed4f402
2dd678d
ea830de
c500040
3078e59
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -7,10 +7,15 @@ | |||||||||||||||
| import com.altinity.ice.internal.jvm.Stats; | ||||||||||||||||
| import com.altinity.ice.internal.parquet.Metadata; | ||||||||||||||||
| import java.io.IOException; | ||||||||||||||||
| import java.util.ArrayList; | ||||||||||||||||
| import java.util.Arrays; | ||||||||||||||||
| import java.util.HashSet; | ||||||||||||||||
| import java.util.Map; | ||||||||||||||||
| import java.util.Set; | ||||||||||||||||
| import java.util.concurrent.ExecutionException; | ||||||||||||||||
| import java.util.concurrent.ExecutorService; | ||||||||||||||||
| import java.util.concurrent.Executors; | ||||||||||||||||
| import java.util.concurrent.Future; | ||||||||||||||||
| import java.util.function.Function; | ||||||||||||||||
| import java.util.function.Supplier; | ||||||||||||||||
| import java.util.stream.Collectors; | ||||||||||||||||
|
|
@@ -66,26 +71,38 @@ public static void run( | |||||||||||||||
| boolean forceTableAuth, | ||||||||||||||||
| boolean s3NoSignRequest, | ||||||||||||||||
| boolean s3CopyObject, | ||||||||||||||||
| String retryListFile) | ||||||||||||||||
| String retryListFile, | ||||||||||||||||
| int threadCount) | ||||||||||||||||
| throws IOException { | ||||||||||||||||
| if (files.length == 0) { | ||||||||||||||||
| // no work to be done | ||||||||||||||||
| return; | ||||||||||||||||
| } | ||||||||||||||||
| if (forceNoCopy) { | ||||||||||||||||
| noCopy = true; | ||||||||||||||||
| } | ||||||||||||||||
| InsertOptions options = | ||||||||||||||||
| InsertOptions.builder() | ||||||||||||||||
| .skipDuplicates(skipDuplicates) | ||||||||||||||||
| .noCommit(noCommit) | ||||||||||||||||
| .noCopy(noCopy) | ||||||||||||||||
| .forceNoCopy(forceNoCopy) | ||||||||||||||||
| .forceTableAuth(forceTableAuth) | ||||||||||||||||
| .s3NoSignRequest(s3NoSignRequest) | ||||||||||||||||
| .s3CopyObject(s3CopyObject) | ||||||||||||||||
| .threadCount(threadCount) | ||||||||||||||||
| .build(); | ||||||||||||||||
|
|
||||||||||||||||
| final InsertOptions finalOptions = | ||||||||||||||||
| options.forceNoCopy() ? options.toBuilder().noCopy(true).build() : options; | ||||||||||||||||
| Table table = catalog.loadTable(nsTable); | ||||||||||||||||
| try (FileIO tableIO = table.io()) { | ||||||||||||||||
| final Supplier<S3Client> s3ClientSupplier; | ||||||||||||||||
| if (forceTableAuth) { | ||||||||||||||||
| if (finalOptions.forceTableAuth()) { | ||||||||||||||||
| if (!(tableIO instanceof S3FileIO)) { | ||||||||||||||||
| throw new UnsupportedOperationException( | ||||||||||||||||
| "--force-table-auth is currently only supported for s3:// tables"); | ||||||||||||||||
| } | ||||||||||||||||
| s3ClientSupplier = ((S3FileIO) tableIO)::client; | ||||||||||||||||
| } else { | ||||||||||||||||
| s3ClientSupplier = () -> S3.newClient(s3NoSignRequest); | ||||||||||||||||
| s3ClientSupplier = () -> S3.newClient(finalOptions.s3NoSignRequest()); | ||||||||||||||||
| } | ||||||||||||||||
| Lazy<S3Client> s3ClientLazy = new Lazy<>(s3ClientSupplier); | ||||||||||||||||
| try { | ||||||||||||||||
|
|
@@ -136,125 +153,66 @@ public static void run( | |||||||||||||||
| : null) { | ||||||||||||||||
| boolean atLeastOneFileAppended = false; | ||||||||||||||||
|
|
||||||||||||||||
| // TODO: parallel | ||||||||||||||||
| for (final String file : filesExpanded) { | ||||||||||||||||
| DataFile df; | ||||||||||||||||
| try { | ||||||||||||||||
| logger.info("{}: processing", file); | ||||||||||||||||
| logger.info("{}: jvm: {}", file, Stats.gather()); | ||||||||||||||||
|
|
||||||||||||||||
| Function<String, Boolean> checkNotExists = | ||||||||||||||||
| dataFile -> { | ||||||||||||||||
| if (tableDataFiles.contains(dataFile)) { | ||||||||||||||||
| if (skipDuplicates) { | ||||||||||||||||
| logger.info("{}: duplicate (skipping)", file); | ||||||||||||||||
| return true; | ||||||||||||||||
| } | ||||||||||||||||
| throw new AlreadyExistsException( | ||||||||||||||||
| String.format("%s is already referenced by the table", dataFile)); | ||||||||||||||||
| } | ||||||||||||||||
| return false; | ||||||||||||||||
| }; | ||||||||||||||||
| int numThreads = Math.min(finalOptions.threadCount(), filesExpanded.size()); | ||||||||||||||||
| ExecutorService executor = Executors.newFixedThreadPool(numThreads); | ||||||||||||||||
| try { | ||||||||||||||||
| var futures = new ArrayList<Future<DataFile>>(); | ||||||||||||||||
| for (final String file : filesExpanded) { | ||||||||||||||||
| futures.add( | ||||||||||||||||
| executor.submit( | ||||||||||||||||
| () -> { | ||||||||||||||||
| try { | ||||||||||||||||
| return processFile( | ||||||||||||||||
| table, | ||||||||||||||||
| catalog, | ||||||||||||||||
| tableIO, | ||||||||||||||||
| inputIO, | ||||||||||||||||
| tableDataFiles, | ||||||||||||||||
| finalOptions, | ||||||||||||||||
| s3ClientLazy, | ||||||||||||||||
| dstDataFileSource, | ||||||||||||||||
| tableSchema, | ||||||||||||||||
| dataFileNamingStrategy, | ||||||||||||||||
| file); | ||||||||||||||||
| } catch (Exception e) { | ||||||||||||||||
| if (retryLog != null) { | ||||||||||||||||
| logger.error( | ||||||||||||||||
| "{}: error (adding to retry list and continuing)", file, e); | ||||||||||||||||
| retryLog.add(file); | ||||||||||||||||
| return null; | ||||||||||||||||
| } else { | ||||||||||||||||
| throw e; | ||||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
| })); | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| InputFile inputFile = | ||||||||||||||||
| Input.newFile(file, catalog, inputIO == null ? tableIO : inputIO); | ||||||||||||||||
| ParquetMetadata metadata = Metadata.read(inputFile); | ||||||||||||||||
| MessageType type = metadata.getFileMetaData().getSchema(); | ||||||||||||||||
| Schema fileSchema = | ||||||||||||||||
| ParquetSchemaUtil.convert(type); // nameMapping applied (when present) | ||||||||||||||||
| if (!sameSchema(table, fileSchema)) { | ||||||||||||||||
| throw new BadRequestException( | ||||||||||||||||
| String.format("%s's schema doesn't match table's schema", file)); | ||||||||||||||||
| } | ||||||||||||||||
| // assuming datafiles can be anywhere when table.location() is empty | ||||||||||||||||
| var noCopyPossible = file.startsWith(table.location()) || forceNoCopy; | ||||||||||||||||
| // TODO: check before uploading anything | ||||||||||||||||
| if (noCopy && !noCopyPossible) { | ||||||||||||||||
| throw new BadRequestException( | ||||||||||||||||
| file + " cannot be added to catalog without copy"); // TODO: explain | ||||||||||||||||
| } | ||||||||||||||||
| long dataFileSizeInBytes; | ||||||||||||||||
| var dataFile = replacePrefix(file, "s3a://", "s3://"); | ||||||||||||||||
| if (noCopy) { | ||||||||||||||||
| if (checkNotExists.apply(dataFile)) { | ||||||||||||||||
| continue; | ||||||||||||||||
| } | ||||||||||||||||
| dataFileSizeInBytes = inputFile.getLength(); | ||||||||||||||||
| } else if (s3CopyObject) { | ||||||||||||||||
| if (!dataFile.startsWith("s3://") || !table.location().startsWith("s3://")) { | ||||||||||||||||
| throw new BadRequestException( | ||||||||||||||||
| "--s3-copy-object is only supported between s3:// buckets"); | ||||||||||||||||
| for (var future : futures) { | ||||||||||||||||
| try { | ||||||||||||||||
| DataFile df = future.get(); | ||||||||||||||||
| if (df != null) { | ||||||||||||||||
| atLeastOneFileAppended = true; | ||||||||||||||||
| appendOp.appendFile(df); | ||||||||||||||||
| } | ||||||||||||||||
| String dstDataFile = dstDataFileSource.get(file); | ||||||||||||||||
| if (checkNotExists.apply(dstDataFile)) { | ||||||||||||||||
| continue; | ||||||||||||||||
| } catch (InterruptedException e) { | ||||||||||||||||
| Thread.currentThread().interrupt(); | ||||||||||||||||
| throw new IOException("Interrupted while processing files", e); | ||||||||||||||||
| } catch (ExecutionException e) { | ||||||||||||||||
| if (retryLog == null) { | ||||||||||||||||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. looks like queued up tasks are not canceled; on line 157 calls shutdown() (not shutdownNow()), i.e. it's going to block waiting for queued up tasks to complete even though tx won't be committed. |
||||||||||||||||
| throw new IOException("Error processing files", e.getCause()); | ||||||||||||||||
| } | ||||||||||||||||
| S3.BucketPath src = S3.bucketPath(dataFile); | ||||||||||||||||
| S3.BucketPath dst = S3.bucketPath(dstDataFile); | ||||||||||||||||
| logger.info("{}: fast copying to {}", file, dstDataFile); | ||||||||||||||||
| CopyObjectRequest copyReq = | ||||||||||||||||
| CopyObjectRequest.builder() | ||||||||||||||||
| .sourceBucket(src.bucket()) | ||||||||||||||||
| .sourceKey(src.path()) | ||||||||||||||||
| .destinationBucket(dst.bucket()) | ||||||||||||||||
| .destinationKey(dst.path()) | ||||||||||||||||
| .build(); | ||||||||||||||||
| s3ClientLazy.getValue().copyObject(copyReq); | ||||||||||||||||
| dataFileSizeInBytes = inputFile.getLength(); | ||||||||||||||||
| dataFile = dstDataFile; | ||||||||||||||||
| } else { | ||||||||||||||||
| String dstDataFile = dstDataFileSource.get(file); | ||||||||||||||||
| if (checkNotExists.apply(dstDataFile)) { | ||||||||||||||||
| continue; | ||||||||||||||||
| } | ||||||||||||||||
| OutputFile outputFile = | ||||||||||||||||
| tableIO.newOutputFile(replacePrefix(dstDataFile, "s3://", "s3a://")); | ||||||||||||||||
| // TODO: support transferTo below (note that compression, etc. might be different) | ||||||||||||||||
| // try (var d = outputFile.create()) { try (var s = inputFile.newStream()) { | ||||||||||||||||
| // s.transferTo(d); }} | ||||||||||||||||
| Parquet.ReadBuilder readBuilder = | ||||||||||||||||
| Parquet.read(inputFile) | ||||||||||||||||
| .createReaderFunc(s -> GenericParquetReaders.buildReader(tableSchema, s)) | ||||||||||||||||
| .project(tableSchema); // TODO: ? | ||||||||||||||||
| // TODO: reuseContainers? | ||||||||||||||||
| Parquet.WriteBuilder writeBuilder = | ||||||||||||||||
| Parquet.write(outputFile) | ||||||||||||||||
| .overwrite( | ||||||||||||||||
| dataFileNamingStrategy == DataFileNamingStrategy.Name.INPUT_FILENAME) | ||||||||||||||||
| .createWriterFunc(GenericParquetWriter::buildWriter) | ||||||||||||||||
| .schema(tableSchema); | ||||||||||||||||
| logger.info("{}: copying to {}", file, dstDataFile); | ||||||||||||||||
| // file size may have changed due to different compression, etc. | ||||||||||||||||
| dataFileSizeInBytes = copy(readBuilder, writeBuilder); | ||||||||||||||||
| dataFile = dstDataFile; | ||||||||||||||||
| } | ||||||||||||||||
| logger.info("{}: adding data file", file); | ||||||||||||||||
| long recordCount = | ||||||||||||||||
| metadata.getBlocks().stream().mapToLong(BlockMetaData::getRowCount).sum(); | ||||||||||||||||
| MetricsConfig metricsConfig = MetricsConfig.forTable(table); | ||||||||||||||||
| Metrics metrics = ParquetUtil.fileMetrics(inputFile, metricsConfig); | ||||||||||||||||
| df = | ||||||||||||||||
| new DataFiles.Builder(table.spec()) | ||||||||||||||||
| .withPath(dataFile) | ||||||||||||||||
| .withFormat("PARQUET") | ||||||||||||||||
| .withRecordCount(recordCount) | ||||||||||||||||
| .withFileSizeInBytes(dataFileSizeInBytes) | ||||||||||||||||
| .withMetrics(metrics) | ||||||||||||||||
| .build(); | ||||||||||||||||
| } catch (Exception e) { // FIXME | ||||||||||||||||
| if (retryLog != null) { | ||||||||||||||||
| logger.error("{}: error (adding to retry list and continuing)", file, e); | ||||||||||||||||
| retryLog.add(file); | ||||||||||||||||
| continue; | ||||||||||||||||
| } else { | ||||||||||||||||
| throw e; | ||||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
| atLeastOneFileAppended = true; | ||||||||||||||||
| appendOp.appendFile(df); | ||||||||||||||||
| } finally { | ||||||||||||||||
| // Cancel any remaining tasks since we won't commit the transaction | ||||||||||||||||
| if (finalOptions.noCommit() || !atLeastOneFileAppended) { | ||||||||||||||||
| executor.shutdownNow(); | ||||||||||||||||
| } else { | ||||||||||||||||
| executor.shutdown(); | ||||||||||||||||
| } | ||||||||||||||||
|
||||||||||||||||
| // Cancel any remaining tasks since we won't commit the transaction | |
| if (finalOptions.noCommit() || !atLeastOneFileAppended) { | |
| executor.shutdownNow(); | |
| } else { | |
| executor.shutdown(); | |
| } | |
| executor.shutdownNow(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should probably awaitTermiantion too (as shutdown/shutdownNow don't do that)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
note to self: retryLog::add is thread-safe so this is fine