-
Notifications
You must be signed in to change notification settings - Fork 3k
Core: read delete files in parallel #3120
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 all commits
5521a34
5c4f33d
288da5e
57f858a
9203a2a
7382b5e
b17a1fd
944d6ac
5da4cf1
f5692a2
f53b20e
0c812ef
9b186c6
9a3c259
004511f
40dfec8
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 |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.util; | ||
| package org.apache.iceberg.io; | ||
|
|
||
| import java.io.Closeable; | ||
| import java.io.IOException; | ||
|
|
@@ -28,25 +28,39 @@ | |
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Future; | ||
| import org.apache.iceberg.exceptions.RuntimeIOException; | ||
| import org.apache.iceberg.io.CloseableGroup; | ||
| import org.apache.iceberg.io.CloseableIterable; | ||
| import org.apache.iceberg.io.CloseableIterator; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Iterables; | ||
|
|
||
| /** | ||
| * Run iterables in parallel. | ||
| * @deprecated please use {@link CloseableIterable#combine(Iterable, ExecutorService, int)} instead. | ||
| */ | ||
| @Deprecated | ||
|
Contributor
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. This class is not deprecated. It is still public and will not be removed. This should be created using
Contributor
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. I see the module here also changed. I don't think we are allowed to move public class to another module?
Contributor
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. Good catch, @jackye1995! |
||
| public class ParallelIterable<T> extends CloseableGroup implements CloseableIterable<T> { | ||
Reo-LEI marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| private final Iterable<? extends Iterable<T>> iterables; | ||
| private final ExecutorService workerPool; | ||
| private final int workerPoolSize; | ||
|
|
||
| /** | ||
| * @deprecated please use {@link CloseableIterable#combine(Iterable, ExecutorService, int)} instead. | ||
|
Contributor
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. You can omit "please" from documentation so that docs are direct and as short as possible. Also, can you add "will be removed in 0.14.0"? We like to keep track of when things can be removed. |
||
| */ | ||
| @Deprecated | ||
| public ParallelIterable(Iterable<? extends Iterable<T>> iterables, | ||
| ExecutorService workerPool) { | ||
| this(iterables, workerPool, Runtime.getRuntime().availableProcessors()); | ||
| } | ||
|
|
||
| ParallelIterable(Iterable<? extends Iterable<T>> iterables, | ||
| ExecutorService workerPool, | ||
| int workerPoolSize) { | ||
rdblue marked this conversation as resolved.
Show resolved
Hide resolved
Contributor
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. Can you fix the indentation here? |
||
| this.iterables = iterables; | ||
| this.workerPool = workerPool; | ||
| this.workerPoolSize = workerPoolSize; | ||
| } | ||
|
|
||
| @Override | ||
| public CloseableIterator<T> iterator() { | ||
| ParallelIterator<T> iter = new ParallelIterator<>(iterables, workerPool); | ||
| ParallelIterator<T> iter = new ParallelIterator<>(iterables, workerPool, workerPoolSize); | ||
| addCloseable(iter); | ||
| return iter; | ||
| } | ||
|
|
@@ -59,7 +73,8 @@ private static class ParallelIterator<T> implements CloseableIterator<T> { | |
| private boolean closed = false; | ||
|
|
||
| private ParallelIterator(Iterable<? extends Iterable<T>> iterables, | ||
| ExecutorService workerPool) { | ||
| ExecutorService workerPool, | ||
| int workerPoolSize) { | ||
| this.tasks = Iterables.transform(iterables, iterable -> | ||
| (Runnable) () -> { | ||
| try (Closeable ignored = (iterable instanceof Closeable) ? | ||
|
|
@@ -72,8 +87,8 @@ private ParallelIterator(Iterable<? extends Iterable<T>> iterables, | |
| } | ||
| }).iterator(); | ||
| this.workerPool = workerPool; | ||
| // submit 2 tasks per worker at a time | ||
| this.taskFutures = new Future[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; | ||
| // submit 2 tasks per worker at a time. | ||
|
Contributor
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. Can you remove the non-functional change on this line? We don't want unnecessary changes to cause commit conflicts. |
||
| this.taskFutures = new Future[workerPoolSize * 2]; | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,7 +30,6 @@ | |
| import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||
| import org.apache.iceberg.types.TypeUtil; | ||
| import org.apache.iceberg.types.Types.StructType; | ||
| import org.apache.iceberg.util.ParallelIterable; | ||
| import org.apache.iceberg.util.ThreadPools; | ||
|
|
||
| /** | ||
|
|
@@ -128,9 +127,9 @@ protected CloseableIterable<FileScanTask> planFiles( | |
| } | ||
|
|
||
| private static CloseableIterable<ManifestFile> allDataManifestFiles(List<Snapshot> snapshots) { | ||
| try (CloseableIterable<ManifestFile> iterable = new ParallelIterable<>( | ||
| Iterables.transform(snapshots, snapshot -> (Iterable<ManifestFile>) () -> snapshot.dataManifests().iterator()), | ||
| ThreadPools.getWorkerPool())) { | ||
| try (CloseableIterable<ManifestFile> iterable = CloseableIterable.combine( | ||
| Iterables.transform(snapshots, snapshot -> CloseableIterable.withNoopClose(snapshot.dataManifests())), | ||
|
Contributor
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. I don't think that this should add a noop close here. Can you avoid adding this in all the updated calls to create a parallel iterable?
Contributor
Author
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. I think I can't avoid adding this, becasue
Contributor
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. This does need to be fixed. |
||
| ThreadPools.getWorkerPool(), ThreadPools.WORKER_THREAD_POOL_SIZE)) { | ||
| return CloseableIterable.withNoopClose(Sets.newHashSet(iterable)); | ||
| } catch (IOException e) { | ||
| throw new RuntimeIOException(e, "Failed to close parallel iterable"); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -39,11 +39,24 @@ private SystemProperties() { | |
| */ | ||
| public static final String SCAN_THREAD_POOL_ENABLED = "iceberg.scan.plan-in-worker-pool"; | ||
|
|
||
| static boolean getBoolean(String systemProperty, boolean defaultValue) { | ||
| /** | ||
| * Whether to use the shared worker pool when planning table scans. | ||
| */ | ||
| public static final String READ_DELETE_FILES_WORKER_POOL_SIZE = "iceberg.worker.read-deletes-num-threads"; | ||
|
|
||
| public static boolean getBoolean(String systemProperty, boolean defaultValue) { | ||
|
Contributor
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. As I said elsewhere, I don't think that this feature should be controlled through a system property. This should be a Flin-specific property for now and we can introduce a similar config for Spark later. But since this violates Spark's threading model on executors, we don't want to make this global. |
||
| String value = System.getProperty(systemProperty); | ||
| if (value != null) { | ||
| return Boolean.parseBoolean(value); | ||
| } | ||
| return defaultValue; | ||
| } | ||
|
|
||
| public static int getInteger(String systemProperty, int defaultValue) { | ||
| String value = System.getProperty(systemProperty); | ||
| if (value != null) { | ||
| return Integer.parseInt(value); | ||
| } | ||
| return defaultValue; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -77,8 +77,9 @@ public static <T> CloseableIterable<T> filter(CloseableIterable<T> rows, Functio | |
| return filter.filter(rows); | ||
| } | ||
|
|
||
| public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDeletes, Types.StructType eqType) { | ||
| try (CloseableIterable<StructLike> deletes = eqDeletes) { | ||
| public static <T extends StructLike> StructLikeSet toEqualitySet(CloseableIterable<T> eqDeletes, | ||
| Types.StructType eqType) { | ||
| try (CloseableIterable<T> deletes = eqDeletes) { | ||
|
Contributor
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. Why was this change needed?
Contributor
Author
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. To avoid to transform
Contributor
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. Seems reasonable. |
||
| StructLikeSet deleteSet = StructLikeSet.create(eqType); | ||
| Iterables.addAll(deleteSet, deletes); | ||
| return deleteSet; | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,9 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.function.Predicate; | ||
| import org.apache.iceberg.Accessor; | ||
| import org.apache.iceberg.DataFile; | ||
|
|
@@ -32,6 +35,7 @@ | |
| import org.apache.iceberg.MetadataColumns; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.StructLike; | ||
| import org.apache.iceberg.SystemProperties; | ||
| import org.apache.iceberg.avro.Avro; | ||
| import org.apache.iceberg.data.avro.DataReader; | ||
| import org.apache.iceberg.data.orc.GenericOrcReader; | ||
|
|
@@ -49,6 +53,8 @@ | |
| import org.apache.iceberg.relocated.com.google.common.collect.Multimap; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Multimaps; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||
| import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; | ||
| import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
| import org.apache.iceberg.types.TypeUtil; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.iceberg.util.Filter; | ||
|
|
@@ -62,6 +68,13 @@ public abstract class DeleteFilter<T> { | |
| MetadataColumns.DELETE_FILE_PATH, | ||
| MetadataColumns.DELETE_FILE_POS); | ||
|
|
||
| private static final int READ_DELETES_WORKER_POOL_SIZE_DEFAULT = 0; // read delete files in serial. | ||
| private static final int READ_DELETES_WORKER_POOL_SIZE = SystemProperties.getInteger( | ||
| SystemProperties.READ_DELETE_FILES_WORKER_POOL_SIZE, READ_DELETES_WORKER_POOL_SIZE_DEFAULT); | ||
| private static final ExecutorService READ_DELETES_SERVICE = READ_DELETES_WORKER_POOL_SIZE <= 1 ? null : | ||
| MoreExecutors.getExitingExecutorService((ThreadPoolExecutor) Executors.newFixedThreadPool( | ||
| READ_DELETES_WORKER_POOL_SIZE, new ThreadFactoryBuilder().setNameFormat("Read-delete-Service-%d").build())); | ||
|
Contributor
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. It looks like the changes other than the ones in this file are to make it easier to use |
||
|
|
||
| private final long setFilterThreshold; | ||
| private final DataFile dataFile; | ||
| private final List<DeleteFile> posDeletes; | ||
|
|
@@ -137,13 +150,10 @@ private List<Predicate<T>> applyEqDeletes() { | |
| Iterable<CloseableIterable<Record>> deleteRecords = Iterables.transform(deletes, | ||
| delete -> openDeletes(delete, deleteSchema)); | ||
|
|
||
| // copy the delete records because they will be held in a set | ||
| CloseableIterable<Record> records = CloseableIterable.transform( | ||
| CloseableIterable.concat(deleteRecords), Record::copy); | ||
|
|
||
| StructLikeSet deleteSet = Deletes.toEqualitySet( | ||
| CloseableIterable.transform( | ||
| records, record -> new InternalRecordWrapper(deleteSchema.asStruct()).wrap(record)), | ||
| CloseableIterable.combine(deleteRecords, READ_DELETES_SERVICE, READ_DELETES_WORKER_POOL_SIZE), | ||
| record -> new InternalRecordWrapper(deleteSchema.asStruct()).wrap(record)), | ||
| deleteSchema.asStruct()); | ||
|
|
||
| Predicate<T> isInDeleteSet = record -> deleteSet.contains(projectRow.wrap(asStructLike(record))); | ||
|
|
@@ -196,7 +206,8 @@ private CloseableIterable<T> applyPosDeletes(CloseableIterable<T> records) { | |
| if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < setFilterThreshold) { | ||
| return Deletes.filter( | ||
| records, this::pos, | ||
| Deletes.toPositionSet(dataFile.path(), CloseableIterable.concat(deletes))); | ||
| Deletes.toPositionSet(dataFile.path(), | ||
| CloseableIterable.combine(deletes, READ_DELETES_SERVICE, READ_DELETES_WORKER_POOL_SIZE))); | ||
| } | ||
|
|
||
| return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(dataFile.path(), deletes)); | ||
|
|
@@ -212,14 +223,12 @@ private CloseableIterable<Record> openDeletes(DeleteFile deleteFile, Schema dele | |
| case AVRO: | ||
| return Avro.read(input) | ||
| .project(deleteSchema) | ||
| .reuseContainers() | ||
|
Contributor
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. is this related?
Contributor
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. I think this is needed because records are now placed on a queue inside the parallel iterator. Reusing the record instance instead of copying would cause a problem. |
||
| .createReaderFunc(DataReader::create) | ||
| .build(); | ||
|
|
||
| case PARQUET: | ||
| Parquet.ReadBuilder builder = Parquet.read(input) | ||
| .project(deleteSchema) | ||
| .reuseContainers() | ||
| .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(deleteSchema, fileSchema)); | ||
|
|
||
| if (deleteFile.content() == FileContent.POSITION_DELETES) { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.