Skip to content

Commit

Permalink
ANALYZE statement: Analyzer, planner and execution
Browse files Browse the repository at this point in the history
Extracted-From: prestodb/presto#11376
  • Loading branch information
jessesleeping authored and sopel39 committed Jan 29, 2019
1 parent 82574c8 commit df3f4df
Show file tree
Hide file tree
Showing 47 changed files with 1,161 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,7 @@ private synchronized void addConnectorInternal(MaterializedConnector connector)
metadataManager.getTablePropertyManager().addProperties(connectorId, connector.getTableProperties());
metadataManager.getColumnPropertyManager().addProperties(connectorId, connector.getColumnProperties());
metadataManager.getSchemaPropertyManager().addProperties(connectorId, connector.getSchemaProperties());
metadataManager.getAnalyzePropertyManager().addProperties(connectorId, connector.getAnalyzeProperties());
metadataManager.getSessionPropertyManager().addConnectorSessionProperties(connectorId, connector.getSessionProperties());
}

Expand Down Expand Up @@ -337,6 +338,7 @@ private static class MaterializedConnector
private final List<PropertyMetadata<?>> tableProperties;
private final List<PropertyMetadata<?>> schemaProperties;
private final List<PropertyMetadata<?>> columnProperties;
private final List<PropertyMetadata<?>> analyzeProperties;

public MaterializedConnector(ConnectorId connectorId, Connector connector)
{
Expand Down Expand Up @@ -425,6 +427,10 @@ public MaterializedConnector(ConnectorId connectorId, Connector connector)
List<PropertyMetadata<?>> columnProperties = connector.getColumnProperties();
requireNonNull(columnProperties, "Connector %s returned a null column properties set");
this.columnProperties = ImmutableList.copyOf(columnProperties);

List<PropertyMetadata<?>> analyzeProperties = connector.getAnalyzeProperties();
requireNonNull(analyzeProperties, "Connector %s returned a null analyze properties set");
this.analyzeProperties = ImmutableList.copyOf(analyzeProperties);
}

public ConnectorId getConnectorId()
Expand Down Expand Up @@ -496,5 +502,10 @@ public List<PropertyMetadata<?>> getSchemaProperties()
{
return schemaProperties;
}

