Skip to content

Commit

Permalink
ANALYZE statement: Implement ANALYZE in Hive connector
Browse files Browse the repository at this point in the history
  • Loading branch information
jessesleeping committed Feb 1, 2019
1 parent 70add48 commit b7079d3
Show file tree
Hide file tree
Showing 15 changed files with 1,268 additions and 35 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* Licensed 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.facebook.presto.hive;

import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.session.PropertyMetadata;
import com.facebook.presto.spi.type.TypeManager;
import com.google.common.collect.ImmutableList;

import javax.inject.Inject;

import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Optional;

import static com.facebook.presto.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableSet.toImmutableSet;

public class HiveAnalyzeProperties
{
public static final String PARTITIONS_PROPERTY = "partitions";

private final List<PropertyMetadata<?>> analyzeProperties;

@Inject
public HiveAnalyzeProperties(TypeManager typeManager)
{
analyzeProperties = ImmutableList.of(
new PropertyMetadata<>(
PARTITIONS_PROPERTY,
"Partitions to be analyzed",
typeManager.getType(parseTypeSignature("array(array(varchar))")),
List.class,
null,
false,
HiveAnalyzeProperties::decodePartitionLists,
value -> value));
}

public List<PropertyMetadata<?>> getAnalyzeProperties()
{
return analyzeProperties;
}

@SuppressWarnings("unchecked")
public static Optional<List<List<String>>> getPartitionList(Map<String, Object> properties)
{
List<List<String>> partitions = (List<List<String>>) properties.get(PARTITIONS_PROPERTY);
return partitions == null ? Optional.empty() : Optional.of(partitions);
}

private static List<List<String>> decodePartitionLists(Object object)
{
if (object == null) {
return null;
}

// replace null partition value with hive default partition
return ImmutableList.copyOf(((Collection<?>) object).stream()
.peek(HiveAnalyzeProperties::throwIfNull)
.map(partition -> ((Collection<?>) partition).stream()
.map(name -> firstNonNull((String) name, HIVE_DEFAULT_DYNAMIC_PARTITION))
.collect(toImmutableList()))
.collect(toImmutableSet()));
}

private static void throwIfNull(Object object)
{
if (object == null) {
throw new PrestoException(INVALID_ANALYZE_PROPERTY, "Invalid null value in analyze partitions property");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ public void configure(Binder binder)

binder.bind(HiveSessionProperties.class).in(Scopes.SINGLETON);
binder.bind(HiveTableProperties.class).in(Scopes.SINGLETON);
binder.bind(HiveAnalyzeProperties.class).in(Scopes.SINGLETON);

binder.bind(NamenodeStats.class).in(Scopes.SINGLETON);
newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class, connectorId));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ public class HiveConnector
private final List<PropertyMetadata<?>> sessionProperties;
private final List<PropertyMetadata<?>> schemaProperties;
private final List<PropertyMetadata<?>> tableProperties;
private final List<PropertyMetadata<?>> analyzeProperties;

private final ConnectorAccessControl accessControl;
private final ClassLoader classLoader;

Expand All @@ -75,6 +77,7 @@ public HiveConnector(
List<PropertyMetadata<?>> sessionProperties,
List<PropertyMetadata<?>> schemaProperties,
List<PropertyMetadata<?>> tableProperties,
List<PropertyMetadata<?>> analyzeProperties,
ConnectorAccessControl accessControl,
ClassLoader classLoader)
{
Expand All @@ -90,6 +93,7 @@ public HiveConnector(
this.sessionProperties = ImmutableList.copyOf(requireNonNull(sessionProperties, "sessionProperties is null"));
this.schemaProperties = ImmutableList.copyOf(requireNonNull(schemaProperties, "schemaProperties is null"));
this.tableProperties = ImmutableList.copyOf(requireNonNull(tableProperties, "tableProperties is null"));
this.analyzeProperties = ImmutableList.copyOf(requireNonNull(analyzeProperties, "analyzeProperties is null"));
this.accessControl = requireNonNull(accessControl, "accessControl is null");
this.classLoader = requireNonNull(classLoader, "classLoader is null");
}
Expand Down Expand Up @@ -150,6 +154,12 @@ public List<PropertyMetadata<?>> getSchemaProperties()
return schemaProperties;
}

@Override
public List<PropertyMetadata<?>> getAnalyzeProperties()
{
return analyzeProperties;
}

@Override
public List<PropertyMetadata<?>> getTableProperties()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,7 @@ public Connector create(String catalogName, Map<String, String> config, Connecto
ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class);
HiveSessionProperties hiveSessionProperties = injector.getInstance(HiveSessionProperties.class);
HiveTableProperties hiveTableProperties = injector.getInstance(HiveTableProperties.class);
HiveAnalyzeProperties hiveAnalyzeProperties = injector.getInstance(HiveAnalyzeProperties.class);
ConnectorAccessControl accessControl = new PartitionsAwareAccessControl(injector.getInstance(ConnectorAccessControl.class));
Set<Procedure> procedures = injector.getInstance(Key.get(new TypeLiteral<Set<Procedure>>() {}));

Expand All @@ -144,6 +145,7 @@ public Connector create(String catalogName, Map<String, String> config, Connecto
hiveSessionProperties.getSessionProperties(),
HiveSchemaProperties.SCHEMA_PROPERTIES,
hiveTableProperties.getTableProperties(),
hiveAnalyzeProperties.getAnalyzeProperties(),
accessControl,
classLoader);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ public enum HiveErrorCode
// HIVE_TOO_MANY_BUCKET_SORT_FILES(36) is deprecated
HIVE_CORRUPTED_COLUMN_STATISTICS(37, EXTERNAL),
HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT(38, USER_ERROR),
HIVE_UNKNOWN_COLUMN_STATISTIC_TYPE(39, INTERNAL_ERROR),
/**/;

private final ErrorCode errorCode;
Expand Down
Loading

0 comments on commit b7079d3

Please sign in to comment.