JingsongLi commented on a change in pull request #1145: URL: https://github.com/apache/iceberg/pull/1145#discussion_r457819343
########## File path: flink/src/main/java/org/apache/iceberg/flink/IcebergStreamWriter.java ########## @@ -0,0 +1,226 @@ +/* + * 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; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Locale; +import java.util.Map; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.OutputFileFactory; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.flink.writer.FileAppenderFactory; +import org.apache.iceberg.flink.writer.TaskWriter; +import org.apache.iceberg.flink.writer.TaskWriterFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; + +class IcebergStreamWriter extends AbstractStreamOperator<DataFile> + implements OneInputStreamOperator<Row, DataFile>, BoundedOneInput { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(IcebergStreamWriter.class); + + private final String tablePath; + private final PartitionSpec spec; + private final LocationProvider locations; + private final Map<String, String> properties; + private final FileIO io; + private final EncryptionManager encryptionManager; + private final Schema schema; + + private transient TaskWriter<Row> writer; + private transient int subTaskId; + private transient int attemptId; + + private IcebergStreamWriter(String tablePath, PartitionSpec spec, LocationProvider locations, + Map<String, String> properties, FileIO io, EncryptionManager encryptionManager, + Schema schema) { + this.tablePath = tablePath; + this.spec = spec; + this.locations = locations; + this.properties = properties; + this.io = io; + this.encryptionManager = encryptionManager; + this.schema = schema; + } + + @Override + public void open() { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + + // Initialize the task writer. + FileFormat fileFormat = getFileFormat(); + long targetFileSize = getTargetFileSizeBytes(); + FileAppenderFactory<Row> appenderFactory = new FlinkFileAppenderFactory(schema, properties); + OutputFileFactory outputFileFactory = new OutputFileFactory(spec, fileFormat, locations, io, + encryptionManager, subTaskId, attemptId); + this.writer = TaskWriterFactory.createTaskWriter(schema, spec, appenderFactory, outputFileFactory, + targetFileSize, fileFormat); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + LOG.info("Iceberg writer({}) begin preparing for checkpoint {}", toString(), checkpointId); + // close all open files and emit files to downstream committer operator + writer.close(); + + writer.pollCompleteFiles().forEach(this::emit); + LOG.info("Iceberg writer({}) completed preparing for checkpoint {}", toString(), checkpointId); + } + + @Override + public void processElement(StreamRecord<Row> element) throws Exception { + Row value = element.getValue(); + writer.append(value); + + // Emit the data file entries to downstream committer operator if there exist any complete files. + writer.pollCompleteFiles().forEach(this::emit); + } + + @Override + public void dispose() throws Exception { + super.dispose(); + if (writer != null) { + writer.close(); + writer = null; + } + } + + @Override + public void endInput() throws IOException { + // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the remaining + // data files to downstream before closing the writer so that we won't miss any of them. + writer.close(); + for (DataFile dataFile : writer.pollCompleteFiles()) { + emit(dataFile); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("table_path", tablePath) + .add("subtask_id", subTaskId) + .add("attempt_id", attemptId) + .toString(); + } + + private void emit(DataFile dataFile) { + output.collect(new StreamRecord<>(dataFile)); + } + + private FileFormat getFileFormat() { + String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + return FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH)); + } + + private long getTargetFileSizeBytes() { + return PropertyUtil.propertyAsLong(properties, + WRITE_TARGET_FILE_SIZE_BYTES, + WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + } + + private static class FlinkFileAppenderFactory implements FileAppenderFactory<Row> { + private final Schema schema; + private final Map<String, String> props; + + private FlinkFileAppenderFactory(Schema schema, Map<String, String> props) { + this.schema = schema; + this.props = props; + } + + @Override + public FileAppender<Row> newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); + try { + switch (format) { + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(FlinkParquetWriters::buildWriter) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(FlinkAvroWriter::new) + .setAll(props) + .schema(schema) + .overwrite() + .build(); + + case ORC: + default: + throw new UnsupportedOperationException("Cannot write unknown format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + static IcebergStreamWriter createStreamWriter(String path, TableSchema tableSchema, Configuration conf) { Review comment: Can we just pass a Iceberg `Table` instead of `path` here? We can get ride of `TableUtil.findTable` (Or just move it to test). ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
