Skip to content

Commit

Permalink
[FLINK-12677][hive][sql-client] Add descriptor, validator, and factor…
Browse files Browse the repository at this point in the history
…y for HiveCatalog

This PR adds descriptor, validator, and factory for HiveCatalog, and tests in flink-sql-client.

This closes apache#8589.
  • Loading branch information
bowenli86 committed Jun 7, 2019
1 parent b5711d6 commit 80d62e2
Show file tree
Hide file tree
Showing 16 changed files with 794 additions and 62 deletions.
64 changes: 48 additions & 16 deletions flink-connectors/flink-connector-hive/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -35,11 +35,6 @@ under the License.

<packaging>jar</packaging>

<properties>
<hive.version>2.3.4</hive.version>
<hivemetastore.hadoop.version>2.7.2</hivemetastore.hadoop.version>
</properties>

<dependencies>

<!-- core dependencies -->
Expand Down Expand Up @@ -73,13 +68,61 @@ under the License.
<artifactId>hadoop-common</artifactId>
<version>${hivemetastore.hadoop.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.curator</groupId>
<artifactId>curator-client</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.curator</groupId>
<artifactId>curator-recipes</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</exclusion>
<exclusion>
<groupId>commons-digester</groupId>
<artifactId>commons-digester</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-json</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-core</artifactId>
<version>${hivemetastore.hadoop.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-common</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-core-asl</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- Hive dependencies -->
Expand Down Expand Up @@ -108,10 +151,6 @@ under the License.
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
Expand Down Expand Up @@ -338,13 +377,6 @@ under the License.

<!-- test dependencies -->

<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<version>10.10.2.0</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,8 +79,6 @@

import javax.annotation.Nullable;

import java.io.File;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.ArrayList;
import java.util.HashMap;
Expand All @@ -95,8 +93,10 @@
* A catalog implementation for Hive.
*/
public class HiveCatalog extends AbstractCatalog {
// Default database of Hive metastore
public static final String DEFAULT_DB = "default";

private static final Logger LOG = LoggerFactory.getLogger(HiveCatalog.class);
private static final String DEFAULT_DB = "default";
private static final StorageFormatFactory storageFormatFactory = new StorageFormatFactory();
private static final String DEFAULT_HIVE_TABLE_STORAGE_FORMAT = "TextFile";

Expand All @@ -114,46 +114,22 @@ public class HiveCatalog extends AbstractCatalog {

private HiveMetastoreClientWrapper client;

public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveSiteFilePath) {
this(catalogName,
defaultDatabase == null ? DEFAULT_DB : defaultDatabase,
getHiveConf(loadHiveSiteUrl(hiveSiteFilePath)));
}

public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable URL hiveSiteUrl) {
this(catalogName,
defaultDatabase == null ? DEFAULT_DB : defaultDatabase,
getHiveConf(hiveSiteUrl));
createHiveConf(hiveSiteUrl));
}

public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable HiveConf hiveConf) {
@VisibleForTesting
protected HiveCatalog(String catalogName, String defaultDatabase, HiveConf hiveConf) {
super(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase);

this.hiveConf = hiveConf == null ? getHiveConf(null) : hiveConf;
this.hiveConf = hiveConf == null ? createHiveConf(null) : hiveConf;

LOG.info("Created HiveCatalog '{}'", catalogName);
}

private static URL loadHiveSiteUrl(String filePath) {

URL url = null;

if (!StringUtils.isNullOrWhitespaceOnly(filePath)) {
try {
url = new File(filePath).toURI().toURL();

LOG.info("Successfully loaded '{}'", filePath);

} catch (MalformedURLException e) {
throw new CatalogException(
String.format("Failed to get hive-site.xml from the given path '%s'", filePath), e);
}
}

return url;
}

private static HiveConf getHiveConf(URL hiveSiteUrl) {
private static HiveConf createHiveConf(URL hiveSiteUrl) {
LOG.info("Setting hive-site location as {}", hiveSiteUrl);

HiveConf.setHiveSiteLocation(hiveSiteUrl);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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:https://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.table.catalog.hive.descriptors;

import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.descriptors.CatalogDescriptor;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;

import java.util.Map;

import static org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator.CATALOG_HIVE_SITE_PATH;
import static org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator.CATALOG_TYPE_VALUE_HIVE;

/**
* Catalog descriptor for {@link HiveCatalog}.
*/
public class HiveCatalogDescriptor extends CatalogDescriptor {

private String hiveSitePath;

// TODO : set default database
public HiveCatalogDescriptor() {
super(CATALOG_TYPE_VALUE_HIVE, 1);
}

public HiveCatalogDescriptor hiveSitePath(String hiveSitePath) {
Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(hiveSitePath));
this.hiveSitePath = hiveSitePath;

return this;
}

@Override
protected Map<String, String> toCatalogProperties() {
final DescriptorProperties properties = new DescriptorProperties();

if (hiveSitePath != null) {
properties.putString(CATALOG_HIVE_SITE_PATH, hiveSitePath);
}

return properties.asMap();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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:https://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.table.catalog.hive.descriptors;

import org.apache.flink.table.descriptors.CatalogDescriptorValidator;
import org.apache.flink.table.descriptors.DescriptorProperties;

/**
* Validator for {@link HiveCatalogDescriptor}.
*/
public class HiveCatalogValidator extends CatalogDescriptorValidator {
public static final String CATALOG_TYPE_VALUE_HIVE = "hive";
public static final String CATALOG_HIVE_SITE_PATH = "hive-site-path";

@Override
public void validate(DescriptorProperties properties) {
super.validate(properties);
properties.validateValue(CATALOG_TYPE, CATALOG_TYPE_VALUE_HIVE, false);
properties.validateString(CATALOG_HIVE_SITE_PATH, true, 1);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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:https://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.table.catalog.hive.factories;

import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.hive.HiveCatalog;
import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.factories.CatalogFactory;
import org.apache.flink.util.StringUtils;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;

import static org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator.CATALOG_HIVE_SITE_PATH;
import static org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator.CATALOG_TYPE_VALUE_HIVE;
import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_DEFAULT_DATABASE;
import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_PROPERTY_VERSION;
import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_TYPE;

/**
* Catalog factory for {@link HiveCatalog}.
*/
public class HiveCatalogFactory implements CatalogFactory {
private static final Logger LOG = LoggerFactory.getLogger(HiveCatalogFactory.class);

@Override
public Map<String, String> requiredContext() {
Map<String, String> context = new HashMap<>();
context.put(CATALOG_TYPE, CATALOG_TYPE_VALUE_HIVE); // hive
context.put(CATALOG_PROPERTY_VERSION, "1"); // backwards compatibility
return context;
}

@Override
public List<String> supportedProperties() {
List<String> properties = new ArrayList<>();

// default database
properties.add(CATALOG_DEFAULT_DATABASE);

properties.add(CATALOG_HIVE_SITE_PATH);

return properties;
}

@Override
public Catalog createCatalog(String name, Map<String, String> properties) {
final DescriptorProperties descriptorProperties = getValidatedProperties(properties);

final String defaultDatabase =
descriptorProperties.getOptionalString(CATALOG_DEFAULT_DATABASE)
.orElse(HiveCatalog.DEFAULT_DB);

final Optional<String> hiveSitePath = descriptorProperties.getOptionalString(CATALOG_HIVE_SITE_PATH);

return new HiveCatalog(name, defaultDatabase, loadHiveSiteUrl(hiveSitePath.orElse(null)));
}

private static URL loadHiveSiteUrl(String filePath) {

URL url = null;

if (!StringUtils.isNullOrWhitespaceOnly(filePath)) {
try {
url = new File(filePath).toURI().toURL();

LOG.info("Successfully loaded '{}'", filePath);

} catch (MalformedURLException e) {
throw new CatalogException(
String.format("Failed to get hive-site.xml from the given path '%s'", filePath), e);
}
}

return url;
}

private static DescriptorProperties getValidatedProperties(Map<String, String> properties) {
final DescriptorProperties descriptorProperties = new DescriptorProperties(true);
descriptorProperties.putProperties(properties);

new HiveCatalogValidator().validate(descriptorProperties);

return descriptorProperties;
}
}
Loading

0 comments on commit 80d62e2

Please sign in to comment.