Skip to content
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

[AMORO-2344] [Flink]: Support UnifiedCatalog to contain Iceberg format table in Flink Engine #2427

Merged
merged 6 commits into from
Dec 19, 2023
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
6 changes: 6 additions & 0 deletions mixed/flink/flink-common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,12 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-metrics-dropwizard</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
Comment on lines +146 to +151
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this dependency?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Iceberg connector's operator(IcebergStreamingWriter.writerMetrics) needs this dependency.


<!-- format dependencies -->
<dependency>
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import com.google.common.base.Objects;
import com.netease.arctic.NoSuchDatabaseException;
import com.netease.arctic.flink.InternalCatalogBuilder;
import com.netease.arctic.flink.catalog.factories.ArcticCatalogFactoryOptions;
import com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions;
import com.netease.arctic.flink.table.DynamicTableFactory;
import com.netease.arctic.flink.table.descriptors.ArcticValidator;
import com.netease.arctic.flink.util.ArcticUtils;
Expand Down Expand Up @@ -92,8 +92,8 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;

/** Catalogs for arctic data lake. */
public class ArcticCatalog extends AbstractCatalog {
/** Catalogs for mixed table format(include mixed-iceberg and mixed-hive). */
public class MixedCatalog extends AbstractCatalog {
public static final String DEFAULT_DB = "default";

/**
Expand All @@ -106,12 +106,12 @@ public class ArcticCatalog extends AbstractCatalog {

private com.netease.arctic.catalog.ArcticCatalog internalCatalog;

public ArcticCatalog(String name, String defaultDatabase, InternalCatalogBuilder catalogBuilder) {
public MixedCatalog(String name, String defaultDatabase, InternalCatalogBuilder catalogBuilder) {
super(name, defaultDatabase);
this.catalogBuilder = catalogBuilder;
}

public ArcticCatalog(ArcticCatalog copy) {
public MixedCatalog(MixedCatalog copy) {
this(copy.getName(), copy.getDefaultDatabase(), copy.catalogBuilder);
}

Expand Down Expand Up @@ -229,8 +229,7 @@ private void fillTableMetaPropertiesIfLookupLike(
properties.put(ArcticValidator.ARCTIC_CATALOG.key(), tableIdentifier.getCatalog());
properties.put(ArcticValidator.ARCTIC_TABLE.key(), tableIdentifier.getTableName());
properties.put(ArcticValidator.ARCTIC_DATABASE.key(), tableIdentifier.getDatabase());
properties.put(
ArcticCatalogFactoryOptions.METASTORE_URL.key(), catalogBuilder.getMetastoreUrl());
properties.put(CatalogFactoryOptions.METASTORE_URL.key(), catalogBuilder.getMetastoreUrl());
}

private static List<String> toPartitionKeys(PartitionSpec spec, Schema icebergSchema) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,21 +18,23 @@

package com.netease.arctic.flink.catalog.factories;

import static com.netease.arctic.flink.catalog.ArcticCatalog.DEFAULT_DB;
import static com.netease.arctic.ams.api.properties.CatalogMetaProperties.TABLE_FORMATS;
import static com.netease.arctic.flink.catalog.MixedCatalog.DEFAULT_DB;

import com.netease.arctic.ams.api.properties.CatalogMetaProperties;
import com.netease.arctic.flink.catalog.ArcticCatalog;
import com.netease.arctic.flink.catalog.FlinkUnifiedCatalog;
import com.netease.arctic.flink.catalog.MixedCatalog;
import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.table.catalog.CommonCatalogOptions;

/** {@link ConfigOption}s for {@link ArcticCatalog}. */
/** {@link ConfigOption}s for {@link MixedCatalog} and {@link FlinkUnifiedCatalog}. */
@Internal
public class ArcticCatalogFactoryOptions {
public class CatalogFactoryOptions {
public static final String MIXED_ICEBERG_IDENTIFIER = "mixed_iceberg";
public static final String MIXED_HIVE_IDENTIFIER = "mixed_hive";
@Deprecated public static final String IDENTIFIER = "arctic";
@Deprecated public static final String LEGACY_MIXED_IDENTIFIER = "arctic";
public static final String UNIFIED_IDENTIFIER = "unified";

public static final ConfigOption<String> DEFAULT_DATABASE =
Expand Down Expand Up @@ -81,4 +83,10 @@ public class ArcticCatalogFactoryOptions {
PROPERTIES_PREFIX + "." + CatalogMetaProperties.AUTH_CONFIGS_KEY_KEYTAB_ENCODE)
.stringType()
.noDefaultValue();

public static final ConfigOption<String> FLINK_TABLE_FORMATS =
ConfigOptions.key(TABLE_FORMATS)
.stringType()
.noDefaultValue()
.withDescription("This illustrates the table format contained in the catalog.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,38 +18,43 @@

package com.netease.arctic.flink.catalog.factories;

import static com.netease.arctic.flink.catalog.factories.ArcticCatalogFactoryOptions.DEFAULT_DATABASE;
import static com.netease.arctic.ams.api.Constants.THRIFT_TABLE_SERVICE_NAME;
import static com.netease.arctic.ams.api.properties.CatalogMetaProperties.TABLE_FORMATS;
import static com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions.DEFAULT_DATABASE;
import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION;

import com.netease.arctic.UnifiedCatalog;
import com.netease.arctic.UnifiedCatalogLoader;
import com.netease.arctic.ams.api.TableFormat;
import com.netease.arctic.ams.api.client.ArcticThriftUrl;
import com.netease.arctic.flink.catalog.FlinkUnifiedCatalog;
import com.netease.arctic.utils.CatalogUtil;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.table.catalog.AbstractCatalog;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.factories.CatalogFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;

/** Factory for {@link FlinkUnifiedCatalog}. */
public class FlinkCatalogFactory implements CatalogFactory {
public class FlinkUnifiedCatalogFactory implements CatalogFactory {

private static final Set<TableFormat> SUPPORTED_FORMATS =
Sets.newHashSet(TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE);
public static final Set<TableFormat> SUPPORTED_FORMATS =
Sets.newHashSet(TableFormat.MIXED_ICEBERG, TableFormat.MIXED_HIVE, TableFormat.ICEBERG);

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.UNIFIED_IDENTIFIER;
return CatalogFactoryOptions.UNIFIED_IDENTIFIER;
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
Set<ConfigOption<?>> requiredOptions = new HashSet<>();
requiredOptions.add(ArcticCatalogFactoryOptions.METASTORE_URL);
requiredOptions.add(CatalogFactoryOptions.METASTORE_URL);
return requiredOptions;
}

Expand All @@ -68,33 +73,34 @@ public Catalog createCatalog(Context context) {
helper.validate();

final String defaultDatabase = helper.getOptions().get(DEFAULT_DATABASE);
String metastoreUrl = helper.getOptions().get(ArcticCatalogFactoryOptions.METASTORE_URL);
String metastoreUrl = helper.getOptions().get(CatalogFactoryOptions.METASTORE_URL);

Map<TableFormat, AbstractCatalog> availableCatalogs = Maps.newHashMap();
SUPPORTED_FORMATS.forEach(
tableFormat -> {
if (!availableCatalogs.containsKey(tableFormat)) {
availableCatalogs.put(tableFormat, createCatalog(context, tableFormat));
}
});
String amoroCatalogName =
ArcticThriftUrl.parse(metastoreUrl, THRIFT_TABLE_SERVICE_NAME).catalogName();
UnifiedCatalog unifiedCatalog =
UnifiedCatalogLoader.loadUnifiedCatalog(metastoreUrl, amoroCatalogName, Maps.newHashMap());
Configuration hadoopConf = unifiedCatalog.authenticationContext().getConfiguration();

Set<TableFormat> tableFormats =
CatalogUtil.tableFormats(unifiedCatalog.metastoreType(), unifiedCatalog.properties());
validate(tableFormats);

return new FlinkUnifiedCatalog(
metastoreUrl, context.getName(), defaultDatabase, availableCatalogs);
metastoreUrl, defaultDatabase, unifiedCatalog, context, hadoopConf);
}

private AbstractCatalog createCatalog(Context context, TableFormat tableFormat) {
CatalogFactory catalogFactory;

switch (tableFormat) {
case MIXED_ICEBERG:
case MIXED_HIVE:
catalogFactory = new ArcticCatalogFactory();
break;
default:
throw new UnsupportedOperationException(
String.format("Unsupported table format: [%s] in the amoro catalog." + tableFormat));
private void validate(Set<TableFormat> expectedFormats) {
if (expectedFormats.isEmpty()) {
throw new IllegalArgumentException(
String.format(
"The table formats must be specified in the catalog properties: [%s]",
TABLE_FORMATS));
}
if (!SUPPORTED_FORMATS.containsAll(expectedFormats)) {
throw new IllegalArgumentException(
String.format(
"The table formats [%s] are not supported in the unified catalog, the supported table formats are [%s].",
expectedFormats, SUPPORTED_FORMATS));
}

return (AbstractCatalog) catalogFactory.createCatalog(context);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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 com.netease.arctic.flink.catalog.factories.iceberg;

import org.apache.flink.table.catalog.Catalog;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.flink.FlinkCatalogFactory;

import java.util.Map;

/** Creating Iceberg Catalog by the hadoop configuration which stored in the AMS. */
public class IcebergFlinkCatalogFactory extends FlinkCatalogFactory {
private final Configuration hadoopConf;

public IcebergFlinkCatalogFactory(Configuration hadoopConf) {
this.hadoopConf = hadoopConf;
}

@Override
public Catalog createCatalog(String name, Map<String, String> properties) {
return super.createCatalog(name, properties, hadoopConf);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,35 +16,39 @@
* limitations under the License.
*/

package com.netease.arctic.flink.catalog.factories;
package com.netease.arctic.flink.catalog.factories.mixed;

import static com.netease.arctic.flink.catalog.factories.ArcticCatalogFactoryOptions.DEFAULT_DATABASE;
import static com.netease.arctic.flink.catalog.factories.ArcticCatalogFactoryOptions.METASTORE_URL;
import static com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions.DEFAULT_DATABASE;
import static com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions.FLINK_TABLE_FORMATS;
import static com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions.METASTORE_URL;
import static com.netease.arctic.flink.table.KafkaConnectorOptionsUtil.getKafkaParams;
import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION;

import com.netease.arctic.flink.InternalCatalogBuilder;
import com.netease.arctic.flink.catalog.ArcticCatalog;
import com.netease.arctic.flink.catalog.MixedCatalog;
import com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.factories.CatalogFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional;
import java.util.Set;

/** Factory for {@link ArcticCatalog} */
public class ArcticCatalogFactory implements CatalogFactory {
/** Factory for {@link MixedCatalog} */
public class MixedCatalogFactory implements CatalogFactory {

private static final Logger LOG = LoggerFactory.getLogger(ArcticCatalogFactory.class);
private static final Logger LOG = LoggerFactory.getLogger(MixedCatalogFactory.class);

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.IDENTIFIER;
return CatalogFactoryOptions.LEGACY_MIXED_IDENTIFIER;
}

@Override
Expand All @@ -57,13 +61,16 @@ public Catalog createCatalog(Context context) {
final String defaultDatabase = helper.getOptions().get(DEFAULT_DATABASE);
String metastoreUrl = helper.getOptions().get(METASTORE_URL);
final Map<String, String> arcticCatalogProperties = getKafkaParams(context.getOptions());
final Map<String, String> catalogProperties = Maps.newHashMap(arcticCatalogProperties);

return new ArcticCatalog(
Optional<String> tableFormatsOptional = helper.getOptions().getOptional(FLINK_TABLE_FORMATS);
tableFormatsOptional.ifPresent(
tableFormats -> catalogProperties.put(FLINK_TABLE_FORMATS.key(), tableFormats));

return new MixedCatalog(
context.getName(),
defaultDatabase,
InternalCatalogBuilder.builder()
.metastoreUrl(metastoreUrl)
.properties(arcticCatalogProperties));
InternalCatalogBuilder.builder().metastoreUrl(metastoreUrl).properties(catalogProperties));
}

@Override
Expand All @@ -79,14 +86,16 @@ public Set<ConfigOption<?>> optionalOptions() {
options.add(DEFAULT_DATABASE);

// authorization config
options.add(ArcticCatalogFactoryOptions.AUTH_AMS_CONFIGS_DISABLE);
options.add(ArcticCatalogFactoryOptions.AUTH_METHOD);
options.add(ArcticCatalogFactoryOptions.SIMPLE_USER_NAME);
options.add(ArcticCatalogFactoryOptions.KEYTAB_LOGIN_USER);
options.add(ArcticCatalogFactoryOptions.KRB5_CONF_PATH);
options.add(ArcticCatalogFactoryOptions.KRB5_CONF_ENCODE);
options.add(ArcticCatalogFactoryOptions.KEYTAB_PATH);
options.add(ArcticCatalogFactoryOptions.KEYTAB_ENCODE);
options.add(CatalogFactoryOptions.AUTH_AMS_CONFIGS_DISABLE);
options.add(CatalogFactoryOptions.AUTH_METHOD);
options.add(CatalogFactoryOptions.SIMPLE_USER_NAME);
options.add(CatalogFactoryOptions.KEYTAB_LOGIN_USER);
options.add(CatalogFactoryOptions.KRB5_CONF_PATH);
options.add(CatalogFactoryOptions.KRB5_CONF_ENCODE);
options.add(CatalogFactoryOptions.KEYTAB_PATH);
options.add(CatalogFactoryOptions.KEYTAB_ENCODE);

options.add(CatalogFactoryOptions.FLINK_TABLE_FORMATS);
return options;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,19 @@
* limitations under the License.
*/

package com.netease.arctic.flink.catalog.factories;
package com.netease.arctic.flink.catalog.factories.mixed;

import com.netease.arctic.flink.catalog.ArcticCatalog;
import com.netease.arctic.flink.catalog.MixedCatalog;
import com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions;

/**
* The factory to create {@link ArcticCatalog} with {@link
* ArcticCatalogFactoryOptions#MIXED_HIVE_IDENTIFIER} identifier.
* The factory to create {@link MixedCatalog} with {@link
* CatalogFactoryOptions#MIXED_HIVE_IDENTIFIER} identifier.
*/
public class MixedHiveCatalogFactory extends ArcticCatalogFactory {
public class MixedHiveCatalogFactory extends MixedCatalogFactory {

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.MIXED_HIVE_IDENTIFIER;
return CatalogFactoryOptions.MIXED_HIVE_IDENTIFIER;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,19 @@
* limitations under the License.
*/

package com.netease.arctic.flink.catalog.factories;
package com.netease.arctic.flink.catalog.factories.mixed;

import com.netease.arctic.flink.catalog.ArcticCatalog;
import com.netease.arctic.flink.catalog.MixedCatalog;
import com.netease.arctic.flink.catalog.factories.CatalogFactoryOptions;

/**
* The factory to create {@link ArcticCatalog} with {@link
* ArcticCatalogFactoryOptions#MIXED_ICEBERG_IDENTIFIER} identifier.
* The factory to create {@link MixedCatalog} with {@link
* CatalogFactoryOptions#MIXED_ICEBERG_IDENTIFIER} identifier.
*/
public class MixedIcebergCatalogFactory extends ArcticCatalogFactory {
public class MixedIcebergCatalogFactory extends MixedCatalogFactory {

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.MIXED_ICEBERG_IDENTIFIER;
return CatalogFactoryOptions.MIXED_ICEBERG_IDENTIFIER;
}
}
Loading