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-2423] [Flink]: Using 'mixed_iceberg' and 'mixed_hive' indentifier to CREATE CATALOG and deprecate 'arctic' identifier #2424

Merged
merged 4 commits into from
Dec 14, 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
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,9 @@
/** {@link ConfigOption}s for {@link ArcticCatalog}. */
@Internal
public class ArcticCatalogFactoryOptions {
public static final String IDENTIFIER = "arctic";
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";
public static final String UNIFIED_IDENTIFIER = "unified";

public static final ConfigOption<String> DEFAULT_DATABASE =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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;

import com.netease.arctic.flink.catalog.ArcticCatalog;

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

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.MIXED_HIVE_IDENTIFIER;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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;

import com.netease.arctic.flink.catalog.ArcticCatalog;

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

@Override
public String factoryIdentifier() {
return ArcticCatalogFactoryOptions.MIXED_ICEBERG_IDENTIFIER;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
# limitations under the License.
#

com.netease.arctic.flink.catalog.factories.MixedIcebergCatalogFactory
com.netease.arctic.flink.catalog.factories.MixedHiveCatalogFactory
com.netease.arctic.flink.catalog.factories.ArcticCatalogFactory
com.netease.arctic.flink.catalog.factories.FlinkCatalogFactory
com.netease.arctic.flink.table.DynamicTableFactory
Original file line number Diff line number Diff line change
Expand Up @@ -98,13 +98,30 @@ public void before() throws Exception {

@After
public void after() {
sql("DROP TABLE " + CATALOG + "." + DB + "." + TABLE);
sql("DROP DATABASE " + CATALOG + "." + DB);
sql("DROP TABLE IF EXISTS " + CATALOG + "." + DB + "." + TABLE);
sql("DROP DATABASE IF EXISTS " + CATALOG + "." + DB);
Assert.assertTrue(CollectionUtil.isNullOrEmpty(getMixedFormatCatalog().listDatabases()));
sql("USE CATALOG default_catalog");
sql("DROP CATALOG " + CATALOG);
}

@Test
public void testCreateIcebergHiveCatalog() {
sql(
"CREATE CATALOG mixed_iceberg_catalog WITH ('type'='mixed_iceberg', 'metastore.url'='%s')",
getCatalogUrl());
sql(
"CREATE CATALOG mixed_hive_catalog WITH ('type'='mixed_hive', 'metastore.url'='%s')",
getCatalogUrl());

String[] catalogs = getTableEnv().listCatalogs();
Assert.assertArrayEquals(
Arrays.stream(catalogs).sorted().toArray(),
Stream.of("default_catalog", "arcticCatalog", "mixed_iceberg_catalog", "mixed_hive_catalog")
.sorted()
.toArray());
}

@Test
public void testDDL() throws IOException {
sql(
Expand Down