public List<PropertyMetadata<?>> getAnalyzeProperties()
{
return analyzeProperties;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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 io.prestosql.metadata;

import io.prestosql.spi.statistics.TableStatisticsMetadata;

import static java.util.Objects.requireNonNull;

public class AnalyzeMetadata
{
private final TableStatisticsMetadata statisticsMetadata;
private final TableHandle tableHandle;

public AnalyzeMetadata(TableStatisticsMetadata statisticsMetadata, TableHandle tableHandle)
{
this.statisticsMetadata = requireNonNull(statisticsMetadata, "statisticsMetadata is null");
this.tableHandle = requireNonNull(tableHandle, "tableHandle is null");
}

public TableStatisticsMetadata getStatisticsMetadata()
{
return statisticsMetadata;
}

public TableHandle getTableHandle()
{
return tableHandle;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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 io.prestosql.metadata;

import static io.prestosql.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY;

public class AnalyzePropertyManager
extends AbstractPropertyManager
{
public AnalyzePropertyManager()
{
super("analyze", INVALID_ANALYZE_PROPERTY);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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 io.prestosql.metadata;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.prestosql.connector.ConnectorId;
import io.prestosql.spi.connector.ConnectorTableHandle;
import io.prestosql.spi.connector.ConnectorTransactionHandle;

import java.util.Objects;

import static java.util.Objects.requireNonNull;

public class AnalyzeTableHandle
{
private final ConnectorId connectorId;
private final ConnectorTransactionHandle transactionHandle;
private final ConnectorTableHandle connectorHandle;

@JsonCreator
public AnalyzeTableHandle(
@JsonProperty("connectorId") ConnectorId connectorId,
@JsonProperty("transactionHandle") ConnectorTransactionHandle transactionHandle,
@JsonProperty("connectorHandle") ConnectorTableHandle connectorHandle)
{
this.connectorId = requireNonNull(connectorId, "connectorId is null");
this.transactionHandle = requireNonNull(transactionHandle, "transactionHandle is null");
this.connectorHandle = requireNonNull(connectorHandle, "connectorHandle is null");
}

@JsonProperty
public ConnectorId getConnectorId()
{
return connectorId;
}

@JsonProperty
public ConnectorTableHandle getConnectorHandle()
{
return connectorHandle;
}

@JsonProperty
public ConnectorTransactionHandle getTransactionHandle()
{
return transactionHandle;
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
AnalyzeTableHandle that = (AnalyzeTableHandle) o;
return Objects.equals(connectorId, that.connectorId) &&
Objects.equals(transactionHandle, that.transactionHandle) &&
Objects.equals(connectorHandle, that.connectorHandle);
}

@Override
public int hashCode()
{
return Objects.hash(connectorId, transactionHandle, connectorHandle);
}

@Override
public String toString()
{
return connectorId + ":" + connectorHandle + ":" + transactionHandle;
}
}
19 changes: 19 additions & 0 deletions presto-main/src/main/java/io/prestosql/metadata/Metadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ public interface Metadata

Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName);

Optional<TableHandle> getTableHandleForStatisticsCollection(Session session, QualifiedObjectName tableName, Map<String, Object> analyzeProperties);

List<TableLayoutResult> getLayouts(Session session, TableHandle tableHandle, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns);

TableLayout getLayout(Session session, TableLayoutHandle handle);
Expand Down Expand Up @@ -193,8 +195,23 @@ public interface Metadata
/**
* Describes statistics that must be collected during a write.
*/
TableStatisticsMetadata getStatisticsCollectionMetadataForWrite(Session session, String catalogName, ConnectorTableMetadata tableMetadata);

/**
* Describe statistics that must be collected during a statistics collection
*/
TableStatisticsMetadata getStatisticsCollectionMetadata(Session session, String catalogName, ConnectorTableMetadata tableMetadata);

/**
* Begin statistics collection
*/
AnalyzeTableHandle beginStatisticsCollection(Session session, TableHandle tableHandle);

/**
* Finish statistics collection
*/
void finishStatisticsCollection(Session session, AnalyzeTableHandle tableHandle, Collection<ComputedStatistics> computedStatistics);

/**
* Start a SELECT/UPDATE/INSERT/DELETE query
*/
Expand Down Expand Up @@ -361,4 +378,6 @@ public interface Metadata
TablePropertyManager getTablePropertyManager();

ColumnPropertyManager getColumnPropertyManager();

AnalyzePropertyManager getAnalyzePropertyManager();
}
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ public class MetadataManager
private final SchemaPropertyManager schemaPropertyManager;
private final TablePropertyManager tablePropertyManager;
private final ColumnPropertyManager columnPropertyManager;
private final AnalyzePropertyManager analyzePropertyManager;
private final TransactionManager transactionManager;

private final ConcurrentMap<String, Collection<ConnectorMetadata>> catalogsByQueryId = new ConcurrentHashMap<>();
Expand All @@ -134,6 +135,7 @@ public MetadataManager(FeaturesConfig featuresConfig,
SchemaPropertyManager schemaPropertyManager,
TablePropertyManager tablePropertyManager,
ColumnPropertyManager columnPropertyManager,
AnalyzePropertyManager analyzePropertyManager,
TransactionManager transactionManager)
{
this(featuresConfig,
Expand All @@ -144,6 +146,7 @@ public MetadataManager(FeaturesConfig featuresConfig,
schemaPropertyManager,
tablePropertyManager,
columnPropertyManager,
analyzePropertyManager,
transactionManager);
}

Expand All @@ -156,6 +159,7 @@ public MetadataManager(FeaturesConfig featuresConfig,
SchemaPropertyManager schemaPropertyManager,
TablePropertyManager tablePropertyManager,
ColumnPropertyManager columnPropertyManager,
AnalyzePropertyManager analyzePropertyManager,
TransactionManager transactionManager)
{
functions = new FunctionRegistry(typeManager, blockEncodingSerde, featuresConfig);
Expand All @@ -167,6 +171,7 @@ public MetadataManager(FeaturesConfig featuresConfig,
this.schemaPropertyManager = requireNonNull(schemaPropertyManager, "schemaPropertyManager is null");
this.tablePropertyManager = requireNonNull(tablePropertyManager, "tablePropertyManager is null");
this.columnPropertyManager = requireNonNull(columnPropertyManager, "columnPropertyManager is null");
this.analyzePropertyManager = requireNonNull(analyzePropertyManager, "analyzePropertyManager is null");
this.transactionManager = requireNonNull(transactionManager, "transactionManager is null");

verifyComparableOrderableContract();
Expand Down Expand Up @@ -203,6 +208,7 @@ public static MetadataManager createTestMetadataManager(TransactionManager trans
new SchemaPropertyManager(),
new TablePropertyManager(),
new ColumnPropertyManager(),
new AnalyzePropertyManager(),
transactionManager);
}

Expand Down Expand Up @@ -328,6 +334,25 @@ public Optional<TableHandle> getTableHandle(Session session, QualifiedObjectName
return Optional.empty();
}

@Override
public Optional<TableHandle> getTableHandleForStatisticsCollection(Session session, QualifiedObjectName table, Map<String, Object> analyzeProperties)
{
requireNonNull(table, "table is null");

Optional<CatalogMetadata> catalog = getOptionalCatalogMetadata(session, table.getCatalogName());
if (catalog.isPresent()) {
CatalogMetadata catalogMetadata = catalog.get();
ConnectorId connectorId = catalogMetadata.getConnectorId(session, table);
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(connectorId);

ConnectorTableHandle tableHandle = metadata.getTableHandleForStatisticsCollection(session.toConnectorSession(connectorId), table.asSchemaTableName(), analyzeProperties);
if (tableHandle != null) {
return Optional.of(new TableHandle(connectorId, tableHandle));
}
}
return Optional.empty();
}

@Override
public Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName)
{
Expand Down Expand Up @@ -621,6 +646,15 @@ public Optional<NewTableLayout> getInsertLayout(Session session, TableHandle tab
.map(layout -> new NewTableLayout(connectorId, catalogMetadata.getTransactionHandleFor(connectorId), layout));
}

@Override
public TableStatisticsMetadata getStatisticsCollectionMetadataForWrite(Session session, String catalogName, ConnectorTableMetadata tableMetadata)
{
CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, catalogName);
ConnectorMetadata metadata = catalogMetadata.getMetadata();
ConnectorId connectorId = catalogMetadata.getConnectorId();
return metadata.getStatisticsCollectionMetadataForWrite(session.toConnectorSession(connectorId), tableMetadata);
}

@Override
public TableStatisticsMetadata getStatisticsCollectionMetadata(Session session, String catalogName, ConnectorTableMetadata tableMetadata)
{
Expand All @@ -630,6 +664,26 @@ public TableStatisticsMetadata getStatisticsCollectionMetadata(Session session,
return metadata.getStatisticsCollectionMetadata(session.toConnectorSession(connectorId), tableMetadata);
}

@Override
public AnalyzeTableHandle beginStatisticsCollection(Session session, TableHandle tableHandle)
{
ConnectorId connectorId = tableHandle.getConnectorId();
CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, connectorId);
ConnectorMetadata metadata = catalogMetadata.getMetadata();

ConnectorTransactionHandle transactionHandle = catalogMetadata.getTransactionHandleFor(connectorId);
ConnectorTableHandle connectorTableHandle = metadata.beginStatisticsCollection(session.toConnectorSession(connectorId), tableHandle.getConnectorHandle());
return new AnalyzeTableHandle(connectorId, transactionHandle, connectorTableHandle);
}

@Override
public void finishStatisticsCollection(Session session, AnalyzeTableHandle tableHandle, Collection<ComputedStatistics> computedStatistics)
{
ConnectorId connectorId = tableHandle.getConnectorId();
CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, connectorId);
catalogMetadata.getMetadata().finishStatisticsCollection(session.toConnectorSession(), tableHandle.getConnectorHandle(), computedStatistics);
}

@Override
public Optional<NewTableLayout> getNewTableLayout(Session session, String catalogName, ConnectorTableMetadata tableMetadata)
{
Expand Down Expand Up @@ -1064,6 +1118,11 @@ public ColumnPropertyManager getColumnPropertyManager()
return columnPropertyManager;
}

public AnalyzePropertyManager getAnalyzePropertyManager()
{
return analyzePropertyManager;
}

private ViewDefinition deserializeView(String data)
{
try {
Expand Down
Loading

0 comments on commit df3f4df

Please sign in to comment.