You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/07/23 11:52:48 UTC

[GitHub] [inlong] yunqingmoswu commented on a diff in pull request #5141: [INLONG-5131][Sort] Import all changelog mode data ingest into Iiceberg or DLC

yunqingmoswu commented on code in PR #5141:
URL: https://github.com/apache/inlong/pull/5141#discussion_r928113966


##########
inlong-sort/sort-connectors/iceberg/src/main/java/org/apache/inlong/sort/iceberg/FlinkDynamicTableFactory.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.inlong.sort.iceberg;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynMethods;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.flink.IcebergTableSource;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Copy from org.apache.iceberg.flink:iceberg-flink-runtime-1.13:0.13.1
+ *
+ * <p>
+ * Factory for creating configured instances of {@link IcebergTableSource} and {@link
+ * IcebergTableSink}.We modify KafkaDynamicTableSink to support append-mode .
+ * </p>
+ */
+public class FlinkDynamicTableFactory implements DynamicTableSinkFactory, DynamicTableSourceFactory {
+
+    static final String FACTORY_IDENTIFIER = "iceberg-inlong";
+
+    private static final ConfigOption<String> CATALOG_NAME =
+            ConfigOptions.key("catalog-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Catalog name");
+
+    private static final ConfigOption<String> CATALOG_TYPE =
+            ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE)
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Catalog type, the optional types are: custom, hadoop, hive.");
+
+    private static final ConfigOption<String> CATALOG_DATABASE =
+            ConfigOptions.key("catalog-database")
+                    .stringType()
+                    .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME)
+                    .withDescription("Database name managed in the iceberg catalog.");
+
+    private static final ConfigOption<String> CATALOG_TABLE =
+            ConfigOptions.key("catalog-table")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Table name managed in the underlying iceberg catalog and database.");
+
+    // Flink 1.13.x change the return type from CatalogTable interface to ResolvedCatalogTable which extends the
+    // CatalogTable. Here we use the dynamic method loading approach to avoid adding explicit CatalogTable or
+    // ResolvedCatalogTable class into the iceberg-flink-runtime jar for compatibility purpose.
+    private static final DynMethods.UnboundMethod GET_CATALOG_TABLE = DynMethods.builder("getCatalogTable")
+            .impl(Context.class, "getCatalogTable")
+            .orNoop()
+            .build();
+
+    private final FlinkCatalog catalog;
+
+    public FlinkDynamicTableFactory() {
+        this.catalog = null;
+    }
+
+    public FlinkDynamicTableFactory(FlinkCatalog catalog) {
+        this.catalog = catalog;
+    }
+
+    private static CatalogTable loadCatalogTable(Context context) {
+        return GET_CATALOG_TABLE.invoke(context);
+    }
+
+    private static TableLoader createTableLoader(CatalogBaseTable catalogBaseTable,
+            Map<String, String> tableProps,
+            String databaseName,
+            String tableName) {
+        Configuration flinkConf = new Configuration();
+        tableProps.forEach(flinkConf::setString);
+
+        String catalogName = flinkConf.getString(CATALOG_NAME);
+        Preconditions.checkNotNull(catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key());
+
+        String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName);
+        Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null");
+
+        String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName);
+        Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null");
+
+        org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf();
+        FlinkCatalogFactory factory = new FlinkCatalogFactory();
+        FlinkCatalog flinkCatalog = (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf);
+        ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable);
+
+        // Create database if not exists in the external catalog.
+        if (!flinkCatalog.databaseExists(catalogDatabase)) {
+            try {
+                flinkCatalog.createDatabase(catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true);
+            } catch (DatabaseAlreadyExistException e) {
+                throw new AlreadyExistsException(e, "Database %s already exists in the iceberg catalog %s.",
+                        catalogName,
+                        catalogDatabase);
+            }
+        }
+
+        // Create table if not exists in the external catalog.
+        if (!flinkCatalog.tableExists(objectPath)) {
+            try {
+                flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true);
+            } catch (TableAlreadyExistException e) {
+                throw new AlreadyExistsException(e, "Table %s already exists in the database %s and catalog %s",
+                        catalogTable, catalogDatabase, catalogName);
+            }
+        }
+
+        return TableLoader.fromCatalog(flinkCatalog.getCatalogLoader(),
+                TableIdentifier.of(catalogDatabase, catalogTable));
+    }
+
+    private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) {
+        Preconditions.checkNotNull(catalog, "Flink catalog cannot be null");
+        return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath));
+    }
+
+    @Override
+    public DynamicTableSource createDynamicTableSource(Context context) {
+        ObjectIdentifier objectIdentifier = context.getObjectIdentifier();
+        CatalogTable catalogTable = loadCatalogTable(context);
+        Map<String, String> tableProps = catalogTable.getOptions();
+        TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema());
+
+        TableLoader tableLoader;
+        if (catalog != null) {
+            tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath());
+        } else {
+            tableLoader = createTableLoader(catalogTable, tableProps, objectIdentifier.getDatabaseName(),
+                    objectIdentifier.getObjectName());
+        }
+
+        return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration());
+    }
+
+    @Override
+    public DynamicTableSink createDynamicTableSink(Context context) {
+        ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
+        CatalogTable catalogTable = loadCatalogTable(context);
+        Map<String, String> tableProps = catalogTable.getOptions();
+        TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema());
+
+        TableLoader tableLoader;
+        if (catalog != null) {
+            tableLoader = createTableLoader(catalog, objectPath);
+        } else {
+            tableLoader = createTableLoader(catalogTable, tableProps, objectPath.getDatabaseName(),
+                    objectPath.getObjectName());
+        }
+
+        return new IcebergTableSink(tableLoader, tableSchema, catalogTable);
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        Set<ConfigOption<?>> options = Sets.newHashSet();
+        options.add(CATALOG_TYPE);
+        options.add(CATALOG_NAME);
+        return options;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {

Review Comment:
   Please add 'sink.ignore.changelog' option to the optionalOptions().



##########
inlong-sort/sort-connectors/iceberg/src/main/java/org/apache/inlong/sort/iceberg/FlinkCatalogFactory.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.inlong.sort.iceberg;
+
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Strings;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+/**
+ * A Flink Catalog factory implementation that creates {@link FlinkCatalog}.
+ * <p>
+ * This supports the following catalog configuration options:
+ * <ul>
+ *   <li><code>type</code> - Flink catalog factory key, should be "iceberg"</li>
+ *   <li><code>catalog-type</code> - iceberg catalog type, "hive" or "hadoop"</li>
+ *   <li><code>uri</code> - the Hive Metastore URI (Hive catalog only)</li>
+ *   <li><code>clients</code> - the Hive Client Pool Size (Hive catalog only)</li>
+ *   <li><code>warehouse</code> - the warehouse path (Hadoop catalog only)</li>
+ *   <li><code>default-database</code> - a database name to use as the default</li>
+ *   <li><code>base-namespace</code> - a base namespace as the prefix for all databases (Hadoop catalog only)</li>
+ *   <li><code>cache-enabled</code> - whether to enable catalog cache</li>
+ * </ul>
+ * </p>
+ * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override
+ * {@link #createCatalogLoader(String, Map, Configuration)}.
+ */
+public class FlinkCatalogFactory implements CatalogFactory {

Review Comment:
   it can be replaced by the origin `FlinkCatalogFactory ` ?



##########
inlong-sort/sort-connectors/iceberg/src/main/java/org/apache/inlong/sort/iceberg/FlinkCatalog.java:
##########
@@ -0,0 +1,731 @@
+/*
+ * 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.inlong.sort.iceberg;
+
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+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.Factory;
+import org.apache.flink.util.StringUtils;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.UpdateProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.util.FlinkCompatibilityUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+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;
+
+/**
+ * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * The mapping between Flink database and Iceberg namespace:
+ * Supplying a base namespace for a given catalog, so if you have a catalog that supports a 2-level namespace, you
+ * would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
+ * </p>
+ * The Iceberg table manages its partitions by itself. The partition of the Iceberg table is independent of the
+ * partition of Flink.
+ */
+public class FlinkCatalog extends AbstractCatalog {

Review Comment:
   it can be replaced by the origin  `FlinkCatalog` ?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org