Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions core/src/main/java/org/apache/iceberg/util/PropertyUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -52,4 +52,13 @@ public static long propertyAsLong(Map<String, String> properties,
}
return defaultValue;
}

public static String propertyAsString(Map<String, String> properties,
String property, String defaultValue) {
String value = properties.get(property);
if (value != null) {
return properties.get(property);
}
return defaultValue;
}
}
47 changes: 47 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java
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);
}

}
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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
*
* @param <T> data type of record.
*/
interface TaskWriterFactory<T> extends Serializable {
public interface TaskWriterFactory<T> extends Serializable {

/**
* Initialize the factory with a given taskId and attemptId.
Expand Down
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,
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,
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);
}

@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 {
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);
LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner);
}
}

if (originalThrowable instanceof Exception) {
throw originalThrowable;
} else {
throw new RuntimeException(originalThrowable);
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ protected TableEnvironment getTableEnv() {
return tEnv;
}

List<Object[]> sql(String query, Object... args) {
protected List<Object[]> sql(String query, Object... args) {
TableResult tableResult = getTableEnv().executeSql(String.format(query, args));
tableResult.getJobClient().ifPresent(c -> {
try {
Expand Down
Loading