-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Add a rewrite datafile action for flink #1623
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
Merged
Merged
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
d02e742
RewriteDataFilesAction for flink
0ffd260
RewriteDataFilesAction for flink
b87a3eb
fix unit test
1d19c03
fix checkstyle
607885e
RewriteDataFilesAction for flink
634e9c4
fix Typo
6b011aa
use DataStream instead of dataset
72c5a98
set parallelism for map
aa78d40
add max Parallelism for RewriteDataFilesAction
054ca38
move maxParallelism to flink RewriteDataFilesAction
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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
47 changes: 47 additions & 0 deletions
47
flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
This file contains hidden or 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,47 @@ | ||
| /* | ||
| * 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.actions; | ||
|
|
||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.iceberg.Table; | ||
|
|
||
| public class Actions { | ||
|
|
||
| private StreamExecutionEnvironment env; | ||
| private Table table; | ||
|
|
||
| private Actions(StreamExecutionEnvironment env, Table table) { | ||
| this.env = env; | ||
| this.table = table; | ||
| } | ||
|
|
||
| public static Actions forTable(StreamExecutionEnvironment env, Table table) { | ||
| return new Actions(env, table); | ||
| } | ||
|
|
||
| public static Actions forTable(Table table) { | ||
| return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(), table); | ||
| } | ||
|
|
||
| public RewriteDataFilesAction rewriteDataFiles() { | ||
| return new RewriteDataFilesAction(env, table); | ||
| } | ||
|
|
||
| } |
68 changes: 68 additions & 0 deletions
68
flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java
This file contains hidden or 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,68 @@ | ||
| /* | ||
| * 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.actions; | ||
|
|
||
| import java.util.List; | ||
| import org.apache.flink.streaming.api.datastream.DataStream; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.iceberg.CombinedScanTask; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.actions.BaseRewriteDataFilesAction; | ||
| import org.apache.iceberg.flink.source.RowDataRewriter; | ||
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
||
| public class RewriteDataFilesAction extends BaseRewriteDataFilesAction<RewriteDataFilesAction> { | ||
|
|
||
| private StreamExecutionEnvironment env; | ||
| private int maxParallelism; | ||
|
|
||
| public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { | ||
| super(table); | ||
| this.env = env; | ||
| this.maxParallelism = env.getParallelism(); | ||
| } | ||
|
|
||
| @Override | ||
| protected FileIO fileIO() { | ||
| return table().io(); | ||
| } | ||
|
|
||
| @Override | ||
| protected List<DataFile> rewriteDataForTasks(List<CombinedScanTask> combinedScanTasks) { | ||
| int size = combinedScanTasks.size(); | ||
| int parallelism = Math.min(size, maxParallelism); | ||
| DataStream<CombinedScanTask> dataStream = env.fromCollection(combinedScanTasks); | ||
| RowDataRewriter rowDataRewriter = new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); | ||
| List<DataFile> addedDataFiles = rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); | ||
| return addedDataFiles; | ||
| } | ||
|
|
||
| @Override | ||
| protected RewriteDataFilesAction self() { | ||
| return this; | ||
| } | ||
|
|
||
| public void maxParallelism(int parallelism) { | ||
| Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %d", parallelism); | ||
| this.maxParallelism = parallelism; | ||
| } | ||
| } |
This file contains hidden or 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
155 changes: 155 additions & 0 deletions
155
flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java
This file contains hidden or 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,155 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.List; | ||
| import java.util.Locale; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.flink.api.common.functions.RichMapFunction; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.streaming.api.datastream.DataStream; | ||
| import org.apache.flink.streaming.api.datastream.DataStreamUtils; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.iceberg.CombinedScanTask; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.TableProperties; | ||
| import org.apache.iceberg.encryption.EncryptionManager; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
| import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; | ||
| import org.apache.iceberg.flink.sink.TaskWriterFactory; | ||
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.io.TaskWriter; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.util.PropertyUtil; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; | ||
|
|
||
| public class RowDataRewriter { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); | ||
|
|
||
| private final Schema schema; | ||
| private final FileFormat format; | ||
| private final String nameMapping; | ||
| private final FileIO io; | ||
| private final boolean caseSensitive; | ||
| private final EncryptionManager encryptionManager; | ||
| private final TaskWriterFactory<RowData> taskWriterFactory; | ||
|
|
||
| public RowDataRewriter(Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { | ||
| this.schema = table.schema(); | ||
| this.caseSensitive = caseSensitive; | ||
| this.io = io; | ||
| this.encryptionManager = encryptionManager; | ||
| this.nameMapping = PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); | ||
|
|
||
| String formatString = PropertyUtil.propertyAsString(table.properties(), TableProperties.DEFAULT_FILE_FORMAT, | ||
openinx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); | ||
| this.format = FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH)); | ||
| RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); | ||
| this.taskWriterFactory = new RowDataTaskWriterFactory( | ||
| table.schema(), | ||
| flinkSchema, | ||
| table.spec(), | ||
| table.locationProvider(), | ||
| io, | ||
| encryptionManager, | ||
| Long.MAX_VALUE, | ||
openinx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| format, | ||
| table.properties()); | ||
| } | ||
|
|
||
| public List<DataFile> rewriteDataForTasks(DataStream<CombinedScanTask> dataStream, int parallelism) { | ||
| RewriteMap map = new RewriteMap(schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); | ||
| DataStream<List<DataFile>> ds = dataStream.map(map).setParallelism(parallelism); | ||
| return Lists.newArrayList(DataStreamUtils.collect(ds)).stream().flatMap(Collection::stream) | ||
| .collect(Collectors.toList()); | ||
| } | ||
|
|
||
| public static class RewriteMap extends RichMapFunction<CombinedScanTask, List<DataFile>> { | ||
|
|
||
| private TaskWriter<RowData> writer; | ||
| private int subTaskId; | ||
| private int attemptId; | ||
|
|
||
| private final Schema schema; | ||
| private final String nameMapping; | ||
| private final FileIO io; | ||
| private final boolean caseSensitive; | ||
| private final EncryptionManager encryptionManager; | ||
| private final TaskWriterFactory<RowData> taskWriterFactory; | ||
|
|
||
| public RewriteMap(Schema schema, String nameMapping, FileIO io, boolean caseSensitive, | ||
| EncryptionManager encryptionManager, TaskWriterFactory<RowData> taskWriterFactory) { | ||
| this.schema = schema; | ||
| this.nameMapping = nameMapping; | ||
| this.io = io; | ||
| this.caseSensitive = caseSensitive; | ||
| this.encryptionManager = encryptionManager; | ||
| this.taskWriterFactory = taskWriterFactory; | ||
| } | ||
|
|
||
| @Override | ||
| public void open(Configuration parameters) { | ||
| this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); | ||
| this.attemptId = getRuntimeContext().getAttemptNumber(); | ||
| // Initialize the task writer factory. | ||
| this.taskWriterFactory.initialize(subTaskId, attemptId); | ||
| } | ||
|
|
||
openinx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| @Override | ||
| public List<DataFile> map(CombinedScanTask task) throws Exception { | ||
| // Initialize the task writer. | ||
| this.writer = taskWriterFactory.create(); | ||
| try (RowDataIterator iterator = | ||
| new RowDataIterator(task, io, encryptionManager, schema, schema, nameMapping, caseSensitive)) { | ||
| while (iterator.hasNext()) { | ||
| RowData rowData = iterator.next(); | ||
| writer.write(rowData); | ||
| } | ||
| return Lists.newArrayList(writer.complete()); | ||
| } catch (Throwable originalThrowable) { | ||
| try { | ||
openinx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); | ||
| writer.abort(); | ||
| LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); | ||
| } catch (Throwable inner) { | ||
| if (originalThrowable != inner) { | ||
| originalThrowable.addSuppressed(inner); | ||
openinx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); | ||
| } | ||
| } | ||
|
|
||
| if (originalThrowable instanceof Exception) { | ||
| throw originalThrowable; | ||
| } else { | ||
| throw new RuntimeException(originalThrowable); | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or 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
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.