forked from apache/iceberg
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Flink: support limit pushdown in FLIP-27 source (apache#10748)
- Loading branch information
Showing
6 changed files
with
228 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
56 changes: 56 additions & 0 deletions
56
....19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.source.reader; | ||
|
||
import org.apache.iceberg.CombinedScanTask; | ||
import org.apache.iceberg.encryption.EncryptionManager; | ||
import org.apache.iceberg.flink.source.DataIterator; | ||
import org.apache.iceberg.flink.source.FileScanTaskReader; | ||
import org.apache.iceberg.io.FileIO; | ||
import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
||
class LimitableDataIterator<T> extends DataIterator<T> { | ||
private final RecordLimiter limiter; | ||
|
||
LimitableDataIterator( | ||
FileScanTaskReader<T> fileScanTaskReader, | ||
CombinedScanTask task, | ||
FileIO io, | ||
EncryptionManager encryption, | ||
RecordLimiter limiter) { | ||
super(fileScanTaskReader, task, io, encryption); | ||
Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); | ||
this.limiter = limiter; | ||
} | ||
|
||
@Override | ||
public boolean hasNext() { | ||
if (limiter.reachedLimit()) { | ||
return false; | ||
} | ||
|
||
return super.hasNext(); | ||
} | ||
|
||
@Override | ||
public T next() { | ||
limiter.increment(); | ||
return super.next(); | ||
} | ||
} |
45 changes: 45 additions & 0 deletions
45
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.source.reader; | ||
|
||
import java.util.concurrent.atomic.AtomicLong; | ||
import org.apache.flink.annotation.Internal; | ||
|
||
@Internal | ||
class RecordLimiter { | ||
private final long limit; | ||
private final AtomicLong counter; | ||
|
||
static RecordLimiter create(long limit) { | ||
return new RecordLimiter(limit); | ||
} | ||
|
||
private RecordLimiter(long limit) { | ||
this.limit = limit; | ||
this.counter = new AtomicLong(0); | ||
} | ||
|
||
public boolean reachedLimit() { | ||
return limit > 0 && counter.get() >= limit; | ||
} | ||
|
||
public void increment() { | ||
counter.incrementAndGet(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
84 changes: 84 additions & 0 deletions
84
...flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,84 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.source.reader; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
import java.nio.file.Path; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import org.apache.flink.table.data.RowData; | ||
import org.apache.iceberg.CombinedScanTask; | ||
import org.apache.iceberg.FileFormat; | ||
import org.apache.iceberg.data.GenericAppenderFactory; | ||
import org.apache.iceberg.data.Record; | ||
import org.apache.iceberg.encryption.EncryptionManager; | ||
import org.apache.iceberg.encryption.PlaintextEncryptionManager; | ||
import org.apache.iceberg.flink.TestFixtures; | ||
import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; | ||
import org.apache.iceberg.hadoop.HadoopFileIO; | ||
import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
import org.junit.jupiter.api.BeforeAll; | ||
import org.junit.jupiter.api.io.TempDir; | ||
import org.junit.jupiter.params.ParameterizedTest; | ||
import org.junit.jupiter.params.provider.ValueSource; | ||
|
||
public class TestLimitableDataIterator { | ||
@TempDir private static Path temporaryFolder; | ||
|
||
private final RowDataFileScanTaskReader reader = | ||
new RowDataFileScanTaskReader( | ||
TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); | ||
private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); | ||
private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); | ||
|
||
private static CombinedScanTask combinedScanTask; | ||
private static int totalRecords; | ||
|
||
@BeforeAll | ||
public static void beforeClass() throws Exception { | ||
GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); | ||
List<List<Record>> recordBatchList = | ||
ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); | ||
combinedScanTask = | ||
ReaderUtil.createCombinedScanTask( | ||
recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); | ||
totalRecords = 3 * 2; | ||
} | ||
|
||
@ParameterizedTest | ||
@ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) | ||
public void testUnlimited(long limit) { | ||
LimitableDataIterator<RowData> dataIterator = | ||
new LimitableDataIterator<>( | ||
reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); | ||
|
||
List<RowData> result = Lists.newArrayList(); | ||
while (dataIterator.hasNext()) { | ||
result.add(dataIterator.next()); | ||
} | ||
|
||
if (limit <= 0 || limit > totalRecords) { | ||
// read all records | ||
assertThat(result).hasSize(totalRecords); | ||
} else { | ||
assertThat(result).hasSize((int) limit); | ||
} | ||
} | ||
} |