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
54 changes: 52 additions & 2 deletions flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.hadoop.HadoopCatalog;
import org.apache.iceberg.hive.HiveCatalog;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;

/**
* Serializable loader to load an Iceberg {@link Catalog}.
Expand All @@ -33,10 +34,59 @@ public interface CatalogLoader extends Serializable {
Catalog loadCatalog(Configuration hadoopConf);

static CatalogLoader hadoop(String name, String warehouseLocation) {
return conf -> new HadoopCatalog(name, conf, warehouseLocation);
return new HadoopCatalogLoader(name, warehouseLocation);
}

static CatalogLoader hive(String name, String uri, int clientPoolSize) {
return conf -> new HiveCatalog(name, uri, clientPoolSize, conf);
return new HiveCatalogLoader(name, uri, clientPoolSize);
}

class HadoopCatalogLoader implements CatalogLoader {
private final String catalogName;
private final String warehouseLocation;

private HadoopCatalogLoader(String catalogName, String warehouseLocation) {
this.catalogName = catalogName;
this.warehouseLocation = warehouseLocation;
}

@Override
public Catalog loadCatalog(Configuration hadoopConf) {
return new HadoopCatalog(catalogName, hadoopConf, warehouseLocation);
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("catalogName", catalogName)
.add("warehouseLocation", warehouseLocation)
.toString();
}
}

class HiveCatalogLoader implements CatalogLoader {
private final String catalogName;
private final String uri;
private final int clientPoolSize;

private HiveCatalogLoader(String catalogName, String uri, int clientPoolSize) {
this.catalogName = catalogName;
this.uri = uri;
this.clientPoolSize = clientPoolSize;
}

@Override
public Catalog loadCatalog(Configuration hadoopConf) {
return new HiveCatalog(catalogName, uri, clientPoolSize, hadoopConf);
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("catalogName", catalogName)
.add("uri", uri)
.add("clientPoolSize", clientPoolSize)
.toString();
}
}
}
19 changes: 17 additions & 2 deletions flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.flink.table.api.TableSchema;
Expand All @@ -48,6 +49,7 @@
import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.factories.TableFactory;
import org.apache.flink.util.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CachingCatalog;
Expand Down Expand Up @@ -129,7 +131,7 @@ private Namespace toNamespace(String database) {
return Namespace.of(namespace);
}

private TableIdentifier toIdentifier(ObjectPath path) {
TableIdentifier toIdentifier(ObjectPath path) {
return TableIdentifier.of(toNamespace(path.getDatabaseName()), path.getObjectName());
}

Expand Down Expand Up @@ -292,7 +294,7 @@ public CatalogTable getTable(ObjectPath tablePath) throws TableNotExistException
return toCatalogTable(table);
}

private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException {
Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException {
try {
return icebergCatalog.loadTable(toIdentifier(tablePath));
} catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
Expand Down Expand Up @@ -505,6 +507,19 @@ static CatalogTable toCatalogTable(Table table) {
return new CatalogTableImpl(schema, partitionKeys, table.properties(), null);
}

@Override
public Optional<TableFactory> getTableFactory() {
return Optional.of(new FlinkTableFactory(this));
}

CatalogLoader getCatalogLoader() {
return catalogLoader;
}

Configuration getHadoopConf() {
return this.hadoopConf;
}

// ------------------------------ Unsupported methods ---------------------------------------------

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.util.List;
import java.util.Map;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.StreamTableSinkFactory;
import org.apache.flink.table.sinks.StreamTableSink;
import org.apache.flink.table.utils.TableSchemaUtils;

public class FlinkTableFactory implements StreamTableSinkFactory<RowData> {
private final FlinkCatalog catalog;

public FlinkTableFactory(FlinkCatalog catalog) {
this.catalog = catalog;
}

@Override
public StreamTableSink<RowData> createTableSink(Context context) {
ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
TableLoader tableLoader = createTableLoader(objectPath);
TableSchema tableSchema = getPhysicalSchema(context);
return new IcebergTableSink(context.isBounded(), tableLoader, catalog.getHadoopConf(), tableSchema);
}

@Override
public Map<String, String> requiredContext() {
throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI");
}

@Override
public List<String> supportedProperties() {
throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI");
}

private TableSchema getPhysicalSchema(Context context) {
return TableSchemaUtils.getPhysicalSchema(context.getTable().getSchema());
}

private TableLoader createTableLoader(ObjectPath objectPath) {
return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath));
}
}
90 changes: 90 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* 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.util.Map;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.sinks.AppendStreamTableSink;
import org.apache.flink.table.sinks.OverwritableTableSink;
import org.apache.flink.table.sinks.PartitionableTableSink;
import org.apache.flink.table.sinks.TableSink;
import org.apache.flink.table.types.DataType;
import org.apache.flink.util.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.flink.sink.FlinkSink;

public class IcebergTableSink implements AppendStreamTableSink<RowData>, OverwritableTableSink, PartitionableTableSink {
private final boolean isBounded;
private final TableLoader tableLoader;
private final Configuration hadoopConf;
private final TableSchema tableSchema;

private boolean overwrite = false;

public IcebergTableSink(boolean isBounded, TableLoader tableLoader, Configuration hadoopConf,
TableSchema tableSchema) {
this.isBounded = isBounded;
this.tableLoader = tableLoader;
this.hadoopConf = hadoopConf;
this.tableSchema = tableSchema;
}

@Override
public DataStreamSink<?> consumeDataStream(DataStream<RowData> dataStream) {
Preconditions.checkState(!overwrite || isBounded, "Unbounded data stream doesn't support overwrite operation.");

return FlinkSink.forRowData(dataStream)
.tableLoader(tableLoader)
.hadoopConf(hadoopConf)
.tableSchema(tableSchema)
.overwrite(overwrite)
.build();
}

@Override
public DataType getConsumedDataType() {
return tableSchema.toRowDataType().bridgedTo(RowData.class);
}

@Override
public TableSchema getTableSchema() {
return this.tableSchema;
}

@Override
public TableSink<RowData> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
// This method has been deprecated and it will be removed in future version, so left the empty implementation here.
return this;
}

@Override
public void setOverwrite(boolean overwrite) {
this.overwrite = overwrite;
}

@Override
public void setStaticPartition(Map<String, String> partitions) {
// The flink's PartitionFanoutWriter will handle the static partition write policy automatically.
}
}
16 changes: 16 additions & 0 deletions flink/src/main/java/org/apache/iceberg/flink/TableLoader.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;

/**
* Serializable loader to load an Iceberg {@link Table}.
Expand Down Expand Up @@ -71,6 +72,13 @@ public Table loadTable() {
@Override
public void close() {
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("location", location)
.toString();
}
}

class CatalogTableLoader implements TableLoader {
Expand Down Expand Up @@ -103,5 +111,13 @@ public void close() throws IOException {
((Closeable) catalog).close();
}
}

@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("tableIdentifier", identifier)
.add("catalogLoader", catalogLoader)
.toString();
}
}
}
20 changes: 18 additions & 2 deletions flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.iceberg.flink.sink;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Locale;
import java.util.Map;
import org.apache.flink.api.common.functions.MapFunction;
Expand Down Expand Up @@ -110,6 +112,7 @@ public static class Builder {
private Configuration hadoopConf;
private Table table;
private TableSchema tableSchema;
private boolean overwrite = false;

private Builder() {
}
Expand Down Expand Up @@ -155,16 +158,29 @@ public Builder tableSchema(TableSchema newTableSchema) {
return this;
}

public Builder overwrite(boolean newOverwrite) {
this.overwrite = newOverwrite;
return this;
}

@SuppressWarnings("unchecked")
public DataStreamSink<RowData> build() {
Preconditions.checkArgument(rowDataInput != null,
"Please use forRowData() to initialize the input DataStream.");
Preconditions.checkNotNull(table, "Table shouldn't be null");
Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");

if (table == null) {
tableLoader.open(hadoopConf);
try (TableLoader loader = tableLoader) {
this.table = loader.loadTable();
} catch (IOException e) {
throw new UncheckedIOException("Failed to load iceberg table from table loader: " + tableLoader, e);
}
}

IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf, overwrite);

DataStream<Void> returnStream = rowDataInput
.transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
Expand Down
Loading