[ 
https://issues.apache.org/jira/browse/DRILL-1328?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15290064#comment-15290064
 ] 

ASF GitHub Bot commented on DRILL-1328:
---------------------------------------

Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/425#discussion_r63799682
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
 ---
    @@ -0,0 +1,161 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.drill.exec.planner.common;
    +
    +import java.util.Map;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.rel.RelNode;
    +import org.apache.calcite.rel.RelVisitor;
    +import org.apache.calcite.rel.core.TableScan;
    +import org.apache.drill.exec.client.DrillClient;
    +import org.apache.drill.exec.ops.QueryContext;
    +import org.apache.drill.exec.planner.logical.DrillTable;
    +import org.apache.drill.exec.proto.UserBitShared;
    +import org.apache.drill.exec.server.DrillbitContext;
    +import org.apache.drill.exec.server.rest.QueryWrapper.Listener;
    +
    +/**
    + * Wraps the stats table info including schema and tableName. Also 
materializes stats from storage and keeps them in
    + * memory.
    + */
    +public class DrillStatsTable {
    +  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(DrillStatsTable.class);
    +
    +  /**
    +   * List of columns in stats table.
    +   */
    +  public static final String COL_COLUMN = "column";
    +  public static final String COL_COMPUTED = "computed";
    +  public static final String COL_STATCOUNT = "statcount";
    +  public static final String COL_NDV = "ndv";
    +
    +  private final String schemaName;
    +  private final String tableName;
    +
    +  private final Map<String, Long> ndv = Maps.newHashMap();
    +  private double rowCount = -1;
    +
    +  private boolean materialized = false;
    +
    +  public DrillStatsTable(String schemaName, String tableName) {
    +    this.schemaName = schemaName;
    +    this.tableName = tableName;
    +  }
    +
    +  /**
    +   * Get number of distinct values of given column. If stats are not 
present for the given column, a null is returned.
    +   *
    +   * Note: returned data may not be accurate. Accuracy depends on whether 
the table data has changed after the
    +   * stats are computed.
    +   *
    +   * @param col
    +   * @return
    +   */
    +  public Double getNdv(String col) {
    +    Preconditions.checkState(materialized, "Stats are not yet 
materialized.");
    +
    +    final String upperCol = col.toUpperCase();
    +    final Long ndvCol = ndv.get(upperCol);
    +    if (ndvCol != null) {
    +      return Math.min(ndvCol, rowCount);
    +    }
    +
    +    return null;
    +  }
    +
    +  /**
    +   * Get row count of the table. Returns null if stats are not present.
    +   *
    +   * Note: returned data may not be accurate. Accuracy depends on whether 
the table data has changed after the
    +   * stats are computed.
    +   *
    +   * @return
    +   */
    +  public Double getRowCount() {
    +    Preconditions.checkState(materialized, "Stats are not yet 
materialized.");
    +    return rowCount > 0 ? rowCount : null;
    +  }
    +
    +  /**
    +   * Read the stats from storage and keep them in memory.
    +   * @param context
    +   * @throws Exception
    +   */
    +  public void materialize(final QueryContext context) throws Exception {
    +    if (materialized) {
    +      return;
    +    }
    +
    +    final String fullTableName = "`" + schemaName + "`.`" + tableName + 
"`";
    +    final String sql = "SELECT a.* FROM " + fullTableName + " AS a INNER 
JOIN " +
    +        "(SELECT `" + COL_COLUMN + "`, max(`" + COL_COMPUTED +"`) AS `" + 
COL_COMPUTED + "` " +
    +        "FROM " + fullTableName + " GROUP BY `" + COL_COLUMN + "`) AS b " +
    +        "ON a.`" + COL_COLUMN + "` = b.`" + COL_COLUMN +"` and a.`" + 
COL_COMPUTED + "` = b.`" + COL_COMPUTED + "`";
    +
    +    final DrillbitContext dc = context.getDrillbitContext();
    +    try(final DrillClient client = new DrillClient(dc.getConfig(), 
dc.getClusterCoordinator(), dc.getAllocator())) {
    --- End diff --
    
    Few questions here:
    (1) How does this work when authentication is enabled?
    (2) Why does this use a listener from the rest package?
    (3) Are there alternative way to run this query?


> Support table statistics
> ------------------------
>
>                 Key: DRILL-1328
>                 URL: https://issues.apache.org/jira/browse/DRILL-1328
>             Project: Apache Drill
>          Issue Type: Improvement
>            Reporter: Cliff Buchanan
>             Fix For: Future
>
>         Attachments: 0001-PRE-Set-value-count-in-splitAndTransfer.patch
>
>
> This consists of several subtasks
> * implement operators to generate statistics
> * add "analyze table" support to parser/planner
> * create a metadata provider to allow statistics to be used by optiq in 
> planning optimization
> * implement statistics functions
> Right now, the bulk of this functionality is implemented, but it hasn't been 
> rigorously tested and needs to have some definite answers for some of the 
> parts "around the edges" (how analyze table figures out where the table 
> statistics are located, how a table "append" should work in a read only file 
> system)
> Also, here are a few known caveats:
> * table statistics are collected by creating a sql query based on the string 
> path of the table. This should probably be done with a Table reference.
> * Case sensitivity for column statistics is probably iffy
> * Math for combining two column NDVs into a joint NDV should be checked.
> * Schema changes aren't really being considered yet.
> * adding getDrillTable is probably unnecessary; it might be better to do 
> getTable().unwrap(DrillTable.class)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to