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

[FLINK-13400]Remove Hive and Hadoop dependencies from SQL Client #16532

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
486 changes: 486 additions & 0 deletions flink-end-to-end-tests/flink-end-to-end-tests-hive/pom.xml

Large diffs are not rendered by default.

@@ -0,0 +1,178 @@
/*
* 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.flink.tests.util.hive;

import org.apache.flink.client.cli.DefaultCLI;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.client.config.Environment;
import org.apache.flink.table.client.gateway.context.DefaultContext;
import org.apache.flink.table.client.gateway.context.ExecutionContext;
import org.apache.flink.table.client.gateway.context.SessionContext;
import org.apache.flink.testutils.junit.FailsOnJava11;
import org.apache.flink.util.FileUtils;
import org.apache.flink.util.StringUtils;
import org.apache.flink.util.TestLogger;

import org.junit.Test;
import org.junit.experimental.categories.Category;

import java.io.File;
import java.io.IOException;
import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;

/** End-to-end test for the hive connectors. */
public class SQLClientHiveITCase extends TestLogger {

public static final String CATALOGS_ENVIRONMENT_FILE = "test-sql-client-catalogs.yaml";

private SessionContext sessionContext;

@Test
@Category(FailsOnJava11.class)
public void testCatalogs() throws Exception {
final String inmemoryCatalog = "inmemorycatalog";
final String hiveCatalog = "hivecatalog";
final String hiveDefaultVersionCatalog = "hivedefaultversion";

final ExecutionContext context = createCatalogExecutionContext();
final TableEnvironment tableEnv = context.getTableEnvironment();

assertEquals(inmemoryCatalog, tableEnv.getCurrentCatalog());
assertEquals("mydatabase", tableEnv.getCurrentDatabase());

Catalog catalog = tableEnv.getCatalog(hiveCatalog).orElse(null);
assertNotNull(catalog);
assertTrue(catalog instanceof HiveCatalog);
assertEquals("2.3.4", ((HiveCatalog) catalog).getHiveVersion());

catalog = tableEnv.getCatalog(hiveDefaultVersionCatalog).orElse(null);
assertNotNull(catalog);
assertTrue(catalog instanceof HiveCatalog);
// make sure we have assigned a default hive version
assertFalse(StringUtils.isNullOrWhitespaceOnly(((HiveCatalog) catalog).getHiveVersion()));

tableEnv.useCatalog(hiveCatalog);

assertEquals(hiveCatalog, tableEnv.getCurrentCatalog());

Set<String> allCatalogs = new HashSet<>(Arrays.asList(tableEnv.listCatalogs()));
assertEquals(6, allCatalogs.size());
assertEquals(
new HashSet<>(
Arrays.asList(
"default_catalog",
inmemoryCatalog,
hiveCatalog,
hiveDefaultVersionCatalog,
"catalog1",
"catalog2")),
allCatalogs);

sessionContext.close();
}

@Test
@Category(FailsOnJava11.class)
public void testDatabases() throws Exception {
final String hiveCatalog = "hivecatalog";

final ExecutionContext context = createCatalogExecutionContext();
final TableEnvironment tableEnv = context.getTableEnvironment();

assertEquals(1, tableEnv.listDatabases().length);
assertEquals("mydatabase", tableEnv.listDatabases()[0]);

tableEnv.useCatalog(hiveCatalog);

assertEquals(2, tableEnv.listDatabases().length);
assertEquals(
new HashSet<>(
Arrays.asList(
HiveCatalog.DEFAULT_DB,
TestHiveCatalogFactory.ADDITIONAL_TEST_DATABASE)),
new HashSet<>(Arrays.asList(tableEnv.listDatabases())));

tableEnv.useCatalog(hiveCatalog);

assertEquals(HiveCatalog.DEFAULT_DB, tableEnv.getCurrentDatabase());

tableEnv.useDatabase(TestHiveCatalogFactory.ADDITIONAL_TEST_DATABASE);

assertEquals(
TestHiveCatalogFactory.ADDITIONAL_TEST_DATABASE, tableEnv.getCurrentDatabase());

sessionContext.close();
}

private ExecutionContext createCatalogExecutionContext() throws Exception {
final Map<String, String> replaceVars = new HashMap<>();
replaceVars.put("$VAR_EXECUTION_TYPE", "streaming");
replaceVars.put("$VAR_RESULT_MODE", "changelog");
replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append");
replaceVars.put("$VAR_MAX_ROWS", "100");
return createExecutionContext(CATALOGS_ENVIRONMENT_FILE, replaceVars);
}

private ExecutionContext createExecutionContext(String file, Map<String, String> replaceVars)
throws Exception {
final Environment env = parseModified(file, replaceVars);
return createExecutionContext(env);
}

private Environment parseModified(String fileName, Map<String, String> replaceVars)
throws IOException {
final URL url = this.getClass().getResource(fileName);
Objects.requireNonNull(url);
String schema = FileUtils.readFileUtf8(new File(url.getFile()));

for (Map.Entry<String, String> replaceVar : replaceVars.entrySet()) {
schema = schema.replace(replaceVar.getKey(), replaceVar.getValue());
}

return Environment.parse(schema);
}

private ExecutionContext createExecutionContext(Environment env) throws Exception {
final Configuration configuration = new Configuration();
DefaultContext defaultContext =
new DefaultContext(
env,
new ArrayList<>(),
configuration,
Collections.singletonList(new DefaultCLI()));
sessionContext = SessionContext.create(defaultContext, "test-session");
return sessionContext.getExecutionContext();
}
}
@@ -0,0 +1,124 @@
/*
* 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.flink.tests.util.hive;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.CatalogDatabaseImpl;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.ResolvedCatalogTable;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.catalog.hive.HiveTestUtils;
import org.apache.flink.table.catalog.hive.factories.HiveCatalogFactory;
import org.apache.flink.table.catalog.hive.factories.HiveCatalogFactoryOptions;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.types.DataType;

import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;

/**
* A test factory that is the same as {@link HiveCatalogFactory} except returning a {@link
* HiveCatalog} always with an embedded Hive metastore to test logic of {@link HiveCatalogFactory}.
*/
public class TestHiveCatalogFactory extends HiveCatalogFactory {
public static final String ADDITIONAL_TEST_DATABASE = "additional_test_database";
public static final String TEST_TABLE = "test_table";
static final String TABLE_WITH_PARAMETERIZED_TYPES = "param_types_table";

@Override
public String factoryIdentifier() {
return "hive-test";
}

@Override
public Catalog createCatalog(Context context) {
final Configuration configuration = Configuration.fromMap(context.getOptions());

// Developers may already have their own production/testing hive-site.xml set in their
// environment,
// and Flink tests should avoid using those hive-site.xml.
// Thus, explicitly create a testing HiveConf for unit tests here
Catalog hiveCatalog =
HiveTestUtils.createHiveCatalog(
context.getName(),
configuration.getString(HiveCatalogFactoryOptions.HIVE_VERSION));

// Creates an additional database to test tableEnv.useDatabase() will switch current
// database of the catalog
hiveCatalog.open();
try {
hiveCatalog.createDatabase(
ADDITIONAL_TEST_DATABASE,
new CatalogDatabaseImpl(new HashMap<>(), null),
false);
hiveCatalog.createTable(
new ObjectPath(ADDITIONAL_TEST_DATABASE, TEST_TABLE),
createResolvedTable(new String[] {"testcol"}, new DataType[] {DataTypes.INT()}),
false);
// create a table to test parameterized types
hiveCatalog.createTable(
new ObjectPath("default", TABLE_WITH_PARAMETERIZED_TYPES),
createResolvedTable(
new String[] {"dec", "ch", "vch"},
new DataType[] {
DataTypes.DECIMAL(10, 10), DataTypes.CHAR(5), DataTypes.VARCHAR(15)
}),
false);
} catch (DatabaseAlreadyExistException
| TableAlreadyExistException
| DatabaseNotExistException e) {
throw new CatalogException(e);
}

return hiveCatalog;
}

private ResolvedCatalogTable createResolvedTable(
String[] fieldNames, DataType[] fieldDataTypes) {
final Map<String, String> options = new HashMap<>();
options.put(FactoryUtil.CONNECTOR.key(), SqlCreateHiveTable.IDENTIFIER);
final CatalogTable origin =
CatalogTable.of(
Schema.newBuilder().fromFields(fieldNames, fieldDataTypes).build(),
null,
Collections.emptyList(),
options);
final List<Column> resolvedColumns =
IntStream.range(0, fieldNames.length)
.mapToObj(i -> Column.physical(fieldNames[i], fieldDataTypes[i]))
.collect(Collectors.toList());
return new ResolvedCatalogTable(
origin, new ResolvedSchema(resolvedColumns, Collections.emptyList(), null));
}
}
@@ -0,0 +1,16 @@
# 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.

org.apache.flink.tests.util.hive.TestHiveCatalogFactory
@@ -0,0 +1 @@

1 change: 1 addition & 0 deletions flink-end-to-end-tests/pom.xml
Expand Up @@ -88,6 +88,7 @@ under the License.
<module>flink-python-test</module>
<module>flink-end-to-end-tests-hbase</module>
<module>flink-glue-schema-registry-test</module>
<module>flink-end-to-end-tests-hive</module>
</modules>

<dependencyManagement>
Expand Down