aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorGautam Parai <gparai@maprtech.com>2014-08-21 14:59:53 -0700
committerGautam Parai <gparai@maprtech.com>2019-02-28 12:01:24 -0800
commit469be17597e7b7c6bc1de9863dcb6c5604a55f0c (patch)
tree76a1c2572cfb19a75a0f82e6d165db333797fe3b
parent3233d8aaff57ac71bd3b726efcd5fdaa92aef861 (diff)
DRILL-1328: Support table statistics - Part 2
Add support for avg row-width and major type statistics. Parallelize the ANALYZE implementation and stats UDF implementation to improve stats collection performance. Update/fix rowcount, selectivity and ndv computations to improve plan costing. Add options for configuring collection/usage of statistics. Add new APIs and implementation for stats writer (as a precursor to Drill Metastore APIs). Fix several stats/costing related issues identified while running TPC-H nad TPC-DS queries. Add support for CPU sampling and nested scalar columns. Add more testcases for collection and usage of statistics and fix remaining unit/functional test failures. Thanks to Venki Korukanti (@vkorukanti) for the description below (modified to account for new changes). He graciously agreed to rebase the patch to latest master, fixed few issues and added few tests. FUNCS: Statistics functions as UDFs: Separate Currently using FieldReader to ensure consistent output type so that Unpivot doesn't get confused. All stats columns should be Nullable, so that stats functions can return NULL when N/A. * custom versions of "count" that always return BigInt * HyperLogLog based NDV that returns BigInt that works only on VarChars * HyperLogLog with binary output that only works on VarChars OPS: Updated protobufs for new ops OPS: Implemented StatisticsMerge OPS: Implemented StatisticsUnpivot ANALYZE: AnalyzeTable functionality * JavaCC syntax more-or-less copied from LucidDB. * (Basic) AnalyzePrule: DrillAnalyzeRel -> UnpivotPrel StatsMergePrel FilterPrel(for sampling) StatsAggPrel ScanPrel ANALYZE: Add getMetadataTable() to AbstractSchema USAGE: Change field access in QueryWrapper USAGE: Add getDrillTable() to DrillScanRelBase and ScanPrel * since ScanPrel does not inherit from DrillScanRelBase, this requires adding a DrillTable to the constructor * This is done so that a custom ReflectiveRelMetadataProvider can access the DrillTable associated with Logical/Physical scans. USAGE: Attach DrillStatsTable to DrillTable. * DrillStatsTable represents the data scanned from a corresponding ".stats.drill" table * In order to avoid doing query execution right after the ".stats.drill" table is found, metadata is not actually collected until the MaterializationVisitor is used. ** Currently, the metadata source must be a string (so that a SQL query can be created). Doing this with a table is probably more complicated. ** Query is set up to extract only the most recent statistics results for each column. closes #729
-rw-r--r--contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java2
-rw-r--r--contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java16
-rw-r--r--contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java1
-rw-r--r--contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java20
-rw-r--r--contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java19
-rw-r--r--contrib/native/client/src/protobuf/UserBitShared.pb.cc15
-rw-r--r--contrib/native/client/src/protobuf/UserBitShared.pb.h7
-rw-r--r--contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java1
-rw-r--r--contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java1
-rw-r--r--contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java9
-rw-r--r--contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java9
-rwxr-xr-xcontrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java7
-rwxr-xr-xcontrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java9
-rw-r--r--contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java1
-rw-r--r--contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java1
-rw-r--r--contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java1
-rw-r--r--contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java1
-rw-r--r--exec/java-exec/src/main/codegen/data/Parser.tdd20
-rw-r--r--exec/java-exec/src/main/codegen/includes/parserImpls.ftl18
-rw-r--r--exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java5
-rw-r--r--exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java139
-rw-r--r--exec/java-exec/src/main/codegen/templates/RecordWriter.java6
-rw-r--r--exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java93
-rw-r--r--exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java119
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java42
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java28
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java5597
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java24
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java29
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java20
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java38
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java5
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java26
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java69
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java214
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java3
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java223
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java48
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java150
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java103
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java87
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java68
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java130
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java62
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java62
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java198
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java95
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java96
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java44
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java249
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java (renamed from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java)5
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java405
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java38
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java114
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java102
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java14
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java386
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java171
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java63
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java201
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java26
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java3
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java113
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java4
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java33
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java3
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java20
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java29
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java25
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsMergePrel.java89
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java6
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java2
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java20
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java19
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java199
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java3
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java8
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java7
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java27
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java29
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java7
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java19
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java11
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java67
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java21
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java9
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java100
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java7
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java482
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java17
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java17
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java17
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java19
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java50
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java12
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngFormatPlugin.java19
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java13
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java13
-rw-r--r--exec/java-exec/src/main/resources/drill-module.conf9
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/TestSelectivity.java4
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java43
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java3
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java3
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java3
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java375
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java2
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java36
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java10
-rw-r--r--logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java13
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java20
-rw-r--r--protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java4
-rw-r--r--protocol/src/main/protobuf/UserBitShared.proto1
143 files changed, 11155 insertions, 890 deletions
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
index 1c30264fd..aeb117a8a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
@@ -94,7 +94,7 @@ public abstract class TableFormatPlugin implements FormatPlugin {
@Override
public AbstractWriter getWriter(PhysicalOperator child, String location,
- boolean append, List<String> partitionColumns) throws IOException {
+ List<String> partitionColumns) throws IOException {
throw new UnsupportedOperationException();
}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index b5cff5807..3011f4efb 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -24,6 +24,7 @@ import java.util.Set;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.StoragePluginOptimizerRule;
import org.apache.drill.exec.store.dfs.FileSelection;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
import org.apache.drill.exec.store.mapr.db.json.JsonScanSpec;
import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Connection;
@@ -134,6 +136,20 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
return getGroupScan(userName, selection, columns, (IndexDesc) null /* indexDesc */);
}
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
@JsonIgnore
public Configuration getHBaseConf() {
return hbaseConf;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
index 422a269b0..07943d990 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
@@ -279,6 +279,7 @@ public abstract class MapRDBGroupScan extends AbstractDbGroupScan {
return storagePlugin;
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
index 76466ab65..92f134fb7 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
@@ -26,11 +26,14 @@ import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
import org.apache.drill.exec.physical.base.AbstractWriter;
import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.dfs.FileSelection;
import org.apache.drill.exec.store.dfs.FormatMatcher;
import org.apache.drill.exec.store.mapr.TableFormatPlugin;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class StreamsFormatPlugin extends TableFormatPlugin {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamsFormatPlugin.class);
@@ -64,7 +67,7 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
@Override
public AbstractWriter getWriter(PhysicalOperator child, String location,
- boolean append, List<String> partitionColumns) throws IOException {
+ List<String> partitionColumns) throws IOException {
throw new UnsupportedOperationException();
}
@@ -77,4 +80,19 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
throw UserException.unsupportedError().message("MapR streams can not be querried at this time.").build(logger);
}
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
}
diff --git a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
index bf4b4b479..6f81ac633 100644
--- a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
+++ b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
@@ -17,6 +17,8 @@
*/
package org.apache.drill.exec.store.syslog;
+import java.io.IOException;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.SchemaPath;
@@ -34,6 +36,8 @@ import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class SyslogFormatPlugin extends EasyFormatPlugin<SyslogFormatConfig> {
@@ -79,4 +83,19 @@ public class SyslogFormatPlugin extends EasyFormatPlugin<SyslogFormatConfig> {
public int getWriterOperatorType() {
throw new UnsupportedOperationException("Drill does not support writing records to Syslog format.");
}
+
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
}
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index 1e0712091..0db64d348 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -754,7 +754,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
"entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA"
"TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN"
"CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ"
- "UESTED\020\006*\247\t\n\020CoreOperatorType\022\021\n\rSINGLE_"
+ "UESTED\020\006*\351\t\n\020CoreOperatorType\022\021\n\rSINGLE_"
"SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER"
"\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n"
"\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006"
@@ -784,10 +784,12 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
"\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_LIMI"
"T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT"
"ER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_SCAN"
- "\020:*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSA"
- "SL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL"
- "_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apach"
- "e.drill.exec.protoB\rUserBitSharedH\001", 5555);
+ "\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIVOT_"
+ "MAPS\020<\022\024\n\020STATISTICS_MERGE\020=*g\n\nSaslStat"
+ "us\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020"
+ "SASL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013"
+ "SASL_FAILED\020\004B.\n\033org.apache.drill.exec.p"
+ "rotoB\rUserBitSharedH\001", 5621);
::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
"UserBitShared.proto", &protobuf_RegisterTypes);
UserCredentials::default_instance_ = new UserCredentials();
@@ -967,6 +969,9 @@ bool CoreOperatorType_IsValid(int value) {
case 56:
case 57:
case 58:
+ case 59:
+ case 60:
+ case 61:
return true;
default:
return false;
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index b95b311c8..a8e6ccba6 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -262,11 +262,14 @@ enum CoreOperatorType {
PCAPNG_SUB_SCAN = 55,
RUNTIME_FILTER = 56,
ROWKEY_JOIN = 57,
- SYSLOG_SUB_SCAN = 58
+ SYSLOG_SUB_SCAN = 58,
+ STATISTICS_AGGREGATE = 59,
+ UNPIVOT_MAPS = 60,
+ STATISTICS_MERGE = 61
};
bool CoreOperatorType_IsValid(int value);
const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER;
-const CoreOperatorType CoreOperatorType_MAX = SYSLOG_SUB_SCAN;
+const CoreOperatorType CoreOperatorType_MAX = STATISTICS_MERGE;
const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1;
const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor();
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
index 22e318b29..e1b41e85e 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
@@ -388,6 +388,7 @@ public class HBaseGroupScan extends AbstractGroupScan implements DrillHBaseConst
return this.storagePluginConfig;
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
index d63174015..dfa2ee391 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
@@ -120,6 +120,7 @@ public class HiveScan extends AbstractGroupScan {
return hiveStoragePlugin.getConfig();
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
index a98193939..199d922ba 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.jdbc;
import java.util.List;
import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -38,14 +39,14 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
public class JdbcGroupScan extends AbstractGroupScan {
private final String sql;
- private final List<String> columns;
+ private final List<SchemaPath> columns;
private final JdbcStoragePlugin plugin;
private final double rows;
@JsonCreator
public JdbcGroupScan(
@JsonProperty("sql") String sql,
- @JsonProperty("columns") List<String> columns,
+ @JsonProperty("columns") List<SchemaPath> columns,
@JsonProperty("config") StoragePluginConfig config,
@JsonProperty("rows") double rows,
@JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
@@ -56,7 +57,7 @@ public class JdbcGroupScan extends AbstractGroupScan {
this.rows = rows;
}
- JdbcGroupScan(String sql, List<String> columns, JdbcStoragePlugin plugin, double rows) {
+ JdbcGroupScan(String sql, List<SchemaPath> columns, JdbcStoragePlugin plugin, double rows) {
super("");
this.sql = sql;
this.columns = columns;
@@ -91,7 +92,7 @@ public class JdbcGroupScan extends AbstractGroupScan {
return sql;
}
- public List<String> getColumns() {
+ public List<SchemaPath> getColumns() {
return columns;
}
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
index b8229402b..85f88a872 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
@@ -17,9 +17,11 @@
*/
package org.apache.drill.exec.store.jdbc;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
+import java.util.List;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.adapter.jdbc.JdbcImplementor;
import org.apache.calcite.plan.ConventionTraitDef;
@@ -32,6 +34,7 @@ import org.apache.calcite.rel.RelShuttleImpl;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.sql.SqlDialect;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
import org.apache.drill.exec.planner.physical.Prel;
@@ -91,7 +94,11 @@ public class JdbcPrel extends AbstractRelNode implements Prel {
@Override
public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) {
- JdbcGroupScan output = new JdbcGroupScan(sql, rowType.getFieldNames(), convention.getPlugin(), rows);
+ List<SchemaPath> columns = new ArrayList<>();
+ for (String col : rowType.getFieldNames()) {
+ columns.add(SchemaPath.getSimplePath(col));
+ }
+ JdbcGroupScan output = new JdbcGroupScan(sql, columns, convention.getPlugin(), rows);
return creator.addMetadata(this, output);
}
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
index 011c9bc58..5c6def26a 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
@@ -35,6 +35,7 @@ import javax.sql.DataSource;
import org.apache.drill.common.AutoCloseables;
import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.common.types.TypeProtos.MajorType;
import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -75,9 +76,9 @@ class JdbcRecordReader extends AbstractRecordReader {
private final String sql;
private ImmutableList<ValueVector> vectors;
private ImmutableList<Copier<?>> copiers;
- private final List<String> columns;
+ private final List<SchemaPath> columns;
- public JdbcRecordReader(DataSource source, String sql, String storagePluginName, List<String> columns) {
+ public JdbcRecordReader(DataSource source, String sql, String storagePluginName, List<SchemaPath> columns) {
this.source = source;
this.sql = sql;
this.storagePluginName = storagePluginName;
@@ -206,7 +207,7 @@ class JdbcRecordReader extends AbstractRecordReader {
ImmutableList.Builder<Copier<?>> copierBuilder = ImmutableList.builder();
for (int i = 1; i <= columnsCount; i++) {
- String name = columns.get(i - 1);
+ String name = columns.get(i - 1).getRootSegmentPath();
// column index in ResultSetMetaData starts from 1
int jdbcType = meta.getColumnType(i);
int width = meta.getPrecision(i);
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
index 9bc6de891..c9d5f0daf 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
@@ -18,6 +18,7 @@
package org.apache.drill.exec.store.jdbc;
import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.physical.base.AbstractSubScan;
import org.apache.drill.exec.proto.beans.CoreOperatorType;
@@ -36,12 +37,12 @@ public class JdbcSubScan extends AbstractSubScan {
private final String sql;
private final JdbcStoragePlugin plugin;
- private final List<String> columns;
+ private final List<SchemaPath> columns;
@JsonCreator
public JdbcSubScan(
@JsonProperty("sql") String sql,
- @JsonProperty("columns") List<String> columns,
+ @JsonProperty("columns") List<SchemaPath> columns,
@JsonProperty("config") StoragePluginConfig config,
@JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
super("");
@@ -50,7 +51,7 @@ public class JdbcSubScan extends AbstractSubScan {
this.plugin = (JdbcStoragePlugin) plugins.getPlugin(config);
}
- JdbcSubScan(String sql, List<String> columns, JdbcStoragePlugin plugin) {
+ JdbcSubScan(String sql, List<SchemaPath> columns, JdbcStoragePlugin plugin) {
super("");
this.sql = sql;
this.columns = columns;
@@ -66,7 +67,7 @@ public class JdbcSubScan extends AbstractSubScan {
return sql;
}
- public List<String> getColumns() {
+ public List<SchemaPath> getColumns() {
return columns;
}
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
index 510d3ad48..dded560e2 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
@@ -304,6 +304,7 @@ public class KafkaGroupScan extends AbstractGroupScan {
return kafkaStoragePlugin.getConfig();
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
index 104595092..593ca126b 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
@@ -246,6 +246,7 @@ public class KuduGroupScan extends AbstractGroupScan {
return kuduStoragePlugin.getConfig();
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
index e33acd477..46ea567d9 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
@@ -561,6 +561,7 @@ public class MongoGroupScan extends AbstractGroupScan implements
return Lists.newArrayList(affinityMap.values());
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
index ad258dfef..8a09637a1 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
@@ -149,6 +149,7 @@ public class OpenTSDBGroupScan extends AbstractGroupScan {
return storagePluginConfig;
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index ec56af455..4461f6d98 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -42,25 +42,6 @@
"COMPUTE",
"ESTIMATE",
"STATISTICS",
- "COLUMNS",
- "SAMPLE"
- ]
-
- # List of keywords from "keywords" section that are not reserved by SQL:2003 standard.
- # Example: "DATABASES", "TABLES" are keywords but are not reserved by SQL:2003 standard.
- # First keyword that starts the statement should be a reserved keyword, otherwise the current parser
- # ends up considering it as a expression and fails.
- nonReservedKeywords: [
- "DATABASES",
- "REPLACE",
- "SCHEMAS",
- "TABLES",
- "FILES",
- "METADATA",
- "COMPUTE",
- "ESTIMATE",
- "STATISTICS",
- "COLUMNS",
"SAMPLE"
]
@@ -77,7 +58,6 @@
"SqlRefreshMetadata()",
"SqlCreateFunction()",
"SqlDropFunction()",
- "SqlRefreshMetadata()",
"SqlAnalyzeTable()"
]
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 2606006d7..2da1c4a64 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -550,9 +550,9 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
SqlIdentifier id;
}
{
- id = SimpleIdentifier() {list.add(id);}
+ id = CompoundIdentifier() {list.add(id);}
(
- <COMMA> id = SimpleIdentifier() {list.add(id);}) *
+ <COMMA> id = CompoundIdentifier() {list.add(id);}) *
{
return Pair.of(new SqlNodeList(list, getPos()), null);
}
@@ -560,8 +560,8 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
</#if>
/**
* Parses a analyze statement.
- * ANALYZE TABLE tblname {COMPUTE | ESTIMATE} | STATISTICS FOR
- * {ALL COLUMNS | COLUMNS (field1, field2, ...)} [ SAMPLE numeric PERCENT ]
+ * ANALYZE TABLE tblname {COMPUTE | ESTIMATE} | STATISTICS
+ * [(column1, column2, ...)] [ SAMPLE numeric PERCENT ]
*/
SqlNode SqlAnalyzeTable() :
{
@@ -580,12 +580,10 @@ SqlNode SqlAnalyzeTable() :
|
<ESTIMATE> { estimate = SqlLiteral.createBoolean(true, pos); }
)
- <STATISTICS> <FOR>
- (
- ( <ALL> <COLUMNS> )
- |
- ( <COLUMNS> fieldList = ParseRequiredFieldList("Table") )
- )
+ <STATISTICS>
+ [
+ (fieldList = ParseRequiredFieldList("Table"))
+ ]
[
<SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
{
diff --git a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
index 070e947aa..2bd5faf1e 100644
--- a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
@@ -81,4 +81,9 @@ public abstract class AbstractRecordWriter implements RecordWriter {
</#list>
</#list>
</#list>
+
+ @Override
+ public void postProcessing() throws IOException {
+ // no op
+ }
}
diff --git a/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
new file mode 100644
index 000000000..14851271b
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
@@ -0,0 +1,139 @@
+/*
+ * 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.
+ */
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/JSONBaseStatisticsRecordWriter.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.List;
+
+/**
+ * Abstract implementation of StatisticsRecordWriter interface which exposes interface:
+ * {@link #writeHeader(List)}
+ * {@link #addField(int,String)}
+ * to output the data in string format instead of implementing addField for each type holder.
+ *
+ * This is useful for text format writers such as CSV, TSV etc.
+ *
+ * NB: Source code generated using FreeMarker template ${.template_name}
+ */
+public abstract class JSONBaseStatisticsRecordWriter implements StatisticsRecordWriter {
+
+ protected JsonOutput gen;
+ protected boolean skipNullFields = true;
+
+ <#list vv.types as type>
+ <#list type.minor as minor>
+ <#list vv.modes as mode>
+ <#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
+ @Override
+ public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader) {
+ return new ${mode.prefix}${minor.class}JsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class ${mode.prefix}${minor.class}JsonConverter extends FieldConverter {
+
+ public ${mode.prefix}${minor.class}JsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ <#if mode.prefix == "Nullable" >
+ if (!skipNullFields || this.reader.isSet()) {
+ gen.writeFieldName(fieldName);
+ }
+ <#else>
+ gen.writeFieldName(fieldName);
+ </#if>
+ }
+
+ @Override
+ public void writeField() throws IOException {
+
+ <#assign typeName = minor.class >
+
+ <#switch minor.class>
+ <#case "UInt1">
+ <#case "UInt2">
+ <#case "UInt4">
+ <#case "UInt8">
+ <#assign typeName = "unsupported">
+ <#break>
+
+ <#case "Decimal9">
+ <#case "Decimal18">
+ <#case "Decimal28Sparse">
+ <#case "Decimal28Dense">
+ <#case "Decimal38Dense">
+ <#case "Decimal38Sparse">
+ <#case "VarDecimal">
+ <#assign typeName = "Decimal">
+ <#break>
+ <#case "Float4">
+ <#assign typeName = "Float">
+ <#break>
+ <#case "Float8">
+ <#assign typeName = "Double">
+ <#break>
+
+ <#case "IntervalDay">
+ <#case "IntervalYear">
+ <#assign typeName = "Interval">
+ <#break>
+
+ <#case "Bit">
+ <#assign typeName = "Boolean">
+ <#break>
+
+ <#case "TimeStamp">
+ <#assign typeName = "Timestamp">
+ <#break>
+
+ <#case "VarBinary">
+ <#assign typeName = "Binary">
+ <#break>
+
+ </#switch>
+
+ <#if typeName == "unsupported">
+ throw new UnsupportedOperationException("Unable to currently write ${minor.class} type to JSON.");
+ <#else>
+ <#if mode.prefix == "Nullable" >
+ if (!skipNullFields || this.reader.isSet()) {
+ gen.write${typeName}(reader);
+ }
+ <#else>
+ gen.write${typeName}(reader);
+ </#if>
+ </#if>
+ }
+ }
+ </#list>
+ </#list>
+ </#list>
+
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/RecordWriter.java b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
index 470c5f26e..e07ab7c4b 100644
--- a/exec/java-exec/src/main/codegen/templates/RecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
@@ -85,7 +85,11 @@ public interface RecordWriter {
* @throws IOException
*/
void endRecord() throws IOException;
-
+ /**
+ * Called after adding all the records to perform any post processing related tasks
+ * @throws IOException
+ */
+ void postProcessing() throws IOException;
void abort() throws IOException;
void cleanup() throws IOException;
}
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
new file mode 100644
index 000000000..85c5bdebb
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordWriter.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import java.io.IOException;
+import java.util.Map;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+/** StatisticsRecordWriter interface. */
+public interface StatisticsRecordWriter {
+
+ /**
+ * Initialize the writer.
+ *
+ * @param writerOptions Contains key, value pair of settings.
+ * @throws IOException
+ */
+ void init(Map<String, String> writerOptions) throws IOException;
+
+ /**
+ * Update the schema in RecordWriter. Called at least once before starting writing the records.
+ * @param batch
+ * @throws IOException
+ */
+ void updateSchema(VectorAccessible batch) throws IOException;
+
+ /**
+ * Check if the writer should start a new partition, and if so, start a new partition
+ */
+ public void checkForNewPartition(int index);
+
+ /**
+ * Returns if the writer is a blocking writer i.e. consumes all input before writing it out
+ * @return TRUE, if writer is blocking. FALSE, otherwise
+ */
+ boolean isBlockingWriter();
+
+ /**
+ * Called before starting writing fields in a record.
+ * @throws IOException
+ */
+ void startStatisticsRecord() throws IOException;
+
+ <#list vv.types as type>
+ <#list type.minor as minor>
+ <#list vv.modes as mode>
+ /** Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>. */
+ public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader);
+
+ </#list>
+ </#list>
+ </#list>
+
+ /**
+ * Called after adding all fields in a particular statistics record are added using
+ * add{TypeHolder}(fieldId, TypeHolder) methods.
+ * @throws IOException
+ */
+ void endStatisticsRecord() throws IOException;
+ /**
+ * For a blocking writer, called after processing all the records to flush out the writes
+ * @throws IOException
+ */
+ void flushBlockingWriter() throws IOException;
+ void abort() throws IOException;
+ void cleanup() throws IOException;
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
new file mode 100644
index 000000000..dd3ab7af3
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ */
+
+import org.apache.drill.exec.planner.physical.WriterPrel;
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordWriterImpl.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import com.google.common.collect.Lists;
+
+import org.apache.drill.exec.planner.physical.WriterPrel;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import java.io.IOException;
+import java.util.List;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+/** Reads records from the RecordValueAccessor and writes into StatisticsRecordWriter. */
+public class StatisticsRecordWriterImpl {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsRecordWriterImpl.class);
+
+ private VectorAccessible batch;
+ private StatisticsRecordWriter recordWriter;
+ private List<FieldConverter> fieldConverters;
+
+ public StatisticsRecordWriterImpl(VectorAccessible batch, StatisticsRecordWriter recordWriter)
+ throws IOException {
+ this.batch = batch;
+ this.recordWriter = recordWriter;
+ initFieldWriters();
+ }
+
+ public int writeStatistics(int recordCount) throws IOException {
+ int counter = 0;
+
+ for (; counter < recordCount; counter++) {
+ recordWriter.checkForNewPartition(counter);
+ recordWriter.startStatisticsRecord();
+ // write the current record
+ for (FieldConverter converter : fieldConverters) {
+ converter.setPosition(counter);
+ converter.startField();
+ converter.writeField();
+ converter.endField();
+ }
+ recordWriter.endStatisticsRecord();
+ }
+
+ return counter;
+ }
+
+ public void flushBlockingWriter() throws IOException {
+ if (recordWriter.isBlockingWriter()) {
+ recordWriter.flushBlockingWriter();
+ }
+ }
+
+ private void initFieldWriters() throws IOException {
+ fieldConverters = Lists.newArrayList();
+ try {
+ int fieldId = 0;
+ for (VectorWrapper w : batch) {
+ if (w.getField().getName().equalsIgnoreCase(WriterPrel.PARTITION_COMPARATOR_FIELD)) {
+ continue;
+ }
+ FieldReader reader = w.getValueVector().getReader();
+ FieldConverter converter = getConverter(recordWriter, fieldId++, w.getField().getName(), reader);
+ fieldConverters.add(converter);
+ }
+ } catch(Exception e) {
+ logger.error("Failed to create FieldWriter.", e);
+ throw new IOException("Failed to initialize FieldWriters.", e);
+ }
+ }
+
+ public static FieldConverter getConverter(StatisticsRecordWriter recordWriter, int fieldId, String fieldName,
+ FieldReader reader) {
+ switch (reader.getType().getMinorType()) {
+ <#list vv.types as type>
+ <#list type.minor as minor>
+ case ${minor.class?upper_case}:
+ switch (reader.getType().getMode()) {
+ case REQUIRED:
+ return recordWriter.getNew${minor.class}Converter(fieldId, fieldName, reader);
+ case OPTIONAL:
+ return recordWriter.getNewNullable${minor.class}Converter(fieldId, fieldName, reader);
+ case REPEATED:
+ return recordWriter.getNewRepeated${minor.class}Converter(fieldId, fieldName, reader);
+ }
+ </#list>
+ </#list>
+ }
+ throw new UnsupportedOperationException();
+ }
+} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index bab94a180..ac3252ad7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -22,11 +22,13 @@ import org.apache.drill.exec.physical.impl.common.HashTable;
import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
import org.apache.drill.exec.server.options.OptionValidator;
import org.apache.drill.exec.server.options.OptionValidator.OptionDescription;
-import org.apache.drill.exec.server.options.TypeValidators.DateTimeFormatValidator;
-import org.apache.drill.exec.server.options.TypeValidators.IntegerValidator;
+import org.apache.drill.exec.server.options.TypeValidators.AdminUserGroupsValidator;
+import org.apache.drill.exec.server.options.TypeValidators.AdminUsersValidator;
import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DateTimeFormatValidator;
import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
+import org.apache.drill.exec.server.options.TypeValidators.IntegerValidator;
import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
import org.apache.drill.exec.server.options.TypeValidators.MaxWidthValidator;
import org.apache.drill.exec.server.options.TypeValidators.PositiveLongValidator;
@@ -34,8 +36,6 @@ import org.apache.drill.exec.server.options.TypeValidators.PowerOfTwoLongValidat
import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
-import org.apache.drill.exec.server.options.TypeValidators.AdminUsersValidator;
-import org.apache.drill.exec.server.options.TypeValidators.AdminUserGroupsValidator;
import org.apache.drill.exec.testing.ExecutionControls;
import org.apache.drill.exec.vector.ValueVector;
@@ -911,4 +911,38 @@ public final class ExecConstants {
new OptionDescription("Controls whether to return result set for CREATE TABLE / VIEW / FUNCTION, DROP TABLE / VIEW / FUNCTION, " +
"SET, USE, REFRESH METADATA TABLE queries. If set to false affected rows count will be returned instead and result set will be null. " +
"Affects JDBC connections only. Default is true. (Drill 1.15+)"));
+
+ /**
+ * Option whose value is a long value representing the number of bits required for computing ndv (using HLL).
+ * Controls the trade-off between accuracy and memory requirements. The number of bits correlates positively with accuracy
+ */
+ public static final String HLL_ACCURACY = "exec.statistics.ndv_accuracy";
+ public static final LongValidator HLL_ACCURACY_VALIDATOR = new PositiveLongValidator(HLL_ACCURACY, 30,
+ new OptionDescription("Controls trade-off between NDV statistic computation memory cost and accuracy"));
+
+ /**
+ * Option whose value is a boolean value representing whether to perform deterministic sampling. It translates to using
+ * the same (pre-defined) seed for the underlying pseudo-random number generator.
+ */
+ public static final String DETERMINISTIC_SAMPLING = "exec.statistics.deterministic_sampling";
+ public static final BooleanValidator DETERMINISTIC_SAMPLING_VALIDATOR = new BooleanValidator(DETERMINISTIC_SAMPLING,
+ new OptionDescription("Deterministic sampling"));
+
+ /**
+ * Option whose value is a long value representing the expected number of elements in the bloom filter. The bloom filter
+ * computes the number of duplicates which is used for extrapolating the NDV when using sampling. Controls the trade-off
+ * between accuracy and memory requirements. The number of elements correlates positively with accuracy.
+ */
+ public static final String NDV_BLOOM_FILTER_ELEMENTS = "exec.statistics.ndv_extrapolation_bf_elements";
+ public static final LongValidator NDV_BLOOM_FILTER_ELEMENTS_VALIDATOR = new PositiveLongValidator(NDV_BLOOM_FILTER_ELEMENTS, Integer.MAX_VALUE,
+ new OptionDescription("Controls trade-off between NDV statistic computation memory cost and sampling extrapolation accuracy"));
+
+ /**
+ * Option whose value is a double value representing the desired max false positive probability in the bloom filter. The bloom filter
+ * computes the number of duplicates which is used for extrapolating the NDV when using sampling. Controls the trade-off
+ * between accuracy and memory requirements. The probability correlates negatively with the accuracy.
+ */
+ public static final String NDV_BLOOM_FILTER_FPOS_PROB = "exec.statistics.ndv_extrapolation_bf_fpprobability";
+ public static final LongValidator NDV_BLOOM_FILTER_FPOS_PROB_VALIDATOR = new PositiveLongValidator(NDV_BLOOM_FILTER_FPOS_PROB,
+ 100, new OptionDescription("Controls trade-off between NDV statistic computation memory cost and sampling extrapolation accuracy"));
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
index 589e98295..f4d060150 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.fs.Path;
public enum DotDrillType {
VIEW,
- // ,FORMAT
STATS;
private final String ending;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 8470e1fb0..dd5b359e2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
import org.apache.drill.exec.expr.annotations.Output;
import org.apache.drill.exec.expr.annotations.Param;
import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
import org.apache.drill.exec.expr.holders.Float8Holder;
import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
@@ -61,7 +62,34 @@ public class MathFunctions{
public void eval(){
out.value = java.lang.Math.random();
}
+ }
+
+ @FunctionTemplate(name = "rand", isRandom = true,
+ scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+ public static class RandomWithoutSeed implements DrillSimpleFunc{
+ @Output Float8Holder out;
+
+ public void setup(){}
+
+ public void eval(){
+ out.value = java.lang.Math.random();
+ }
+ }
+
+ @FunctionTemplate(name = "rand", isRandom = true,
+ scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+ public static class RandomWithSeed implements DrillSimpleFunc{
+ @Param BigIntHolder seed;
+ @Workspace java.util.Random rand;
+ @Output Float8Holder out;
+ public void setup(){
+ rand = new java.util.Random(seed.value);
+ }
+
+ public void eval(){
+ out.value = rand.nextDouble();
+ }
}
@FunctionTemplate(name = "to_number", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
index c6430dd9c..79765edfc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
@@ -1,4 +1,4 @@
-/*******************************************************************************
+/*
* 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
@@ -6,15 +6,15 @@
* 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.
- ******************************************************************************/
+ */
/*
* This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
@@ -23,28 +23,65 @@
package org.apache.drill.exec.expr.fn.impl;
import io.netty.buffer.DrillBuf;
+import javax.inject.Inject;
import org.apache.drill.exec.expr.DrillAggFunc;
import org.apache.drill.exec.expr.DrillSimpleFunc;
import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
import org.apache.drill.exec.expr.annotations.Output;
import org.apache.drill.exec.expr.annotations.Param;
import org.apache.drill.exec.expr.annotations.Workspace;
import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.Decimal18Holder;
+import org.apache.drill.exec.expr.holders.Decimal28DenseHolder;
+import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.Decimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
+import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.IntervalHolder;
import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
+import org.apache.drill.exec.expr.holders.NullableBitHolder;
+import org.apache.drill.exec.expr.holders.NullableDateHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
+import org.apache.drill.exec.expr.holders.NullableDecimal28DenseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
+import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
+import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
+import org.apache.drill.exec.expr.holders.NullableIntHolder;
+import org.apache.drill.exec.expr.holders.NullableIntervalHolder;
+import org.apache.drill.exec.expr.holders.NullableTimeHolder;
+import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
+import org.apache.drill.exec.expr.holders.NullableVar16CharHolder;
import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.TimeHolder;
+import org.apache.drill.exec.expr.holders.TimeStampHolder;
+import org.apache.drill.exec.expr.holders.Var16CharHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.vector.complex.reader.FieldReader;
-import javax.inject.Inject;
-
@SuppressWarnings("unused")
public class StatisticsAggrFunctions {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggrFunctions.class);
- @FunctionTemplate(name = "statcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
- public static class StatCount implements DrillAggFunc {
+ /* IMPORTANT NOTE: Please make sure to create a new function for each datatype. See the examples below.
+ * This will result in more performant generated code. Use switch-case/if-else statements judiciously
+ * as it MAY cause the generated code to slow down considerably.
+ * */
+ @FunctionTemplate(name = "rowcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class RowCount implements DrillAggFunc {
@Param FieldReader in;
@Workspace BigIntHolder count;
@Output NullableBigIntHolder out;
@@ -71,8 +108,8 @@ public class StatisticsAggrFunctions {
}
}
- @FunctionTemplate(name = "nonnullstatcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
- public static class NonNullStatCount implements DrillAggFunc {
+ @FunctionTemplate(name = "nonnullrowcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NonNullRowCount implements DrillAggFunc {
@Param FieldReader in;
@Workspace BigIntHolder count;
@Output NullableBigIntHolder out;
@@ -101,17 +138,27 @@ public class StatisticsAggrFunctions {
}
}
+ /**
+ * The log2m parameter defines the accuracy of the counter. The larger the
+ * log2m the better the accuracy.
+ * accuracy = 1.04/sqrt(2^log2m)
+ * where
+ * log2m - the number of bits to use as the basis for the HLL instance
+ */
@FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
public static class HllFieldReader implements DrillAggFunc {
@Param FieldReader in;
@Workspace ObjectHolder work;
@Output NullableVarBinaryHolder out;
+ @Inject OptionManager options;
@Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
@Override
public void setup() {
work = new ObjectHolder();
- work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
}
@Override
@@ -134,6 +181,9 @@ public class StatisticsAggrFunctions {
case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
hll.offer(in.readText().toString());
break;
+ case org.apache.drill.common.types.TypeProtos.MinorType.BIGINT_VALUE:
+ hll.offer(in.readLong());
+ break;
default:
work.obj = null;
}
@@ -167,70 +217,310 @@ public class StatisticsAggrFunctions {
@Override
public void reset() {
- work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
}
}
+ /**
+ * The log2m parameter defines the accuracy of the counter. The larger the log2m the better the accuracy where:
+ * accuracy = 1.04/sqrt(2^log2m)
+ * log2m - the number of bits to use as the basis for the HLL instance
+ * The parameter accepts integers in the range [0, 30]
+ */
+ @FunctionTemplate(name = "hll_decode", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+ public static class HllDecode implements DrillSimpleFunc {
- @FunctionTemplate(name = "ndv", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
- public static class NdvVarBinary implements DrillAggFunc {
@Param
- FieldReader in;
+ NullableVarBinaryHolder in;
+ @Output
+ BigIntHolder out;
+
+ @Override
+ public void setup() {
+ }
+
+ public void eval() {
+ out.value = -1;
+
+ if (in.isSet != 0) {
+ byte[] din = new byte[in.end - in.start];
+ in.buffer.getBytes(in.start, din);
+ try {
+ out.value = com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(din).cardinality();
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failure evaluating hll_decode", e);
+ }
+ }
+ }
+ }
+
+ /**
+ * The log2m parameter defines the accuracy of the counter. The larger the log2m the better the accuracy where:
+ * accuracy = 1.04/sqrt(2^log2m)
+ * log2m - the number of bits to use as the basis for the HLL instance
+ * The parameter accepts integers in the range [0, 30]
+ */
+ @FunctionTemplate(name = "hll_merge", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class HllMerge implements DrillAggFunc {
+ @Param NullableVarBinaryHolder in;
+ @Workspace ObjectHolder work;
+ @Output NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ try {
+ if (in.isSet != 0) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes();
+ com.clearspring.analytics.stream.cardinality.HyperLogLog other =
+ com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(buf);
+ hll.addAll(other);
+ }
+ } catch (Exception e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to merge HyperLogLog output", e);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ /**
+ * The log2m parameter defines the accuracy of the counter. The larger the log2m the better the accuracy where:
+ * accuracy = 1.04/sqrt(2^log2m)
+ * log2m - the number of bits to use as the basis for the HLL instance
+ * The parameter accepts integers in the range [0, 30]
+ */
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BitHLLFunction implements DrillAggFunc {
+ @Param
+ BitHolder in;
@Workspace
ObjectHolder work;
@Output
- NullableBigIntHolder out;
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
@Override
public void setup() {
work = new ObjectHolder();
- work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
}
@Override
public void add() {
if (work.obj != null) {
com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
- (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
- int mode = in.getType().getMode().getNumber();
- int type = in.getType().getMinorType().getNumber();
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
- switch (mode) {
- case org.apache.drill.common.types.TypeProtos.DataMode.OPTIONAL_VALUE:
- if (!in.isSet()) {
- hll.offer(null);
- break;
- }
- // fall through //
- case org.apache.drill.common.types.TypeProtos.DataMode.REQUIRED_VALUE:
- switch (type) {
- case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
- hll.offer(in.readText().toString());
- break;
- case org.apache.drill.common.types.TypeProtos.MinorType.FLOAT8_VALUE:
- hll.offer(in.readDouble());
- break;
- case org.apache.drill.common.types.TypeProtos.MinorType.INT_VALUE:
- hll.offer(in.readInteger());
- break;
- case org.apache.drill.common.types.TypeProtos.MinorType.BIGINT_VALUE:
- hll.offer(in.readLong());
- break;
- case org.apache.drill.common.types.TypeProtos.MinorType.DATE_VALUE:
- case org.apache.drill.common.types.TypeProtos.MinorType.TIMESTAMP_VALUE:
- case org.apache.drill.common.types.TypeProtos.MinorType.TIME_VALUE:
- case org.apache.drill.common.types.TypeProtos.MinorType.TIMETZ_VALUE:
- hll.offer(in.readLocalDateTime());
- break;
- case org.apache.drill.common.types.TypeProtos.MinorType.VARBINARY_VALUE:
- hll.offer(in.readByteArray());
- break;
- default:
- work.obj = null;
- }
- break;
- default:
- work.obj = null;
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBitHLLFunction implements DrillAggFunc {
+ @Param
+ NullableBitHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntHLLFunction implements DrillAggFunc {
+ @Param
+ IntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntHLLFunction implements DrillAggFunc {
+ @Param
+ NullableIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
}
}
}
@@ -239,10 +529,994 @@ public class StatisticsAggrFunctions {
public void output() {
if (work.obj != null) {
com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BigIntHLLFunction implements DrillAggFunc {
+ @Param
+ BigIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBigIntHLLFunction implements DrillAggFunc {
+ @Param
+ NullableBigIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float4HLLFunction implements DrillAggFunc {
+ @Param
+ Float4Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat4HLLFunction implements DrillAggFunc {
+ @Param
+ NullableFloat4Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float8HLLFunction implements DrillAggFunc {
+ @Param
+ Float8Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat8HLLFunction implements DrillAggFunc {
+ @Param
+ NullableFloat8Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal9HLLFunction implements DrillAggFunc {
+ @Param
+ Decimal9Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal9HLLFunction implements DrillAggFunc {
+ @Param
+ NullableDecimal9Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal18HLLFunction implements DrillAggFunc {
+ @Param
+ Decimal18Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal18HLLFunction implements DrillAggFunc {
+ @Param
+ NullableDecimal18Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class DateHLLFunction implements DrillAggFunc {
+ @Param
+ DateHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDateHLLFunction implements DrillAggFunc {
+ @Param
+ NullableDateHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeHLLFunction implements DrillAggFunc {
+ @Param
+ TimeHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeHLLFunction implements DrillAggFunc {
+ @Param
+ NullableTimeHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeStampHLLFunction implements DrillAggFunc {
+ @Param
+ TimeStampHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ hll.offer(in.value);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeStampHLLFunction implements DrillAggFunc {
+ @Param
+ NullableTimeStampHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ hll.offer(in.value);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntervalHLLFunction implements DrillAggFunc {
+ @Param
+ IntervalHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace
+ ObjectHolder interval;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ interval = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ interval.obj = new java.util.ArrayList<Integer>(3);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null
+ && interval.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
(com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ java.util.ArrayList<Integer> intervalList = (java.util.ArrayList<Integer>)interval.obj;
+ intervalList.clear();
+ intervalList.add(in.days);
+ intervalList.add(in.months);
+ intervalList.add(in.milliseconds);
+ hll.offer(interval.obj);
+ }
+ }
- out.isSet = 1;
- out.value = hll.cardinality();
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
} else {
out.isSet = 0;
}
@@ -250,36 +1524,4217 @@ public class StatisticsAggrFunctions {
@Override
public void reset() {
- work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ interval.obj = new java.util.ArrayList<Integer>(3);
}
}
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntervalHLLFunction implements DrillAggFunc {
+ @Param
+ NullableIntervalHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace
+ ObjectHolder interval;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
- @FunctionTemplate(name = "hll_decode", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
- public static class HllDecode implements DrillSimpleFunc {
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ interval = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ interval.obj = new java.util.ArrayList<Integer>(3);
+ }
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ if (interval.obj != null) {
+ java.util.ArrayList<Integer> intervalList = (java.util.ArrayList<Integer>)interval.obj;
+ intervalList.clear();
+ intervalList.add(in.days);
+ intervalList.add(in.months);
+ intervalList.add(in.milliseconds);
+ hll.offer(interval.obj);
+ }
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ interval.obj = new java.util.ArrayList<Integer>(3);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarCharHLLFunction implements DrillAggFunc {
@Param
- NullableVarBinaryHolder in;
+ VarCharHolder in;
+ @Workspace
+ ObjectHolder work;
@Output
- BigIntHolder out;
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
@Override
public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
}
- public void eval() {
- out.value = -1;
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
- if (in.isSet != 0) {
- byte[] din = new byte[in.end - in.start];
- in.buffer.getBytes(in.start, din);
try {
- out.value = com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(din).cardinality();
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarCharHLLFunction implements DrillAggFunc {
+ @Param
+ NullableVarCharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
} catch (java.io.IOException e) {
- throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failure evaluation hll_decode", e);
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Var16CharHLLFunction implements DrillAggFunc {
+ @Param
+ Var16CharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+ (in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVar16CharHLLFunction implements DrillAggFunc {
+ @Param
+ NullableVar16CharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+ (in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarBinaryHLLFunction implements DrillAggFunc {
+ @Param
+ VarBinaryHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+ (in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarBinaryHLLFunction implements DrillAggFunc {
+ @Param
+ NullableVarBinaryHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Output
+ NullableVarBinaryHolder out;
+ @Inject OptionManager options;
+ @Inject DrillBuf buffer;
+ @Workspace IntHolder hllAccuracy;
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+ (in.start, in.end, in.buffer).getBytes();
+ hll.offer(buf);
+ } else {
+ hll.offer(null);
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+ (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+ try {
+ byte[] ba = hll.getBytes();
+ out.buffer = buffer.reallocIfNeeded(ba.length);
+ out.start = 0;
+ out.end = ba.length;
+ out.buffer.setBytes(0, ba);
+ out.isSet = 1;
+ } catch (java.io.IOException e) {
+ throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+ }
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BitAvgWidthFunction implements DrillAggFunc {
+ @Param BitHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 1;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBitAvgWidthFunction implements DrillAggFunc {
+ @Param NullableBitHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 8;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntAvgWidthFunction implements DrillAggFunc {
+ @Param IntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntAvgWidthFunction implements DrillAggFunc {
+ @Param NullableIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BigIntAvgWidthFunction implements DrillAggFunc {
+ @Param BigIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBigIntAvgWidthFunction implements DrillAggFunc {
+ @Param NullableBigIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal9AvgWidthFunction implements DrillAggFunc {
+ @Param Decimal9Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal9AvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal9Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal18AvgWidthFunction implements DrillAggFunc {
+ @Param Decimal18Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal18AvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal18Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal28DenseAvgWidthFunction implements DrillAggFunc {
+ @Param Decimal28DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal28DenseAvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal28DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal28SparseAvgWidthFunction implements DrillAggFunc {
+ @Param Decimal28SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal28SparseAvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal28SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal38DenseAvgWidthFunction implements DrillAggFunc {
+ @Param Decimal38DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal38DenseAvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal38DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal38SparseAvgWidthFunction implements DrillAggFunc {
+ @Param Decimal38SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal38SparseAvgWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal38SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float4AvgWidthFunction implements DrillAggFunc {
+ @Param Float4Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Float.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat4AvgWidthFunction implements DrillAggFunc {
+ @Param NullableFloat4Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Float.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float8AvgWidthFunction implements DrillAggFunc {
+ @Param Float8Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Double.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat8AvgWidthFunction implements DrillAggFunc {
+ @Param NullableFloat8Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Double.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class DateAvgWidthFunction implements DrillAggFunc {
+ @Param DateHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDateAvgWidthFunction implements DrillAggFunc {
+ @Param NullableDateHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeAvgWidthFunction implements DrillAggFunc {
+ @Param TimeHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeAvgWidthFunction implements DrillAggFunc {
+ @Param NullableTimeHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeStampAvgWidthFunction implements DrillAggFunc {
+ @Param TimeStampHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeStampAvgWidthFunction implements DrillAggFunc {
+ @Param NullableTimeStampHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value * 8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntervalAvgWidthFunction implements DrillAggFunc {
+ @Param IntervalHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntervalAvgWidthFunction implements DrillAggFunc {
+ @Param NullableIntervalHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarCharAvgWidthFunction implements DrillAggFunc {
+ @Param VarCharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarCharAvgWidthFunction implements DrillAggFunc {
+ @Param NullableVarCharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value/((double)count.value);
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Var16CharAvgWidthFunction implements DrillAggFunc {
+ @Param Var16CharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+ in.start, in.end, in.buffer).getBytes().length;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVar16CharAvgWidthFunction implements DrillAggFunc {
+ @Param NullableVar16CharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value/((double)count.value);
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarBinaryAvgWidthFunction implements DrillAggFunc {
+ @Param VarBinaryHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/((double)count.value);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarBinaryAvgWidthFunction implements DrillAggFunc {
+ @Param NullableVarBinaryHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder count;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ count = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ count.value++;
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value/((double)count.value);
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ count.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BitSumWidthFunction implements DrillAggFunc {
+ @Param BitHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 8;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBitSumWidthFunction implements DrillAggFunc {
+ @Param NullableBitHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 8;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntSumWidthFunction implements DrillAggFunc {
+ @Param IntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntSumWidthFunction implements DrillAggFunc {
+ @Param NullableIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BigIntSumWidthFunction implements DrillAggFunc {
+ @Param BigIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBigIntSumWidthFunction implements DrillAggFunc {
+ @Param NullableBigIntHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal9SumWidthFunction implements DrillAggFunc {
+ @Param Decimal9Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal9SumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal9Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal18SumWidthFunction implements DrillAggFunc {
+ @Param Decimal18Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal18SumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal18Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal28DenseSumWidthFunction implements DrillAggFunc {
+ @Param Decimal28DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal28DenseSumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal28DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal28SparseSumWidthFunction implements DrillAggFunc {
+ @Param Decimal28SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal28SparseSumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal28SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal38DenseSumWidthFunction implements DrillAggFunc {
+ @Param Decimal38DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal38DenseSumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal38DenseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal38SparseSumWidthFunction implements DrillAggFunc {
+ @Param Decimal38SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal38SparseSumWidthFunction implements DrillAggFunc {
+ @Param NullableDecimal38SparseHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 16;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float4SumWidthFunction implements DrillAggFunc {
+ @Param Float4Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Float.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat4SumWidthFunction implements DrillAggFunc {
+ @Param NullableFloat4Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Float.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float8SumWidthFunction implements DrillAggFunc {
+ @Param Float8Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Double.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat8SumWidthFunction implements DrillAggFunc {
+ @Param NullableFloat8Holder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Double.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class DateSumWidthFunction implements DrillAggFunc {
+ @Param DateHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDateSumWidthFunction implements DrillAggFunc {
+ @Param NullableDateHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeSumWidthFunction implements DrillAggFunc {
+ @Param TimeHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeSumWidthFunction implements DrillAggFunc {
+ @Param NullableTimeHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Integer.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeStampSumWidthFunction implements DrillAggFunc {
+ @Param TimeStampHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeStampSumWidthFunction implements DrillAggFunc {
+ @Param NullableTimeStampHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += Long.SIZE;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value/(8.0);
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntervalSumWidthFunction implements DrillAggFunc {
+ @Param IntervalHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntervalSumWidthFunction implements DrillAggFunc {
+ @Param NullableIntervalHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += 12;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarCharSumWidthFunction implements DrillAggFunc {
+ @Param VarCharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarCharSumWidthFunction implements DrillAggFunc {
+ @Param NullableVarCharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value;
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Var16CharSumWidthFunction implements DrillAggFunc {
+ @Param Var16CharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+ in.start, in.end, in.buffer).getBytes().length;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVar16CharSumWidthFunction implements DrillAggFunc {
+ @Param NullableVar16CharHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value;
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarBinarySumWidthFunction implements DrillAggFunc {
+ @Param VarBinaryHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ }
+
+ @Override
+ public void output() {
+ out.value = totWidth.value;
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarBinarySumWidthFunction implements DrillAggFunc {
+ @Param NullableVarBinaryHolder in;
+ @Workspace BigIntHolder totWidth;
+ @Workspace BigIntHolder nonNullCount;
+ @Output NullableFloat8Holder out;
+
+ @Override
+ public void setup() {
+ totWidth = new BigIntHolder();
+ nonNullCount = new BigIntHolder();
+ }
+
+ @Override
+ public void add() {
+ if (in.isSet == 1) {
+ totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes().length;
+ nonNullCount.value += 1;
+ }
+ }
+
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = totWidth.value;
+ } else {
+ out.value = 0;
+ }
+ out.isSet = 1;
+ }
+
+ @Override
+ public void reset() {
+ totWidth.value = 0;
+ nonNullCount.value = 0;
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BitCntDupsFunction implements DrillAggFunc {
+ @Param
+ BitHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBitCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableBitHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntCntDupsFunction implements DrillAggFunc {
+ @Param
+ IntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class BigIntCntDupsFunction implements DrillAggFunc {
+ @Param
+ BigIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableBigIntCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableBigIntHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float4CntDupsFunction implements DrillAggFunc {
+ @Param
+ Float4Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat4CntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableFloat4Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Float8CntDupsFunction implements DrillAggFunc {
+ @Param
+ Float8Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableFloat8CntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableFloat8Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal9CntDupsFunction implements DrillAggFunc {
+ @Param
+ Decimal9Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal9CntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableDecimal9Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Decimal18CntDupsFunction implements DrillAggFunc {
+ @Param
+ Decimal18Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDecimal18CntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableDecimal18Holder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
}
}
}
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
}
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class DateCntDupsFunction implements DrillAggFunc {
+ @Param
+ DateHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableDateCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableDateHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeCntDupsFunction implements DrillAggFunc {
+ @Param
+ TimeHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableTimeHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class TimeStampCntDupsFunction implements DrillAggFunc {
+ @Param
+ TimeStampHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableTimeStampCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableTimeStampHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (!filter.isPresent(String.valueOf(in.value))) {
+ filter.add(String.valueOf(in.value));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class IntervalCntDupsFunction implements DrillAggFunc {
+ @Param
+ IntervalHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Workspace
+ ObjectHolder interval;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ interval = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ interval.obj = new int[3];
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null
+ && interval.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ ((int[])interval.obj)[0] = in.days;
+ ((int[])interval.obj)[1] = in.months;
+ ((int[])interval.obj)[2] = in.milliseconds;
+ if (!filter.isPresent(String.valueOf(interval.obj))) {
+ filter.add(String.valueOf(interval.obj));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ interval.obj = new int[3];
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableIntervalCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableIntervalHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Workspace
+ ObjectHolder interval;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ interval = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ interval.obj = new int[3];
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ if (interval.obj != null) {
+ ((int[]) interval.obj)[0] = in.days;
+ ((int[]) interval.obj)[1] = in.months;
+ ((int[]) interval.obj)[2] = in.milliseconds;
+ if (!filter.isPresent(String.valueOf(interval.obj))) {
+ filter.add(String.valueOf(interval.obj));
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ interval.obj = new int[3];
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarCharCntDupsFunction implements DrillAggFunc {
+ @Param
+ VarCharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarCharCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableVarCharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+ in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class Var16CharCntDupsFunction implements DrillAggFunc {
+ @Param
+ Var16CharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+ (in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVar16CharCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableVar16CharHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+ (in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class VarBinaryCntDupsFunction implements DrillAggFunc {
+ @Param
+ VarBinaryHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+ (in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
+
+ @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class NullableVarBinaryCntDupsFunction implements DrillAggFunc {
+ @Param
+ NullableVarBinaryHolder in;
+ @Workspace
+ ObjectHolder work;
+ @Workspace BigIntHolder dups;
+ @Output
+ NullableBigIntHolder out;
+ @Inject OptionManager options;
+ @Workspace IntHolder ndvBloomFilterElts;
+ @Workspace IntHolder ndvBloomFilterFPProb;
+
+ @Override
+ public void setup() {
+ work = new ObjectHolder();
+ dups.value = 0;
+ ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+ ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+
+ @Override
+ public void add() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ if (in.isSet == 1) {
+ byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+ (in.start, in.end, in.buffer).getBytes();
+ if (!filter.isPresent(buf)) {
+ filter.add(buf);
+ } else {
+ dups.value++;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void output() {
+ if (work.obj != null) {
+ com.clearspring.analytics.stream.membership.BloomFilter filter =
+ (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+ out.isSet = 1;
+ out.value = dups.value;
+ } else {
+ out.isSet = 0;
+ }
+ }
+
+ @Override
+ public void reset() {
+ dups.value = 0;
+ work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+ }
+ }
} \ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
index 8005f046a..9d3003110 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
@@ -17,16 +17,15 @@
*/
package org.apache.drill.exec.ops;
+import io.netty.buffer.DrillBuf;
import java.io.IOException;
import java.util.List;
-
import org.apache.drill.exec.exception.ClassTransformationException;
import org.apache.drill.exec.expr.ClassGenerator;
import org.apache.drill.exec.expr.CodeGenerator;
import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
import org.apache.drill.exec.physical.impl.common.CodeGenMemberInjector;
-
-import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.proto.UserBitShared;
/**
* Common implementation for both the test and production versions
@@ -88,4 +87,23 @@ public abstract class BaseFragmentContext implements FragmentContext {
return getBufferManager().getManagedBuffer(size);
}
+ @Override
+ public String getQueryUserName() {
+ return null;
+ }
+
+ @Override
+ public UserBitShared.QueryId getQueryId() {
+ return null;
+ }
+
+ @Override
+ public String getQueryIdString() {
+ return null;
+ }
+
+ @Override
+ public QueryContext.SqlStatementType getSQLStatementType() {
+ return null;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
index a7e0dc23c..1972cb7ba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
@@ -72,4 +72,5 @@ public class ContextInformation {
public String getSessionId() {
return sessionId;
}
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 5125f7201..46f94f01b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -30,13 +30,15 @@ import org.apache.drill.exec.expr.ClassGenerator;
import org.apache.drill.exec.expr.CodeGenerator;
import org.apache.drill.exec.expr.fn.FunctionLookupContext;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
import io.netty.buffer.DrillBuf;
-import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
/**
* Provides the resources required by a non-exchange operator to execute.
@@ -93,13 +95,14 @@ public interface FragmentContext extends UdfUtilities, AutoCloseable {
throws ClassTransformationException, IOException;
/**
- * Generates code for a class given a {@link CodeGenerator}, and returns the
- * specified number of instances of the generated class. (Note that the name
- * is a misnomer, it would be better called
- * <tt>getImplementationInstances</tt>.)
- *
- * @param cg the code generator
- * @return list of instances of the generated class
+ * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
+ * @return query statement type {@link SqlStatementType}, if known.
+ */
+ public SqlStatementType getSQLStatementType();
+
+ /**
+ * Get this node's identity.
+ * @return A DrillbitEndpoint object.
*/
<T> List<T> getImplementationClass(final CodeGenerator<T> cg, final int instanceCount)
throws ClassTransformationException, IOException;
@@ -139,6 +142,16 @@ public interface FragmentContext extends UdfUtilities, AutoCloseable {
BufferAllocator getAllocator();
+ /**
+ * @return ID {@link java.util.UUID} of the current query
+ */
+ public QueryId getQueryId();
+
+ /**
+ * @return The string representation of the ID {@link java.util.UUID} of the current query
+ */
+ public String getQueryIdString();
+
OperatorContext newOperatorContext(PhysicalOperator popConfig);
OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
index b740c927d..afbcd1c05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.ops;
+import io.netty.buffer.DrillBuf;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@@ -40,6 +41,7 @@ import org.apache.drill.exec.exception.OutOfMemoryException;
import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
import org.apache.drill.exec.expr.holders.ValueHolder;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -48,6 +50,7 @@ import org.apache.drill.exec.proto.BitControl.PlanFragment;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.rpc.RpcOutcomeListener;
@@ -65,14 +68,12 @@ import org.apache.drill.exec.store.SchemaConfig;
import org.apache.drill.exec.testing.ExecutionControls;
import org.apache.drill.exec.util.ImpersonationUtil;
import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
import org.apache.drill.shaded.guava.com.google.common.base.Function;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
-import io.netty.buffer.DrillBuf;
-import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
-
/**
* <p>
* This is the core Context which implements all the Context interfaces:
@@ -513,6 +514,12 @@ public class FragmentContextImpl extends BaseFragmentContext implements Executor
}
@Override
+ public QueryId getQueryId() { return fragment.getHandle().getQueryId();}
+
+ @Override
+ public String getQueryIdString() { return QueryIdHelper.getQueryId(getQueryId()); }
+
+ @Override
public boolean isImpersonationEnabled() {
// TODO(DRILL-2097): Until SimpleRootExec tests are removed, we need to consider impersonation disabled if there is
// no config
@@ -606,4 +613,11 @@ public class FragmentContextImpl extends BaseFragmentContext implements Executor
}
}
}
+
+ @Override
+ public SqlStatementType getSQLStatementType() {
+ Preconditions.checkNotNull(queryContext, "Statement type is only valid for root fragment."
+ + " Calling from non-root fragment");
+ return queryContext.getSQLStatementType();
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index d7706366a..03ed9fc27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -61,6 +61,9 @@ import io.netty.buffer.DrillBuf;
// in fragment contexts
public class QueryContext implements AutoCloseable, OptimizerRulesContext, SchemaConfigInfoProvider {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
+ public enum SqlStatementType {OTHER, ANALYZE, CTAS, EXPLAIN, DESCRIBE_TABLE, DESCRIBE_SCHEMA, REFRESH, SELECT, SETOPTION};
+
private final DrillbitContext drillbitContext;
private final UserSession session;
private final QueryId queryId;
@@ -76,6 +79,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
private final SchemaTreeProvider schemaTreeProvider;
/** Stores constants and their holders by type */
private final Map<String, Map<MinorType, ValueHolder>> constantValueHolderCache;
+ private SqlStatementType stmtType;
/*
* Flag to indicate if close has been called, after calling close the first
@@ -102,6 +106,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
}
queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaPath(), session.getSessionId());
+
contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
allocator = drillbitContext.getAllocator().newChildAllocator(
@@ -112,6 +117,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
viewExpansionContext = new ViewExpansionContext(this);
schemaTreeProvider = new SchemaTreeProvider(drillbitContext);
constantValueHolderCache = Maps.newHashMap();
+ stmtType = null;
}
@Override
@@ -260,6 +266,8 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
* Re-creates drill operator table to refresh functions list from local function registry.
*/
public void reloadDrillOperatorTable() {
+ // This is re-trying the query plan on failure so qualifies to reset the SQL statement.
+ clearSQLStatementType();
table = new DrillOperatorTable(
drillbitContext.getFunctionImplementationRegistry(),
drillbitContext.getOptionManager());
@@ -288,10 +296,6 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
return new PartitionExplorerImpl(getRootSchema());
}
- public DrillbitContext getDrillbitContext() {
- return drillbitContext;
- }
-
@Override
public ValueHolder getConstantValueHolder(String value, MinorType type, Function<DrillBuf, ValueHolder> holderInitializer) {
if (!constantValueHolderCache.containsKey(value)) {
@@ -325,4 +329,30 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
closed = true;
}
}
+
+ /**
+ * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
+ */
+ public void setSQLStatementType(SqlStatementType stmtType) {
+ if (this.stmtType == null) {
+ this.stmtType = stmtType;
+ } else {
+ throw new IllegalStateException(String.format("SQL Statement type is already set to %s", this.stmtType));
+ }
+ }
+
+ /*
+ * Clears the type {@link SqlStatementType} of the statement. Ideally we should not clear the statement type
+ * so this should never be exposed outside the QueryContext
+ */
+ private void clearSQLStatementType() {
+ this.stmtType = null;
+ }
+
+ /**
+ * @return Get the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
+ */
+ public SqlStatementType getSQLStatementType() {
+ return stmtType;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
index b1b2bec14..96b91da85 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
@@ -17,15 +17,14 @@
*/
package org.apache.drill.exec.ops;
-import org.apache.drill.shaded.guava.com.google.common.base.Function;
+import io.netty.buffer.DrillBuf;
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.store.PartitionExplorer;
-
+import org.apache.drill.shaded.guava.com.google.common.base.Function;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
-import io.netty.buffer.DrillBuf;
-
/**
* Defines the query state and shared resources available to UDFs through
* injectables. For use in a function, include a {@link javax.inject.Inject}
@@ -41,6 +40,7 @@ public interface UdfUtilities {
.put(DrillBuf.class, "getManagedBuffer")
.put(PartitionExplorer.class, "getPartitionExplorer")
.put(ContextInformation.class, "getContextInformation")
+ .put(OptionManager.class, "getOptions")
.build();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
index 42e4bb9ff..403492c4b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
@@ -46,8 +46,6 @@ public abstract class AbstractDbGroupScan extends AbstractGroupScan implements D
public abstract StoragePluginConfig getStorageConfig();
- public abstract List<SchemaPath> getColumns();
-
@Override
public boolean supportsSecondaryIndex() {
return false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index a4eb36973..7e2623a19 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -137,6 +137,11 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
}
@Override
+ public List<SchemaPath> getColumns() {
+ return GroupScan.ALL_COLUMNS;
+ }
+
+ @Override
public List<SchemaPath> getPartitionColumns() {
return Lists.newArrayList();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 85d2a2ba0..212581d57 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.physical.config.Screen;
import org.apache.drill.exec.physical.config.SingleSender;
import org.apache.drill.exec.physical.config.Sort;
import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
import org.apache.drill.exec.physical.config.StreamingAggregate;
import org.apache.drill.exec.physical.config.Trace;
import org.apache.drill.exec.physical.config.UnionAll;
@@ -104,6 +105,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
}
@Override
+ public T visitStatisticsMerge(StatisticsMerge agg, X value) throws E {
+ return visitOp(agg, value);
+ }
+
+ @Override
public T visitHashAggregate(HashAggregate agg, X value) throws E {
return visitOp(agg, value);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index 263ef052f..e42ae2dc8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -17,9 +17,9 @@
*/
package org.apache.drill.exec.physical.base;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import java.util.Collection;
import java.util.List;
-
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.SchemaPath;
@@ -28,10 +28,8 @@ import org.apache.drill.exec.ops.UdfUtilities;
import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
/**
* A GroupScan operator represents all data which will be scanned by a given physical
@@ -110,6 +108,12 @@ public interface GroupScan extends Scan, HasAffinity{
boolean supportsPartitionFilterPushdown();
/**
+ * Returns a list of columns scanned by this group scan
+ *
+ */
+ List<SchemaPath> getColumns();
+
+ /**
* Returns a list of columns that can be used for partition pruning
*
*/
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
index 5065da55a..f26e78deb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
@@ -20,10 +20,16 @@ package org.apache.drill.exec.physical.base;
import java.util.List;
import java.util.Set;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.MinorFragmentEndpoint;
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
+import org.apache.drill.exec.record.VectorAccessible;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
public class PhysicalOperatorUtil {
@@ -56,4 +62,22 @@ public class PhysicalOperatorUtil {
return destinations;
}
+
+ /**
+ * Helper method tp materialize the given logical expression using the ExpressionTreeMaterializer
+ * @param expr Logical expression to materialize
+ * @param incoming Incoming record batch
+ * @param context Fragment context
+ */
+ public static LogicalExpression materializeExpression(LogicalExpression expr,
+ VectorAccessible incoming, FragmentContext context) throws SchemaChangeException {
+ ErrorCollector collector = new ErrorCollectorImpl();
+ LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
+ context.getFunctionRegistry());
+ if (collector.hasErrors()) {
+ throw new SchemaChangeException("Failure while materializing expression. "
+ + collector.toErrorString());
+ }
+ return mle;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index a21f578d5..1a63538ed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.physical.config.Screen;
import org.apache.drill.exec.physical.config.SingleSender;
import org.apache.drill.exec.physical.config.Sort;
import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
import org.apache.drill.exec.physical.config.StreamingAggregate;
import org.apache.drill.exec.physical.config.Trace;
import org.apache.drill.exec.physical.config.UnionAll;
@@ -71,6 +72,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
public RETURN visitStatisticsAggregate(StatisticsAggregate agg, EXTRA value) throws EXCEP;
+ public RETURN visitStatisticsMerge(StatisticsMerge agg, EXTRA value) throws EXCEP;
public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
public RETURN visitWriter(Writer op, EXTRA value) throws EXCEP;
public RETURN visitUnpivot(UnpivotMaps op, EXTRA value) throws EXCEP;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
index d227b04aa..999c417fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
@@ -36,6 +36,12 @@ public class SchemalessScan extends AbstractFileGroupScan implements SubScan {
@JsonCreator
public SchemalessScan(@JsonProperty("userName") String userName,
+ @JsonProperty("selectionRoot") String selectionRoot,
+ @JsonProperty("columns") List<SchemaPath> columns) {
+ this(userName, selectionRoot);
+ }
+
+ public SchemalessScan(@JsonProperty("userName") String userName,
@JsonProperty("selectionRoot") String selectionRoot) {
super(userName);
this.selectionRoot = selectionRoot;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
index 95ee6bff7..1fa30abde 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -31,8 +31,6 @@ import java.util.List;
@JsonTypeName("statistics-aggregate")
public class StatisticsAggregate extends StreamingAggregate {
- // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
-
private final List<String> functions;
@JsonCreator
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
new file mode 100644
index 000000000..157066019
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
@@ -0,0 +1,69 @@
+/*
+ * 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.drill.exec.physical.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import java.util.Map;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+
+@JsonTypeName("statistics-merge")
+public class StatisticsMerge extends AbstractSingle {
+
+ private final Map<String, String> functions;
+ private final double samplePercent;
+
+ @JsonCreator
+ public StatisticsMerge(
+ @JsonProperty("child") PhysicalOperator child,
+ @JsonProperty("functions") Map<String, String> functions,
+ @JsonProperty("samplePercent") double samplePercent) {
+ super(child);
+ this.functions = ImmutableMap.copyOf(functions);
+ this.samplePercent = samplePercent;
+ }
+
+ public Map<String, String> getFunctions() {
+ return functions;
+ }
+
+ public double getSamplePercent() {
+ return samplePercent;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value)
+ throws E {
+ return physicalVisitor.visitStatisticsMerge(this, value);
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new StatisticsMerge(child, functions, samplePercent);
+ }
+
+ @Override
+ public int getOperatorType() {
+ return CoreOperatorType.STATISTICS_MERGE_VALUE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
index ac71b11ce..4a561bf4b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
new file mode 100644
index 000000000..6cad07187
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
@@ -0,0 +1,214 @@
+/*
+ * 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.drill.exec.physical.impl;
+
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.Writer;
+import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.StatisticsRecordWriter;
+import org.apache.drill.exec.store.StatisticsRecordWriterImpl;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import java.io.IOException;
+
+public class StatisticsWriterRecordBatch extends AbstractRecordBatch<Writer> {
+
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsWriterRecordBatch.class);
+
+ private StatisticsRecordWriterImpl statsRecordWriterImpl;
+ private StatisticsRecordWriter recordWriter;
+ private long counter = 0;
+ private final RecordBatch incoming;
+ private boolean processed = false;
+ private final String fragmentUniqueId;
+ private BatchSchema schema;
+
+ public StatisticsWriterRecordBatch(Writer writer, RecordBatch incoming, FragmentContext context,
+ StatisticsRecordWriter recordWriter) throws OutOfMemoryException {
+ super(writer, context, false);
+ this.incoming = incoming;
+
+ final ExecProtos.FragmentHandle handle = context.getHandle();
+ fragmentUniqueId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
+ this.recordWriter = recordWriter;
+ }
+
+ @Override
+ public int getRecordCount() {
+ return container.getRecordCount();
+ }
+
+ @Override
+ protected void killIncoming(boolean sendUpstream) {
+ incoming.kill(sendUpstream);
+ }
+
+ @Override
+ public BatchSchema getSchema() {
+ return schema;
+ }
+
+ @Override
+ public void dump() {
+ logger.error("StatisticsWriterRecordBatch[container={}, popConfig={}, counter={}, fragmentUniqueId={}, schema={}]",
+ container, popConfig, counter, fragmentUniqueId, schema);
+ }
+
+ @Override
+ public IterOutcome innerNext() {
+ if (processed) {
+ // if the upstream record batch is already processed and next() is called by
+ // downstream then return NONE to indicate completion
+ return IterOutcome.NONE;
+ }
+ // process the complete upstream in one next() call
+ IterOutcome upstream;
+ try {
+ do {
+ upstream = next(incoming);
+
+ switch(upstream) {
+ case OUT_OF_MEMORY:
+ case STOP:
+ return upstream;
+
+ case NOT_YET:
+ case NONE:
+ break;
+
+ case OK_NEW_SCHEMA:
+ setupNewSchema();
+ // $FALL-THROUGH$
+ case OK:
+ counter += statsRecordWriterImpl.writeStatistics(incoming.getRecordCount());
+ logger.debug("Total records written so far: {}", counter);
+
+ for(final VectorWrapper<?> v : incoming) {
+ v.getValueVector().clear();
+ }
+ break;
+
+ default:
+ throw new UnsupportedOperationException();
+ }
+ } while(upstream != IterOutcome.NONE);
+ // Flush blocking writers now
+ statsRecordWriterImpl.flushBlockingWriter();
+ } catch(IOException ex) {
+ logger.error("Failure during query", ex);
+ kill(false);
+ context.getExecutorState().fail(ex);
+ return IterOutcome.STOP;
+ }
+
+ addOutputContainerData();
+ processed = true;
+
+ closeWriter();
+
+ return IterOutcome.OK_NEW_SCHEMA;
+ }
+
+ private void addOutputContainerData() {
+ final VarCharVector fragmentIdVector = (VarCharVector) container.getValueAccessorById(
+ VarCharVector.class,
+ container.getValueVectorId(SchemaPath.getSimplePath("Fragment")).getFieldIds())
+ .getValueVector();
+ AllocationHelper.allocate(fragmentIdVector, 1, 50);
+ final BigIntVector summaryVector = (BigIntVector) container.getValueAccessorById(BigIntVector.class,
+ container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds())
+ .getValueVector();
+ AllocationHelper.allocate(summaryVector, 1, 8);
+ fragmentIdVector.getMutator().setSafe(0, fragmentUniqueId.getBytes());
+ fragmentIdVector.getMutator().setValueCount(1);
+ summaryVector.getMutator().setSafe(0, counter);
+ summaryVector.getMutator().setValueCount(1);
+
+ container.setRecordCount(1);
+ }
+
+ protected void setupNewSchema() throws IOException {
+ try {
+ // update the schema in RecordWriter
+ stats.startSetup();
+ recordWriter.updateSchema(incoming);
+ // Create two vectors for:
+ // 1. Fragment unique id.
+ // 2. Summary: currently contains number of records written.
+ final MaterializedField fragmentIdField =
+ MaterializedField.create("Fragment", Types.required(TypeProtos.MinorType.VARCHAR));
+ final MaterializedField summaryField =
+ MaterializedField.create("Number of records written",
+ Types.required(TypeProtos.MinorType.BIGINT));
+
+ container.addOrGet(fragmentIdField);
+ container.addOrGet(summaryField);
+ container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+ } finally {
+ stats.stopSetup();
+ }
+
+ statsRecordWriterImpl = new StatisticsRecordWriterImpl(incoming, recordWriter);
+ container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+ schema = container.getSchema();
+ }
+
+ /** Clean up needs to be performed before closing writer. Partially written data will be removed. */
+ private void closeWriter() {
+ if (recordWriter == null) {
+ return;
+ }
+
+ try {
+ //Perform any cleanup prior to closing the writer
+ recordWriter.cleanup();
+ } catch(IOException ex) {
+ context.getExecutorState().fail(ex);
+ } finally {
+ try {
+ if (!processed) {
+ recordWriter.abort();
+ }
+ } catch (IOException e) {
+ logger.error("Abort failed. There could be leftover output files.", e);
+ } finally {
+ recordWriter = null;
+ }
+ }
+ }
+
+ @Override
+ public void close() {
+ closeWriter();
+ super.close();
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index 3a8485a24..7c13f7269 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -188,6 +188,9 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
}
try {
+ //Perform any post processing tasks prior to cleaning up the writer
+ recordWriter.postProcessing();
+ //Perform any cleanup prior to closing the writer
recordWriter.cleanup();
} catch(IOException ex) {
context.getExecutorState().fail(ex);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
index 396fd3692..ebdd4a4a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
@@ -32,7 +32,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InternalBatch.class);
private final VectorContainer container;
- private final RecordBatch incoming;
+ private final FragmentContext context;
private final BatchSchema schema;
private final SelectionVector2 sv2;
private final SelectionVector4 sv4;
@@ -56,7 +56,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
this.sv2 = null;
}
this.schema = incoming.getSchema();
- this.incoming = incoming;
+ this.context = incoming.getContext();
this.container = VectorContainer.getTransferClone(incoming, ignoreWrappers, oContext);
}
@@ -92,6 +92,6 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
}
public FragmentContext getContext() {
- return incoming.getContext();
+ return context;
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
deleted file mode 100644
index ea5a7b3e4..000000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/**
- * 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.physical.impl.aggregate;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import com.sun.codemodel.JExpr;
-import org.apache.drill.common.expression.ErrorCollector;
-import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.FunctionCallFactory;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.ValueExpressions;
-import org.apache.drill.exec.exception.ClassTransformationException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.ClassGenerator;
-import org.apache.drill.exec.expr.CodeGenerator;
-import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.StatisticsAggregate;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.TypedFieldId;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.complex.FieldIdUtil;
-import org.apache.drill.exec.vector.complex.MapVector;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * TODO: This needs cleanup. Currently the key values are constants and we compare the constants for
- * every record. Seems unnecessary.
- *
- * Example input and output:
- * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
- * Schema of output:
- * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
- * "computed" : BIGINT - What time is it computed?
- * "columns" : MAP - Column names
- * "region_id" : VARCHAR
- * "sales_city" : VARCHAR
- * "cnt" : VARCHAR
- * "statscount" : MAP
- * "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id in incoming batch
- * "sales_city" : BIGINT - statscount(sales_city)
- * "cnt" : BIGINT - statscount(cnt)
- * "nonnullstatcount" : MAP
- * "region_id" : BIGINT - nonnullstatcount(region_id)
- * "sales_city" : BIGINT - nonnullstatcount(sales_city)
- * "cnt" : BIGINT - nonnullstatcount(cnt)
- * .... another map for next stats function ....
- */
-public class StatisticsAggBatch extends StreamingAggBatch {
- private List<String> functions;
- private int schema = 0;
-
- public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming, FragmentContext context)
- throws OutOfMemoryException {
- super(popConfig, incoming, context);
- this.functions = popConfig.getFunctions();
- }
-
- private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
- throws SchemaChangeException {
- ErrorCollector collector = new ErrorCollectorImpl();
-
- LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
- MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
- ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
-
- keyExprs.add(mle);
- keyOutputIds.add(container.add(vector));
-
- if (collector.hasErrors()) {
- throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
- }
- }
-
- private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
- throws SchemaChangeException {
- ErrorCollector collector = new ErrorCollectorImpl();
-
- LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
- Class<? extends ValueVector> vvc =
- TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
-
- ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
-
- TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
- assert pfid.getFieldIds().length == 1;
- TypedFieldId.Builder builder = TypedFieldId.newBuilder();
- builder.addId(pfid.getFieldIds()[0]);
- TypedFieldId id =
- FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
- SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
-
- keyExprs.add(mle);
- keyOutputIds.add(id);
-
- if (collector.hasErrors()) {
- throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
- }
- }
-
- private void addMapVector(String name, MapVector parent, LogicalExpression expr, List<LogicalExpression> valueExprs)
- throws SchemaChangeException {
- ErrorCollector collector = new ErrorCollectorImpl();
-
- LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
- Class<? extends ValueVector> vvc =
- TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
- ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
-
- TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
- assert pfid.getFieldIds().length == 1;
- TypedFieldId.Builder builder = TypedFieldId.newBuilder();
- builder.addId(pfid.getFieldIds()[0]);
- TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
- SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
-
- valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
-
- if (collector.hasErrors()) {
- throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
- }
- }
-
- private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs, List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
- throws SchemaChangeException, ClassTransformationException, IOException {
- ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
-
- LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
- LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
- TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
-
- keyExprs.toArray(keyExprsArray);
- valueExprs.toArray(valueExprsArray);
- keyOutputIds.toArray(keyOutputIdsArray);
-
- setupIsSame(cg, keyExprsArray);
- setupIsSameApart(cg, keyExprsArray);
- addRecordValues(cg, valueExprsArray);
- outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
- outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
-
- cg.getBlock("resetValues")._return(JExpr.TRUE);
- getIndex(cg);
-
- container.buildSchema(SelectionVectorMode.NONE);
- StreamingAggregator agg = context.getImplementationClass(cg);
- agg.setup(oContext, incoming, this, ValueVector.MAX_ROW_COUNT);
- return agg;
- }
-
- protected StreamingAggregator createAggregatorInternal()
- throws SchemaChangeException, ClassTransformationException, IOException {
- container.clear();
-
- List<LogicalExpression> keyExprs = Lists.newArrayList();
- List<LogicalExpression> valueExprs = Lists.newArrayList();
- List<TypedFieldId> keyOutputIds = Lists.newArrayList();
-
- createKeyColumn("schema",
- ValueExpressions.getBigInt(schema++),
- keyExprs,
- keyOutputIds
- );
- createKeyColumn("computed",
- ValueExpressions.getBigInt(System.currentTimeMillis()),
- keyExprs,
- keyOutputIds
- );
-
- MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
- container.add(cparent);
- for (MaterializedField mf : incoming.getSchema()) {
- createNestedKeyColumn(
- cparent,
- mf.getName(),
- ValueExpressions.getChar(mf.getName(), 0),
- keyExprs,
- keyOutputIds
- );
- }
-
- for (String func : functions) {
- MapVector parent = new MapVector(func, oContext.getAllocator(), null);
- container.add(parent);
-
- for (MaterializedField mf : incoming.getSchema()) {
- List<LogicalExpression> args = Lists.newArrayList();
- args.add(SchemaPath.getSimplePath(mf.getName()));
- LogicalExpression call = FunctionCallFactory.createExpression(func, args);
-
- addMapVector(mf.getName(), parent, call, valueExprs);
- }
- }
-
- return codegenAggregator(keyExprs, valueExprs, keyOutputIds);
- }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index e1e43bd10..757001609 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -447,7 +447,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
complexWriters.add(writer);
}
- private StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
+ protected StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
cg.getCodeGenerator().plainJavaCapable(true);
// Uncomment out this line to debug the generated code.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
new file mode 100644
index 000000000..a4ee74fa4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
@@ -0,0 +1,48 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
+ protected String name;
+ protected String inputName;
+ protected double samplePercent;
+ protected State state;
+
+ public void initialize(String name, String inputName, double samplePercent) {
+ this.name = name;
+ this.inputName = inputName;
+ this.samplePercent = samplePercent;
+ }
+
+ @Override
+ public abstract void initialize(String inputName, double samplePercent);
+
+ @Override
+ public abstract String getName();
+
+ @Override
+ public abstract String getInput();
+
+ @Override
+ public abstract void merge(MapVector input);
+
+ @Override
+ public abstract void setOutput(MapVector output);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
new file mode 100644
index 000000000..6a9660d84
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
@@ -0,0 +1,150 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.MinorType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
+ private Map<String, Double> sumHolder;
+ ColTypeMergedStatistic types;
+ NNRowCountMergedStatistic nonNullStatCounts;
+ RowCountMergedStatistic statCounts;
+
+ public AvgWidthMergedStatistic () {
+ this.sumHolder = new HashMap<>();
+ types = null;
+ nonNullStatCounts = null;
+ statCounts = null;
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.AVG_WIDTH, inputName, samplePercent);
+ state = State.CONFIG;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ NullableFloat8Vector fv = (NullableFloat8Vector) vv;
+ NullableFloat8Vector.Accessor accessor = fv.getAccessor();
+ String colName = vv.getField().getName();
+ double sum = 0;
+ if (sumHolder.get(colName) != null) {
+ sum = sumHolder.get(colName);
+ }
+ if (!accessor.isNull(0)) {
+ sum += accessor.get(0);
+ sumHolder.put(colName, sum);
+ }
+ }
+ }
+
+ public double getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(
+ String.format("Statistic `%s` has not completed merging statistics", name));
+ }
+ return sumHolder.get(colName)/((samplePercent/100.0) *getRowCount(colName));
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ // Dependencies have been configured correctly
+ assert (state == State.MERGE);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
+ vv.allocateNewSafe();
+ // For variable-length columns, we divide by non-null rows since NULL values do not
+ // take up space. For fixed-length columns NULL values take up space.
+ if (sumHolder.get(colName) != null
+ && getRowCount(colName) > 0) {
+ vv.getMutator().setSafe(0, sumHolder.get(colName)/((samplePercent/100.0) *getRowCount(colName)));
+ } else {
+ vv.getMutator().setNull(0);
+ }
+ }
+ state = State.COMPLETE;
+ }
+
+ public void configure(List<MergedStatistic> statisticList) {
+ assert (state == State.CONFIG);
+ for (MergedStatistic statistic : statisticList) {
+ if (statistic.getName().equals(Statistic.COLTYPE)) {
+ types = (ColTypeMergedStatistic) statistic;
+ } else if (statistic.getName().equals(Statistic.ROWCOUNT)) {
+ statCounts = (RowCountMergedStatistic) statistic;
+ } else if (statistic.getName().equals(Statistic.NNROWCOUNT)) {
+ nonNullStatCounts = (NNRowCountMergedStatistic) statistic;
+ }
+ }
+ assert (types != null && statCounts != null && nonNullStatCounts != null);
+ // Now config complete - moving to MERGE state
+ state = State.MERGE;
+ }
+
+ private long getRowCount(String colName) {
+ byte[] typeAsBytes = types.getStat(colName);
+ int type = -1;
+ ObjectMapper mapper = new ObjectMapper();
+ SimpleModule deModule = new SimpleModule("StatisticsSerDeModeule") //
+ .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+ mapper.registerModule(deModule);
+ try {
+ type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
+ } catch (IOException ex) {
+ //Ignore exception
+ }
+ // If variable length type - then use the nonNullCount. Otherwise, use the Count,
+ // since even NULL values take up the same space.
+ if (type == MinorType.VAR16CHAR.getNumber()
+ || type == MinorType.VARCHAR.getNumber()
+ || type == MinorType.VARBINARY.getNumber()) {
+ return nonNullStatCounts.getStat(colName);
+ } else {
+ return statCounts.getStat(colName);
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
new file mode 100644
index 000000000..2d7194dd7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
@@ -0,0 +1,103 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class CntDupsMergedStatistic extends AbstractMergedStatistic {
+ private Map<String, Long> sumHolder;
+
+ public CntDupsMergedStatistic () {
+ this.sumHolder = new HashMap<>();
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.SUM_DUPS, inputName, samplePercent);
+ state = State.CONFIG;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ NullableBigIntVector fv = (NullableBigIntVector) vv;
+ NullableBigIntVector.Accessor accessor = fv.getAccessor();
+ String colName = vv.getField().getName();
+ long sum = 0;
+ if (sumHolder.get(colName) != null) {
+ sum = sumHolder.get(colName);
+ }
+ if (!accessor.isNull(0)) {
+ sum += accessor.get(0);
+ sumHolder.put(colName, sum);
+ }
+ }
+ }
+
+ public double getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(
+ String.format("Statistic `%s` has not completed merging statistics", name));
+ }
+ return sumHolder.get(colName);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ // Dependencies have been configured correctly
+ assert (state == State.MERGE);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+ vv.allocateNewSafe();
+ // For variable-length columns, we divide by non-null rows since NULL values do not
+ // take up space. For fixed-length columns NULL values take up space.
+ if (sumHolder.get(colName) != null) {
+ vv.getMutator().setSafe(0, sumHolder.get(colName));
+ }
+ }
+ state = State.COMPLETE;
+ }
+
+ public void configure(List<MergedStatistic> statisticList) {
+ assert (state == State.CONFIG);
+ // Now config complete - moving to MERGE state
+ state = State.MERGE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
new file mode 100644
index 000000000..296d055c4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
@@ -0,0 +1,87 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class ColTypeMergedStatistic extends AbstractMergedStatistic {
+ private Map<String, byte[]> typeHolder;
+
+ public ColTypeMergedStatistic () {
+ typeHolder = new HashMap<>();
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.COLTYPE, inputName, samplePercent);
+ state = State.MERGE;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ String colName = vv.getField().getName();
+ if (typeHolder.get(colName) == null) {
+ VarCharVector iv = (VarCharVector) vv;
+ VarCharVector.Accessor accessor = iv.getAccessor();
+ typeHolder.put(colName, accessor.get(0));
+ }
+ }
+ }
+
+ public byte[] getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+ name));
+ }
+ return typeHolder.get(colName);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ VarCharVector vv = (VarCharVector) outMapCol;
+ vv.allocateNewSafe();
+ // Set column name in ValueVector
+ vv.getMutator().setSafe(0, typeHolder.get(colName));
+ }
+ // Now moving to COMPLETE state
+ state = State.COMPLETE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
new file mode 100644
index 000000000..90d3b271f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
@@ -0,0 +1,68 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class ColumnMergedStatistic extends AbstractMergedStatistic {
+
+ public ColumnMergedStatistic () {
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.COLNAME, inputName, samplePercent);
+ state = State.MERGE;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ MapVector outputMap = (MapVector) output;
+ for (ValueVector outMapCol : outputMap) {
+ String colName = outMapCol.getField().getName();
+ VarCharVector vv = (VarCharVector) outMapCol;
+ vv.allocateNewSafe();
+ // Set column name in ValueVector
+ vv.getMutator().setSafe(0, colName.getBytes(), 0, colName.length());
+ }
+ // Now moving to COMPLETE state
+ state = State.COMPLETE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
new file mode 100644
index 000000000..68854f790
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
@@ -0,0 +1,130 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+// Library implementing HLL algorithm to derive approximate #distinct values(NDV). Please refer:
+// 'HyperLogLog: the analysis of a near-optimal cardinality estimation algorithm.' Flajolet et. al.
+import com.clearspring.analytics.stream.cardinality.HyperLogLog;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class HLLMergedStatistic extends AbstractMergedStatistic {
+ private Map<String, HyperLogLog> hllHolder;
+ private long accuracy;
+
+ public HLLMergedStatistic () {
+ this.hllHolder = new HashMap<>();
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.HLL_MERGE, inputName, samplePercent);
+ state = State.CONFIG;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ String colName = vv.getField().getName();
+ HyperLogLog colHLLHolder = null;
+ if (hllHolder.get(colName) != null) {
+ colHLLHolder = hllHolder.get(colName);
+ }
+ NullableVarBinaryVector hllVector = (NullableVarBinaryVector) vv;
+ NullableVarBinaryVector.Accessor accessor = hllVector.getAccessor();
+
+ try {
+ if (!accessor.isNull(0)) {
+ ByteArrayInputStream bais = new ByteArrayInputStream(accessor.get(0), 0, vv.getBufferSize());
+ HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
+ if (colHLLHolder != null) {
+ colHLLHolder.addAll(other);
+ hllHolder.put(colName, colHLLHolder);
+ } else {
+ hllHolder.put(colName, other);
+ }
+ }
+ } catch (Exception ex) {
+ //TODO: Catch IOException/CardinalityMergeException
+ //TODO: logger
+ }
+ }
+ }
+
+ public HyperLogLog getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+ name));
+ }
+ return hllHolder.get(colName);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ // Dependencies have been configured correctly
+ assert (state == State.MERGE);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ HyperLogLog colHLLHolder = hllHolder.get(colName);
+ NullableVarBinaryVector vv = (NullableVarBinaryVector) outMapCol;
+ vv.allocateNewSafe();
+ try {
+ if (colHLLHolder != null) {
+ vv.getMutator().setSafe(0, colHLLHolder.getBytes(),
+ 0, colHLLHolder.getBytes().length);
+ } else {
+ vv.getMutator().setNull(0);
+ }
+ } catch (IOException ex) {
+ // TODO: logger
+ }
+ }
+ state = State.COMPLETE;
+ }
+
+ public void configure(OptionManager optionsManager) {
+ assert (state == State.CONFIG);
+ accuracy = optionsManager.getLong(ExecConstants.HLL_ACCURACY);
+ // Now config complete - moving to MERGE state
+ state = State.MERGE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java
new file mode 100644
index 000000000..26d30f506
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java
@@ -0,0 +1,62 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import org.apache.drill.exec.vector.complex.MapVector;
+
+/*
+ * Interface for implementing a merged statistic. A merged statistic can merge
+ * the input statistics to get the overall value. e.g. `rowcount` merged statistic
+ * should merge all `rowcount` input statistic and return the overall `rowcount`.
+ * Given `rowcount`s 10 and 20, the `rowcount` merged statistic will return 30.
+ */
+public interface MergedStatistic {
+
+ /** Initialize the merged statistic
+ *
+ * @param inputName - the input {@link StatisticsAggBatch} statistic for this merged statistic
+ * @param samplePercent - the sample percentage used for extrapolation post merge phase
+ */
+ void initialize(String inputName, double samplePercent);
+
+ /** Gets the name of the merged statistic
+ *
+ * @return - name of this merged statistic
+ */
+ String getName();
+
+ /**
+ * Gets the name of the input statistic
+ *
+ * @return - name of the input {@link StatisticsAggBatch} statistic for this merged statistic
+ */
+ String getInput();
+
+ /** Merges the input statistic (incoming value vector) into the existing
+ * merged statistic
+ *
+ * @param input - the input value vector to merge
+ */
+ void merge(MapVector input);
+
+ /** Sets the merged statistic value in the output (outgoing value vector)
+ *
+ * @param output - the output vector where to populate the statistic value
+ */
+ void setOutput(MapVector output);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
new file mode 100644
index 000000000..c9213941f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
@@ -0,0 +1,62 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+
+public class MergedStatisticFactory {
+ private HashMap<String,Class<? extends MergedStatistic>> statsClasses = new HashMap<>( );
+ /*
+ * Creates the appropriate statistics object given the name of the statistics and the input statistic
+ */
+ private static MergedStatisticFactory instance = new MergedStatisticFactory();
+ //Can not instantiate
+ private MergedStatisticFactory() {
+ statsClasses.put(Statistic.COLNAME, ColumnMergedStatistic.class);
+ statsClasses.put(Statistic.COLTYPE, ColTypeMergedStatistic.class);
+ statsClasses.put(Statistic.ROWCOUNT, RowCountMergedStatistic.class);
+ statsClasses.put(Statistic.NNROWCOUNT, NNRowCountMergedStatistic.class);
+ statsClasses.put(Statistic.AVG_WIDTH, AvgWidthMergedStatistic.class);
+ statsClasses.put(Statistic.HLL_MERGE, HLLMergedStatistic.class);
+ statsClasses.put(Statistic.NDV, NDVMergedStatistic.class);
+ statsClasses.put(Statistic.SUM_DUPS, CntDupsMergedStatistic.class);
+ }
+
+ private MergedStatistic newMergedStatistic(String outputStatName)
+ throws InstantiationException, IllegalAccessException {
+ MergedStatistic stat = statsClasses.get(outputStatName).newInstance();
+ return stat;
+ }
+
+ public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName, double samplePercent) {
+ try {
+ MergedStatistic statistic = instance.newMergedStatistic(outputStatName);
+ if (statistic == null) {
+ throw new IllegalArgumentException("No implementation found for " + outputStatName);
+ } else {
+ statistic.initialize(inputStatName, samplePercent);
+ return statistic;
+ }
+ } catch (InstantiationException ex) {
+ throw new IllegalArgumentException("Cannot instantiate class for " + outputStatName);
+ } catch (IllegalAccessException ex) {
+ throw new IllegalArgumentException("Cannot access class for " + outputStatName);
+ }
+ }
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
new file mode 100644
index 000000000..fc87fbd9e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
@@ -0,0 +1,198 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+// Library implementing HLL algorithm to derive approximate #distinct values(NDV). Please refer:
+// 'HyperLogLog: the analysis of a near-optimal cardinality estimation algorithm.' Flajolet et. al.
+import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
+import com.clearspring.analytics.stream.cardinality.HyperLogLog;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.MinorType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class NDVMergedStatistic extends AbstractMergedStatistic {
+ private Map<String, HyperLogLog> hllHolder;
+ ColTypeMergedStatistic types;
+ NNRowCountMergedStatistic nonNullStatCounts;
+ RowCountMergedStatistic statCounts;
+ CntDupsMergedStatistic sumDups;
+
+ public NDVMergedStatistic () {
+ this.hllHolder = new HashMap<>();
+ types = null;
+ nonNullStatCounts = null;
+ statCounts = null;
+ sumDups = null;
+ state = State.INIT;
+ }
+
+ public static class NDVConfiguration {
+ private final OptionManager optionManager;
+ private final List<MergedStatistic> dependencies;
+
+ public NDVConfiguration (OptionManager optionsManager, List<MergedStatistic> statistics) {
+ this.optionManager = optionsManager;
+ this.dependencies = statistics;
+ }
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.NDV, inputName, samplePercent);
+ state = State.CONFIG;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ // Dependencies have been configured correctly
+ assert (state == State.MERGE);
+ for (ValueVector vv : input) {
+ String colName = vv.getField().getName();
+ HyperLogLog colHLLHolder = null;
+ if (hllHolder.get(colName) != null) {
+ colHLLHolder = hllHolder.get(colName);
+ }
+ NullableVarBinaryVector hllVector = (NullableVarBinaryVector) vv;
+ NullableVarBinaryVector.Accessor accessor = hllVector.getAccessor();
+
+ try {
+ if (!accessor.isNull(0)) {
+ ByteArrayInputStream bais = new ByteArrayInputStream(accessor.get(0), 0, vv.getBufferSize());
+ HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
+ if (colHLLHolder != null) {
+ colHLLHolder.addAll(other);
+ hllHolder.put(colName, colHLLHolder);
+ } else {
+ hllHolder.put(colName, other);
+ }
+ }
+ } catch (CardinalityMergeException ex) {
+ throw new IllegalStateException("Failed to merge the NDV statistics");
+ } catch (Exception ex) {
+ throw new IllegalStateException(ex);
+ }
+ }
+ }
+
+ public long getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics", name));
+ }
+ return hllHolder.get(colName).cardinality();
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ // Dependencies have been configured correctly
+ assert (state == State.MERGE);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ HyperLogLog colHLLHolder = hllHolder.get(colName);
+ NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+ vv.allocateNewSafe();
+ if (colHLLHolder != null) {
+ /* Duj1 estimator - Peter J. Haas & Lynne Stokes (1998) Estimating the Number of Classes in a Finite Population,
+ * Journal of the American Statistical Association, 93:444, 1475-1487
+ * n*d / (n - f1 + f1*n/N) where
+ * n - sample rows
+ * N - total rows
+ * d - ndv of sample
+ * f1 - number of singletons
+ * Cap estimate at N
+ */
+ double sampleRows = (samplePercent/100.0)*getRowCount(colName);
+ double sampleSingletons = sampleRows - sumDups.getStat(colName);
+ double estNdv = (sampleRows * colHLLHolder.cardinality()) /
+ (sampleRows - sampleSingletons + sampleSingletons* samplePercent/100.0);
+ estNdv = Math.min(estNdv, 100.0*sampleRows/samplePercent);
+ vv.getMutator().setSafe(0, 1, (long) estNdv);
+ } else {
+ vv.getMutator().setNull(0);
+ }
+ }
+ state = State.COMPLETE;
+ }
+
+ public void configure(NDVConfiguration ndvConfig) {
+ assert (state == State.CONFIG);
+ for (MergedStatistic statistic : ndvConfig.dependencies) {
+ if (statistic.getName().equals(Statistic.COLTYPE)) {
+ types = (ColTypeMergedStatistic) statistic;
+ } else if (statistic.getName().equals(Statistic.ROWCOUNT)) {
+ statCounts = (RowCountMergedStatistic) statistic;
+ } else if (statistic.getName().equals(Statistic.NNROWCOUNT)) {
+ nonNullStatCounts = (NNRowCountMergedStatistic) statistic;
+ } else if (statistic.getName().equals(Statistic.SUM_DUPS)) {
+ sumDups = (CntDupsMergedStatistic) statistic;
+ }
+ }
+ assert (types != null && statCounts != null && nonNullStatCounts != null && sumDups != null);
+ // Now config complete - moving to MERGE state
+ state = State.MERGE;
+ }
+
+ private long getRowCount(String colName) {
+ byte[] typeAsBytes = types.getStat(colName);
+ int type = -1;
+ ObjectMapper mapper = new ObjectMapper();
+ SimpleModule deModule = new SimpleModule("StatisticsSerDeModule") //
+ .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+ mapper.registerModule(deModule);
+ try {
+ type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
+ } catch (IOException ex) {
+ //Ignore exception
+ }
+ // If variable length type - then use the nonNullCount. Otherwise, use the Count,
+ // since even NULL values take up the same space.
+ if (type == MinorType.VAR16CHAR.getNumber()
+ || type == MinorType.VARCHAR.getNumber()
+ || type == MinorType.VARBINARY.getNumber()) {
+ return nonNullStatCounts.getStat(colName);
+ } else {
+ return statCounts.getStat(colName);
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
new file mode 100644
index 000000000..a93050246
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
@@ -0,0 +1,95 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class NNRowCountMergedStatistic extends AbstractMergedStatistic {
+
+ private Map<String, Long> sumHolder;
+
+ public NNRowCountMergedStatistic() {
+ this.sumHolder = new HashMap<>();
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.NNROWCOUNT, inputName, samplePercent);
+ state = State.MERGE;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ String colName = vv.getField().getName();
+ NullableBigIntVector biv = (NullableBigIntVector) vv;
+ NullableBigIntVector.Accessor accessor = biv.getAccessor();
+ long sum = 0;
+ if (sumHolder.get(colName) != null) {
+ sum = sumHolder.get(colName);
+ }
+ if (!accessor.isNull(0)) {
+ sum += accessor.get(0);
+ sumHolder.put(colName, sum);
+ }
+ }
+ }
+
+ public long getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+ name));
+ }
+ return (long)(100.0*sumHolder.get(colName)/ samplePercent);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+ vv.allocateNewSafe();
+ if (sumHolder.get(colName) != null) {
+ vv.getMutator().setSafe(0, (long)(100.0*sumHolder.get(colName)/ samplePercent));
+ } else {
+ vv.getMutator().setNull(0);
+ }
+ }
+ state = State.COMPLETE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
new file mode 100644
index 000000000..99930311b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
@@ -0,0 +1,96 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class RowCountMergedStatistic extends AbstractMergedStatistic {
+
+ private Map<String, Long> sumHolder;
+
+ public RowCountMergedStatistic() {
+ this.sumHolder = new HashMap<>();
+ //No CONFIG state so move directly to MERGE state
+ state = State.INIT;
+ }
+
+ @Override
+ public void initialize(String inputName, double samplePercent) {
+ super.initialize(Statistic.ROWCOUNT, inputName, samplePercent);
+ state = State.MERGE;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getInput() {
+ return inputName;
+ }
+
+ @Override
+ public void merge(MapVector input) {
+ // Check the input is a Map Vector
+ assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector vv : input) {
+ String colName = vv.getField().getName();
+ NullableBigIntVector biv = (NullableBigIntVector) vv;
+ NullableBigIntVector.Accessor accessor = biv.getAccessor();
+ long sum = 0;
+ if (sumHolder.get(colName) != null) {
+ sum = sumHolder.get(colName);
+ }
+ if (!accessor.isNull(0)) {
+ sum += accessor.get(0);
+ sumHolder.put(colName, sum);
+ }
+ }
+ }
+
+ public long getStat(String colName) {
+ if (state != State.COMPLETE) {
+ throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+ name));
+ }
+ return (long)(100.0*sumHolder.get(colName)/ samplePercent);
+ }
+
+ @Override
+ public void setOutput(MapVector output) {
+ // Check the input is a Map Vector
+ assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+ for (ValueVector outMapCol : output) {
+ String colName = outMapCol.getField().getName();
+ NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+ vv.allocateNewSafe();
+ if (sumHolder.get(colName) != null) {
+ vv.getMutator().setSafe(0, (long)(100.0*sumHolder.get(colName)/ samplePercent));
+ } else {
+ vv.getMutator().setNull(0);
+ }
+ }
+ state = State.COMPLETE;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
new file mode 100644
index 000000000..b9f905dac
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
@@ -0,0 +1,44 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+/*
+ * Base Statistics class - all statistics classes should extend this class
+ */
+public abstract class Statistic {
+ /*
+ * The lifecycle states for statistics
+ */
+ public enum State {INIT, CONFIG, MERGE, COMPLETE};
+ /*
+ * List of statistics used in Drill.
+ */
+ public static final String COLNAME = "column";
+ public static final String COLTYPE = "majortype";
+ public static final String SCHEMA = "schema";
+ public static final String COMPUTED = "computed";
+ public static final String ROWCOUNT = "rowcount";
+ public static final String NNROWCOUNT = "nonnullrowcount";
+ public static final String NDV = "approx_count_distinct";
+ public static final String HLL_MERGE = "hll_merge";
+ public static final String HLL = "hll";
+ public static final String AVG_WIDTH = "avg_width";
+ public static final String SUM_WIDTH = "sum_width";
+ public static final String CNT_DUPS = "approx_count_dups";
+ public static final String SUM_DUPS = "sum";
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
new file mode 100644
index 000000000..df64a5f0b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
@@ -0,0 +1,249 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import com.sun.codemodel.JExpr;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.expression.FunctionCallFactory;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.FieldIdUtil;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/*
+ * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
+ * for every record. Seems unnecessary.
+ *
+ * Example input and output:
+ * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
+ * Schema of outgoing batch:
+ * "columns" : MAP - Column names
+ * "region_id" : VARCHAR
+ * "sales_city" : VARCHAR
+ * "cnt" : VARCHAR
+ * "statscount" : MAP
+ * "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id
+ * in incoming batch
+ * "sales_city" : BIGINT - statscount(sales_city)
+ * "cnt" : BIGINT - statscount(cnt)
+ * "nonnullstatcount" : MAP
+ * "region_id" : BIGINT - nonnullstatcount(region_id)
+ * "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ * "cnt" : BIGINT - nonnullstatcount(cnt)
+ * .... another map for next stats function ....
+ */
+
+public class StatisticsAggBatch extends StreamingAggBatch {
+ // List of statistics functions e.g. rowcount, ndv output by StatisticsAggBatch
+ private List<String> functions;
+ // List of implicit columns for which we do NOT want to compute statistics
+ private Map<String, ColumnExplorer.ImplicitFileColumns> implicitFileColumnsMap;
+
+ public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
+ FragmentContext context) throws OutOfMemoryException {
+ super(popConfig, incoming, context);
+ // Get the list from the physical operator configuration
+ functions = popConfig.getFunctions();
+ implicitFileColumnsMap = ColumnExplorer.initImplicitFileColumns(context.getOptions());
+ }
+
+ /*
+ * Returns whether the given column is an implicit column
+ */
+ private boolean isImplicitFileColumn(MaterializedField mf) {
+ return implicitFileColumnsMap.get(SchemaPath.getSimplePath(mf.getName()).toString()) != null;
+ }
+
+ /*
+ * Create the field id for the value vector corresponding to the materialized expression
+ */
+ private TypedFieldId createVVFieldId(LogicalExpression mle, String name, MapVector parent) {
+ Class<? extends ValueVector> vvc =
+ TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
+ mle.getMajorType().getMode());
+ ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
+ TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
+ assert pfid.getFieldIds().length == 1;
+ TypedFieldId.Builder builder = TypedFieldId.newBuilder();
+ builder.addId(pfid.getFieldIds()[0]);
+ TypedFieldId id =
+ FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
+ SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
+ return id;
+ }
+
+ /*
+ * Creates the key column within the parent value vector
+ */
+ private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
+ List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
+ throws SchemaChangeException {
+ LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+ TypedFieldId id = createVVFieldId(mle, name, parent);
+ keyExprs.add(mle);
+ keyOutputIds.add(id);
+ }
+
+ /*
+ * Creates the value vector within the parent value vector. The map vector key is
+ * is the column name and value is the statistic expression e.g. "salary" : NDV(emp.salary)
+ */
+ private void addMapVector(String name, MapVector parent, LogicalExpression expr,
+ List<LogicalExpression> valueExprs) throws SchemaChangeException {
+ LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+ TypedFieldId id = createVVFieldId(mle, name, parent);
+ valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
+ }
+
+ /*
+ * Generates the code for the statistics aggregate which is subclassed from StreamingAggregator
+ */
+ private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
+ List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
+ throws SchemaChangeException, ClassTransformationException, IOException {
+
+ ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
+ cg.getCodeGenerator().plainJavaCapable(true);
+ // Uncomment out this line to debug the generated code.
+ // cg.getCodeGenerator().saveCodeForDebugging(true);
+
+ LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
+ LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
+ TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
+
+ keyExprs.toArray(keyExprsArray);
+ valueExprs.toArray(valueExprsArray);
+ keyOutputIds.toArray(keyOutputIdsArray);
+
+ setupIsSame(cg, keyExprsArray);
+ setupIsSameApart(cg, keyExprsArray);
+ addRecordValues(cg, valueExprsArray);
+ outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
+ outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
+
+ cg.getBlock("resetValues")._return(JExpr.TRUE);
+ getIndex(cg);
+
+ container.buildSchema(SelectionVectorMode.NONE);
+ StreamingAggregator agg = context.getImplementationClass(cg);
+ agg.setup(oContext, incoming, this, ValueVector.MAX_ROW_COUNT);
+ return agg;
+ }
+
+ @Override
+ protected StreamingAggregator createAggregatorInternal()
+ throws SchemaChangeException, ClassTransformationException, IOException {
+ List<LogicalExpression> keyExprs = Lists.newArrayList();
+ List<LogicalExpression> valueExprs = Lists.newArrayList();
+ List<TypedFieldId> keyOutputIds = Lists.newArrayList();
+ String [] colMeta = new String [] {Statistic.COLNAME, Statistic.COLTYPE};
+ container.clear();
+ // Generate the `column` map containing the columns in the incoming schema. Ignore
+ // the implicit columns
+ for (String col : colMeta) {
+ MapVector parent = new MapVector(col, oContext.getAllocator(), null);
+ container.add(parent);
+ for (MaterializedField mf : incoming.getSchema()) {
+ LogicalExpression expr;
+ if (col.equals(colMeta[0])) {
+ expr = ValueExpressions.getChar(SchemaPath.getSimplePath(mf.getName()).toString(), 0);
+ } else {
+ expr = ValueExpressions.getChar(DrillStatsTable.getMapper().writeValueAsString(mf.getType()), 0);
+ }
+ // Ignore implicit columns
+ if (!isImplicitFileColumn(mf)) {
+ createNestedKeyColumn(
+ parent,
+ SchemaPath.getSimplePath(mf.getName()).toString(),
+ expr,
+ keyExprs,
+ keyOutputIds
+ );
+ }
+ }
+ }
+ // Iterate over the list of statistics and generate a MAP whose key is the column
+ // and the value is the statistic for the column e.g.
+ // NDV <<"employee_id" : 500>, <"salary" : 10>> represents a MAP of NDVs (# distinct values)
+ // employee NDV = 500, salary NDV = 10
+ for (String func : functions) {
+ MapVector parent = new MapVector(func, oContext.getAllocator(), null);
+ container.add(parent);
+
+ for (MaterializedField mf : incoming.getSchema()) {
+ // Check stats collection is only being done for supported data-types. Complex types
+ // such as MAP, LIST are not supported!
+ if (isColMinorTypeValid(mf) && !isImplicitFileColumn(mf)) {
+ List<LogicalExpression> args = Lists.newArrayList();
+ args.add(SchemaPath.getSimplePath(mf.getName()));
+ LogicalExpression call = FunctionCallFactory.createExpression(func, args);
+ addMapVector(SchemaPath.getSimplePath(mf.getName()).toString(), parent, call, valueExprs);
+ }
+ }
+ }
+ // Now generate the code for the statistics aggregate
+ return codegenAggregator(keyExprs, valueExprs, keyOutputIds);
+ }
+
+ private boolean isColMinorTypeValid(MaterializedField mf) throws UnsupportedOperationException {
+ String mTypeStr = null;
+ if (mf.getType().getMinorType() == TypeProtos.MinorType.GENERIC_OBJECT) {
+ mTypeStr = "GENERIC OBJECT";
+ } else if (mf.getType().getMinorType() == TypeProtos.MinorType.LATE) {
+ mTypeStr = "LATE";
+ }else if (mf.getType().getMinorType() == TypeProtos.MinorType.LIST) {
+ mTypeStr = "LIST";
+ } else if (mf.getType().getMinorType() == TypeProtos.MinorType.MAP) {
+ mTypeStr = "MAP";
+ } else if (mf.getType().getMinorType() == TypeProtos.MinorType.UNION) {
+ mTypeStr = "UNION";
+ }
+ if (mTypeStr != null) {
+ return false;
+ //throw new UnsupportedOperationException(String.format("Column %s has data-type %s which is not supported",
+ // mf.getName(), mTypeStr));
+ } else {
+ return true;
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
index aba325cb3..252b73818 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.drill.exec.physical.impl.aggregate;
+package org.apache.drill.exec.physical.impl.statistics;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import java.util.List;
@@ -26,7 +26,6 @@ import org.apache.drill.exec.physical.impl.BatchCreator;
import org.apache.drill.exec.record.CloseableRecordBatch;
import org.apache.drill.exec.record.RecordBatch;
-@SuppressWarnings("unused")
public class StatisticsAggBatchCreator implements BatchCreator<StatisticsAggregate>{
@Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
new file mode 100644
index 000000000..95982b70d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
@@ -0,0 +1,405 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.ArrayList;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.DateVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ *
+ * Example input and output:
+ * Schema of incoming batch:
+ * "columns" : MAP - Column names
+ * "region_id" : VARCHAR
+ * "sales_city" : VARCHAR
+ * "cnt" : VARCHAR
+ * "statscount" : MAP - Number of entries (rows)
+ * "region_id" : BIGINT - statscount(region_id)
+ * in incoming batch
+ * "sales_city" : BIGINT - statscount(sales_city)
+ * "cnt" : BIGINT - statscount(cnt)
+ * "nonnullstatcount" : MAP - Number of non-null entries (rows)
+ * "region_id" : BIGINT - nonnullstatcount(region_id)
+ * "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ * "cnt" : BIGINT - nonnullstatcount(cnt)
+ * .... another map for next stats function ....
+ * Schema of outgoing batch:
+ * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ * "computed" : DATE - What time is it computed?
+ * "columns" : MAP - Column names
+ * "region_id" : VARCHAR
+ * "sales_city" : VARCHAR
+ * "cnt" : VARCHAR
+ * "statscount" : MAP - Number of entries (rows)
+ * "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id
+ * in incoming batch
+ * "sales_city" : BIGINT - statscount(sales_city)
+ * "cnt" : BIGINT - statscount(cnt)
+ * "nonnullstatcount" : MAP - Number of non-null entries (rows)
+ * "region_id" : BIGINT - nonnullstatcount(region_id)
+ * "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ * "cnt" : BIGINT - nonnullstatcount(cnt)
+ * .... another map for next stats function ....
+ */
+public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
+ private Map<String, String> functions;
+ private boolean first = true;
+ private boolean finished = false;
+ private int schema = 0;
+ private int recordCount = 0;
+ private List<String> columnsList = null;
+ private double samplePercent = 100.0;
+ private List<MergedStatistic> mergedStatisticList = null;
+
+ public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
+ FragmentContext context) throws OutOfMemoryException {
+ super(popConfig, context, incoming);
+ functions = popConfig.getFunctions();
+ samplePercent = popConfig.getSamplePercent();
+ mergedStatisticList = new ArrayList<>();
+ }
+
+ /*
+ * Creates key columns for the outgoing batch e.g. `schema`, `computed`. These columns are NOT
+ * table columns for which statistics will be computed.
+ */
+ private void createKeyColumn(String name, LogicalExpression expr)
+ throws SchemaChangeException {
+ LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+ MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
+ ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+ container.add(vector);
+ }
+
+ /*
+ * Adds the `name` column value vector in the `parent` map vector. These `name` columns are
+ * table columns for which statistics will be computed.
+ */
+ private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr)
+ throws SchemaChangeException {
+ LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+ Class<? extends ValueVector> vvc =
+ TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
+ mle.getMajorType().getMode());
+ ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
+ return vector;
+ }
+
+ /*
+ * Identify the list of fields within a map which are generated by StatisticsMerge. Perform
+ * basic sanity check i.e. all maps have the same number of columns and those columns are
+ * the same in each map
+ */
+ private void buildColumnsList() {
+ Map<String, Boolean> inputFunctions = new HashMap<>();
+ // Prepare map of input functions for verifying only they appear in the incoming batch
+ for (String inputFunc : functions.values()) {
+ inputFunctions.put(inputFunc, false);
+ }
+ List<String> lastMapColumnsList = null;
+ //Populate the columns list from the `columns` map
+ for (VectorWrapper<?> vw : incoming) {
+ String inputFunc = vw.getField().getName();
+ if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
+ continue;
+ }
+ if (inputFunctions.get(inputFunc)) {
+ throw new IllegalArgumentException (String.format("The statistic `%s` appears more than once",
+ inputFunc));
+ } else {
+ inputFunctions.put(inputFunc, true);
+ }
+ if (vw.getField().getName().equals(Statistic.COLNAME)) {
+ columnsList = Lists.newArrayList();
+ for (ValueVector vv : vw.getValueVector()) {
+ if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
+ throw new IllegalArgumentException("StatisticsMerge of nested map is not supported");
+ }
+ columnsList.add(vv.getField().getName());
+ }
+ lastMapColumnsList = columnsList;
+ }
+ }
+ // Verify the rest of the maps have the same columns
+ for (VectorWrapper<?> vw : incoming) {
+ String inputFunc = vw.getField().getName();
+ if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
+ continue;
+ }
+ if (!inputFunctions.get(inputFunc)) {
+ throw new IllegalArgumentException (String.format("The statistic `%s` is not expected here",
+ inputFunc));
+ }
+ if (columnsList.size() != lastMapColumnsList.size()
+ || !lastMapColumnsList.containsAll(columnsList)) {
+ // Error!! Maps with different size and/or keys. The map for each statistics (e.g. NDV)
+ // should match exactly with the column map i.e. we did not run into any issues while
+ // generating statistics for all the specified columns
+ throw new IllegalStateException("StatisticsMerge Maps have different fields");
+ }
+ }
+ }
+
+ /* Prepare the outgoing container. Generates the outgoing record batch schema.
+ * Please look at the comments above the class definition which describes the
+ * incoming/outgoing batch schema
+ */
+ private void buildOutputContainer() throws SchemaChangeException {
+ // Populate the list of statistics which will be output in the schema
+ for (VectorWrapper<?> vw : incoming) {
+ for (String outputStatName : functions.keySet()) {
+ if (functions.get(outputStatName).equals(vw.getField().getName())) {
+ mergedStatisticList.add(MergedStatisticFactory.getMergedStatistic(outputStatName,
+ functions.get(outputStatName), samplePercent));
+ }
+ }
+ }
+ // Configure settings/dependencies for statistics, if needed
+ for (MergedStatistic statistic : mergedStatisticList) {
+ if (statistic.getName().equals(Statistic.AVG_WIDTH)) {
+ ((AvgWidthMergedStatistic)statistic).configure(mergedStatisticList);
+ } else if (statistic.getName().equals(Statistic.NDV)) {
+ NDVMergedStatistic.NDVConfiguration config =
+ new NDVMergedStatistic.NDVConfiguration(context.getOptions(),
+ mergedStatisticList);
+ ((NDVMergedStatistic)statistic).configure(config);
+ } else if (statistic.getName().equals(Statistic.SUM_DUPS)) {
+ ((CntDupsMergedStatistic)statistic).configure(mergedStatisticList);
+ } else if (statistic.getName().equals(Statistic.HLL_MERGE)) {
+ ((HLLMergedStatistic)statistic).configure(context.getOptions());
+ }
+ }
+ // Create the schema number and time when computed in the outgoing vector
+ createKeyColumn(Statistic.SCHEMA, ValueExpressions.getBigInt(schema++));
+ GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+ calendar.setTimeInMillis(System.currentTimeMillis());
+ createKeyColumn(Statistic.COMPUTED, ValueExpressions.getDate(calendar));
+
+ // Create output map vectors corresponding to each statistic (e.g. rowcount)
+ for (MergedStatistic statistic : mergedStatisticList) {
+ String targetTypeStatistic = statistic.getInput();
+ for (VectorWrapper<?> vw : incoming) {
+ if (targetTypeStatistic.equals(vw.getField().getName())) {
+ addVectorToOutgoingContainer(statistic.getName(), vw);
+ }
+ }
+ }
+ container.setRecordCount(0);
+ recordCount = 0;
+ container.buildSchema(incoming.getSchema().getSelectionVectorMode());
+ }
+
+ /* Adds a value vector corresponding to the statistic in the outgoing record batch.
+ * Determines the MajorType based on the incoming value vector. Please look at the
+ * comments above the class definition which describes the incoming/outgoing batch schema
+ */
+ private void addVectorToOutgoingContainer(String outStatName, VectorWrapper vw)
+ throws SchemaChangeException {
+ // Input map vector
+ MapVector inputVector = (MapVector) vw.getValueVector();
+ assert inputVector.getPrimitiveVectors().size() > 0;
+ // Proceed to create output map vector with same name e.g. statcount etc.
+ MajorType mt = inputVector.getField().getType();
+ MaterializedField mf = MaterializedField.create(outStatName, mt);
+
+ ValueVector outputValueVector = TypeHelper.getNewVector(mf, oContext.getAllocator());
+ container.add(outputValueVector);
+ MapVector outputVector = (MapVector) outputValueVector;
+
+ for (ValueVector vv : inputVector) {
+ String columnName = vv.getField().getName();
+ // Except column name, type all the rest are actual statistic functions (rely on
+ // statistic calculation functions).
+ if (outStatName.equals(Statistic.COLNAME)
+ || outStatName.equals(Statistic.COLTYPE)) {
+ outputVector.addOrGet(columnName, vv.getField().getType(), vv.getClass());
+ } else {
+ TypeProtos.MinorType minorType;
+ if (outStatName.equals(Statistic.AVG_WIDTH)) {
+ minorType = TypeProtos.MinorType.FLOAT8;
+ } else if (outStatName.equals(Statistic.HLL_MERGE)) {
+ minorType = TypeProtos.MinorType.VARBINARY;
+ } else {
+ minorType = TypeProtos.MinorType.BIGINT;
+ }
+ Class<? extends ValueVector> vvc =
+ TypeHelper.getValueVectorClass(minorType,
+ TypeProtos.DataMode.OPTIONAL);
+ outputVector.addOrGet(columnName, Types.optional(minorType), vvc);
+ }
+ }
+ }
+
+ /* Prepare the outgoing container. Populates the outgoing record batch data.
+ * Please look at the comments above the class definition which describes the
+ * incoming/outgoing batch schema
+ */
+ private IterOutcome buildOutgoingRecordBatch() {
+ for (VectorWrapper<?> vw : container) {
+ String outputStatName = vw.getField().getName();
+ // Populate the `schema` and `computed` fields
+ if (outputStatName.equals(Statistic.SCHEMA)) {
+ BigIntVector vv = (BigIntVector) vw.getValueVector();
+ vv.allocateNewSafe();
+ vv.getMutator().setSafe(0, schema);
+ } else if (outputStatName.equals(Statistic.COMPUTED)) {
+ GregorianCalendar cal = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+ DateVector vv = (DateVector) vw.getValueVector();
+ vv.allocateNewSafe();
+ vv.getMutator().setSafe(0, cal.getTimeInMillis());
+ } else {
+ // Populate the rest of the merged statistics. Each statistic is a map which
+ // contains <COL_NAME, STATS_VALUE> pairs
+ MapVector vv = (MapVector) vw.getValueVector();
+ for (MergedStatistic outputStat : mergedStatisticList) {
+ if (outputStatName.equals(outputStat.getName())) {
+ outputStat.setOutput(vv);
+ vv.getMutator().setValueCount(columnsList.size());
+ break;
+ }
+ }
+ }
+ }
+ ++recordCount;
+ // Populate the number of records (1) inside the outgoing batch.
+ container.setRecordCount(1);
+ return IterOutcome.OK;
+ }
+
+ @Override
+ protected boolean setupNewSchema() throws SchemaChangeException {
+ container.clear();
+ // Generate the list of fields for which statistics will be merged
+ buildColumnsList();
+ // Generate the schema for the outgoing record batch
+ buildOutputContainer();
+ return true;
+ }
+
+ @Override
+ protected IterOutcome doWork() {
+ for (MergedStatistic outputStat : mergedStatisticList) {
+ String inputStat = outputStat.getInput();
+ for (VectorWrapper<?> vw : incoming) {
+ MapVector vv = (MapVector) vw.getValueVector();
+ if (vv.getField().getName().equals(inputStat)) {
+ outputStat.merge(vv);
+ break;
+ }
+ }
+ }
+ return IterOutcome.OK;
+ }
+
+ @Override
+ public VectorContainer getOutgoingContainer() {
+ return this.container;
+ }
+
+ @Override
+ public void dump() {
+
+ }
+
+ @Override
+ public IterOutcome innerNext() {
+ IterOutcome outcome;
+ boolean didSomeWork = false;
+ if (finished) {
+ return IterOutcome.NONE;
+ }
+ try {
+ outer: while (true) {
+ outcome = next(incoming);
+ switch (outcome) {
+ case NONE:
+ break outer;
+ case OUT_OF_MEMORY:
+ case NOT_YET:
+ case STOP:
+ return outcome;
+ case OK_NEW_SCHEMA:
+ if (first) {
+ first = false;
+ if (!setupNewSchema()) {
+ outcome = IterOutcome.OK;
+ }
+ return outcome;
+ }
+ //fall through
+ case OK:
+ assert first == false : "First batch should be OK_NEW_SCHEMA";
+ IterOutcome out = doWork();
+ didSomeWork = true;
+ if (out != IterOutcome.OK) {
+ return out;
+ }
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
+ }
+ }
+ } catch (SchemaChangeException ex) {
+ kill(false);
+ context.getExecutorState().fail(UserException.unsupportedError(ex).build(logger));
+ return IterOutcome.STOP;
+ }
+
+ // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
+ // exhausting all upstream, then return OK. Otherwise, return NONE.
+ if (didSomeWork) {
+ IterOutcome out = buildOutgoingRecordBatch();
+ finished = true;
+ return out;
+ } else {
+ return outcome;
+ }
+ }
+
+ @Override
+ public int getRecordCount() {
+ return recordCount;
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java
new file mode 100644
index 000000000..7599ab0d9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java
@@ -0,0 +1,38 @@
+/*
+ * 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.drill.exec.physical.impl.statistics;
+
+import java.util.List;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+@SuppressWarnings("unused")
+public class StatisticsMergeBatchCreator implements BatchCreator<StatisticsMerge>{
+
+ @Override
+ public CloseableRecordBatch getBatch(ExecutorFragmentContext context, StatisticsMerge config,
+ List<RecordBatch> children) throws ExecutionSetupException {
+ Preconditions.checkArgument(children.size() == 1);
+ return new StatisticsMergeBatch(config, children.iterator().next(), context);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
index 733524f21..e86a92785 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -17,8 +17,9 @@
*/
package org.apache.drill.exec.physical.impl.unpivot;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import java.util.List;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.ops.ExecutorFragmentContext;
import org.apache.drill.exec.physical.config.UnpivotMaps;
@@ -26,7 +27,6 @@ import org.apache.drill.exec.physical.impl.BatchCreator;
import org.apache.drill.exec.record.CloseableRecordBatch;
import org.apache.drill.exec.record.RecordBatch;
-@SuppressWarnings("unused")
public class UnpivotMapsBatchCreator implements BatchCreator<UnpivotMaps>{
@Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
index e98d70e20..0f3517f4c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.unpivot;
import java.util.List;
import java.util.Map;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.types.TypeProtos.MajorType;
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.exec.exception.OutOfMemoryException;
@@ -38,35 +39,34 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
/**
- * TODO: This needs cleanup, especially in state transitions.
- *
* Unpivot maps. Assumptions are:
* 1) all child vectors in a map are of same type.
* 2) Each map contains the same number of fields and field names are also same (types could be different).
*
* Example input and output:
* Schema of input:
- * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
- * "computed" : BIGINT - What time is it computed?
+ * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ * "computed" : BIGINT - What time is it computed?
* "columns" : MAP - Column names
- * "region_id" : VARCHAR
+ * "region_id" : VARCHAR
* "sales_city" : VARCHAR
- * "cnt" : VARCHAR
+ * "cnt" : VARCHAR
* "statscount" : MAP
- * "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id in incoming batch
+ * "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id
+ * in incoming batch
* "sales_city" : BIGINT - statscount(sales_city)
- * "cnt" : BIGINT - statscount(cnt)
+ * "cnt" : BIGINT - statscount(cnt)
* "nonnullstatcount" : MAP
- * "region_id" : BIGINT - nonnullstatcount(region_id)
+ * "region_id" : BIGINT - nonnullstatcount(region_id)
* "sales_city" : BIGINT - nonnullstatcount(sales_city)
- * "cnt" : BIGINT - nonnullstatcount(cnt)
+ * "cnt" : BIGINT - nonnullstatcount(cnt)
* .... another map for next stats function ....
*
* Schema of output:
- * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
- * "computed" : BIGINT - What time is this computed?
- * "column" : column name
- * "statscount" : BIGINT
+ * "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ * "computed" : BIGINT - What time is this computed?
+ * "column" : column name
+ * "statscount" : BIGINT
* "nonnullstatcount" : BIGINT
* .... one column for each map type ...
*/
@@ -74,7 +74,7 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnpivotMapsRecordBatch.class);
private final List<String> mapFieldsNames;
-
+ private boolean first = true;
private int keyIndex = 0;
private List<String> keyList = null;
@@ -99,11 +99,54 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
@Override
public IterOutcome innerNext() {
- if (keyIndex != 0) {
- doWork();
- return IterOutcome.OK;
- } else {
- return super.innerNext();
+
+ IterOutcome upStream = IterOutcome.OK;
+
+ if (keyIndex == 0) {
+ upStream = next(incoming);
+ }
+ // Process according to upstream outcome
+ switch (upStream) {
+ case NONE:
+ case OUT_OF_MEMORY:
+ case NOT_YET:
+ case STOP:
+ return upStream;
+ case OK_NEW_SCHEMA:
+ if (first) {
+ first = false;
+ }
+ try {
+ if (!setupNewSchema()) {
+ upStream = IterOutcome.OK;
+ } else {
+ return upStream;
+ }
+ } catch (SchemaChangeException ex) {
+ kill(false);
+ logger.error("Failure during query", ex);
+ context.getExecutorState().fail(ex);
+ return IterOutcome.STOP;
+ }
+ //fall through
+ case OK:
+ assert first == false : "First batch should be OK_NEW_SCHEMA";
+ try {
+ container.zeroVectors();
+ IterOutcome out = doWork();
+ // Preserve OK_NEW_SCHEMA unless doWork() runs into an issue
+ if (out != IterOutcome.OK) {
+ upStream = out;
+ }
+ } catch (Exception ex) {
+ kill(false);
+ logger.error("Failure during query", ex);
+ context.getExecutorState().fail(ex);
+ return IterOutcome.STOP;
+ }
+ return upStream;
+ default:
+ throw new UnsupportedOperationException("Unsupported upstream state " + upStream);
}
}
@@ -113,7 +156,6 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
private void doTransfer() {
final int inputCount = incoming.getRecordCount();
-
for (TransferPair tp : transferList) {
tp.splitAndTransfer(0, inputCount);
}
@@ -150,7 +192,7 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
keyList = Lists.newArrayList();
for (ValueVector vv : vw.getValueVector()) {
- keyList.add(vv.getField().getName());
+ keyList.add(SchemaPath.getSimplePath(vv.getField().getName()).toString());
}
if (lastMapKeyList == null) {
@@ -168,11 +210,11 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
copySrcVecMap = Maps.newHashMap();
for (VectorWrapper<?> vw : incoming) {
MaterializedField ds = vw.getField();
- String col = vw.getField().getName();
+ String colName = vw.getField().getName();
- if (!mapFieldsNames.contains(col)) {
+ if (!mapFieldsNames.contains(colName)) {
MajorType mt = vw.getValueVector().getField().getType();
- MaterializedField mf = MaterializedField.create(col, mt);
+ MaterializedField mf = MaterializedField.create(colName, mt);
container.add(TypeHelper.getNewVector(mf, oContext.getAllocator()));
copySrcVecMap.put(mf, vw.getValueVector());
continue;
@@ -182,7 +224,7 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
assert mapVector.getPrimitiveVectors().size() > 0;
MajorType mt = mapVector.iterator().next().getField().getType();
- MaterializedField mf = MaterializedField.create(col, mt);
+ MaterializedField mf = MaterializedField.create(colName, mt);
assert !dataSrcVecMap.containsKey(mf);
container.add(TypeHelper.getNewVector(mf, oContext.getAllocator()));
@@ -190,31 +232,27 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
dataSrcVecMap.put(mf, m);
for (ValueVector vv : mapVector) {
- String fieldName = vv.getField().getName();
-
+ String fieldName = SchemaPath.getSimplePath(vv.getField().getName()).toString();
if (!keyList.contains(fieldName)) {
throw new UnsupportedOperationException("Unpivot data vector " +
ds + " contains key " + fieldName + " not contained in key source!");
}
-
if (vv.getField().getType().getMinorType() == MinorType.MAP) {
throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
}
-
m.put(fieldName, vv);
}
}
-
container.buildSchema(incoming.getSchema().getSelectionVectorMode());
}
private void prepareTransfers() {
+ ValueVector vv;
+ TransferPair tp;
transferList = Lists.newArrayList();
+
for (VectorWrapper<?> vw : container) {
MaterializedField mf = vw.getField();
-
- ValueVector vv;
- TransferPair tp;
if (dataSrcVecMap.containsKey(mf)) {
String k = keyList.get(keyIndex);
vv = dataSrcVecMap.get(mf).get(k);
@@ -223,7 +261,6 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
vv = copySrcVecMap.get(mf);
tp = vv.makeTransferPair(vw.getValueVector());
}
-
transferList.add(tp);
}
}
@@ -231,7 +268,6 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
@Override
protected boolean setupNewSchema() throws SchemaChangeException {
container.clear();
-
buildKeyList();
buildOutputContainer();
return true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
index c38cc5a9a..65268923d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
@@ -257,12 +257,13 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus());
newFileSelection.setMetaContext(metaContext);
final FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection);
- final DrillTranslatableTable newTable = new DrillTranslatableTable(
- new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
- table.getUserName(),
- newFormatSelection));
+ final DynamicDrillTable dynamicDrillTable = new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
+ table.getUserName(), newFormatSelection);
+ /* Copy statistics from the original table */
+ dynamicDrillTable.setStatsTable(table.getStatsTable());
+ final DrillTranslatableTable newTable = new DrillTranslatableTable(dynamicDrillTable);
final RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(t.getRelOptSchema(), t.getRowType(), newTable,
- GuavaUtils.convertToUnshadedImmutableList(ImmutableList.of()));
+ GuavaUtils.convertToUnshadedImmutableList(ImmutableList.of()));
// return an EnumerableTableScan with fileSelection being part of digest of TableScan node.
return DirPrunedEnumerableTableScan.create(oldScan.getCluster(), newOptTableImpl, newFileSelection.toString());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java
index cd1f4fa46..d79155663 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java
@@ -88,4 +88,13 @@ public abstract class DrillAggregateRelBase extends Aggregate implements DrillRe
return computeHashAggCost(planner, mq);
}
+ @Override
+ public double estimateRowCount(RelMetadataQuery mq) {
+ // Get the number of distinct group-by key values present in the input
+ if (!DrillRelOptUtil.guessRows(this)) {
+ return mq.getRowCount(this);
+ } else {
+ return super.estimateRowCount(mq);
+ }
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
index b0bca286a..7125815dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
@@ -18,6 +18,7 @@
package org.apache.drill.exec.planner.common;
import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.util.NumberUtil;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.exec.planner.cost.DrillCostBase;
import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
@@ -114,6 +115,7 @@ public abstract class DrillFilterRelBase extends Filter implements DrillRelNode
selectivity = filterMaxSelectivityEstimateFactor;
}
}
- return selectivity * mq.getRowCount(getInput());
+ // The utility function also considers nulls.
+ return NumberUtil.multiply(selectivity, mq.getRowCount(getInput()));
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
index 42fbedb4a..a6b6f4a15 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
@@ -23,7 +23,6 @@ import java.util.List;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.CorrelationId;
@@ -108,7 +107,9 @@ public abstract class DrillJoinRelBase extends Join implements DrillJoin {
LogicalJoin jr = LogicalJoin.create(this.getLeft(), this.getRight(), this.getCondition(),
this.getVariablesSet(), this.getJoinType());
- if (RelOptUtil.analyzeSimpleEquiJoin(jr, joinFields)) {
+ if (!DrillRelOptUtil.guessRows(this) //Statistics present for left and right side of the join
+ && jr.getJoinType() == JoinRelType.INNER
+ && DrillRelOptUtil.analyzeSimpleEquiJoin((Join)jr, joinFields)) {
ImmutableBitSet leq = ImmutableBitSet.of(joinFields[0]);
ImmutableBitSet req = ImmutableBitSet.of(joinFields[1]);
@@ -119,13 +120,14 @@ public abstract class DrillJoinRelBase extends Join implements DrillJoin {
Double rrc = mq.getRowCount(this.getRight());
if (ldrc != null && rdrc != null && lrc != null && rrc != null) {
- return (lrc * rrc) / Math.max(ldrc, rdrc);
+ // Join cardinality = (lrc * rrc) / Math.max(ldrc, rdrc). Avoid overflow by dividing earlier
+ return (lrc / Math.max(ldrc, rdrc)) * rrc;
}
}
return joinRowFactor * Math.max(
- mq.getRowCount(left),
- mq.getRowCount(right));
+ mq.getRowCount(this.getLeft()),
+ mq.getRowCount(this.getRight()));
}
/**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index 32f6ac294..14254666f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -30,9 +30,13 @@ import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.apache.calcite.plan.hep.HepRelVertex;
import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Join;
import org.apache.calcite.rel.rules.ProjectRemoveRule;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -56,6 +60,9 @@ import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.common.types.Types;
import org.apache.drill.exec.planner.logical.DrillRelFactories;
import org.apache.drill.exec.planner.logical.FieldsReWriterUtil;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.resolver.TypeCastRules;
import org.apache.drill.exec.util.Utilities;
@@ -550,4 +557,99 @@ public abstract class DrillRelOptUtil {
}
}
}
+
+ /**
+ * Returns whether statistics-based estimates or guesses are used by the optimizer
+ * for the {@link RelNode} rel.
+ * @param rel : input
+ * @return TRUE if the estimate is a guess, FALSE otherwise
+ * */
+ public static boolean guessRows(RelNode rel) {
+ final PlannerSettings settings =
+ rel.getCluster().getPlanner().getContext().unwrap(PlannerSettings.class);
+ if (!settings.useStatistics()) {
+ return true;
+ }
+ /* We encounter RelSubset/HepRelVertex which are CALCITE constructs, hence we
+ * cannot add guessRows() to the DrillRelNode interface.
+ */
+ if (rel instanceof RelSubset) {
+ if (((RelSubset) rel).getBest() != null) {
+ return guessRows(((RelSubset) rel).getBest());
+ } else if (((RelSubset) rel).getOriginal() != null) {
+ return guessRows(((RelSubset) rel).getOriginal());
+ }
+ } else if (rel instanceof HepRelVertex) {
+ if (((HepRelVertex) rel).getCurrentRel() != null) {
+ return guessRows(((HepRelVertex) rel).getCurrentRel());
+ }
+ } else if (rel instanceof TableScan) {
+ DrillTable table = rel.getTable().unwrap(DrillTable.class);
+ if (table == null) {
+ table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
+ }
+ if (table != null
+ && table.getStatsTable() != null
+ && table.getStatsTable().isMaterialized()) {
+ return false;
+ } else {
+ return true;
+ }
+ } else {
+ for (RelNode child : rel.getInputs()) {
+ if (guessRows(child)) { // at least one child is a guess
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Returns whether the join condition is a simple equi-join or not. A simple equi-join is
+ * defined as an two-table equality join (no self-join)
+ * @param join : input join
+ * @param joinFieldOrdinals: join field ordinal w.r.t. the underlying inputs to the join
+ * @return TRUE if the join is a simple equi-join (not a self-join), FALSE otherwise
+ * */
+ public static boolean analyzeSimpleEquiJoin(Join join, int[] joinFieldOrdinals) {
+ RexNode joinExp = join.getCondition();
+ if(joinExp.getKind() != SqlKind.EQUALS) {
+ return false;
+ } else {
+ RexCall binaryExpression = (RexCall)joinExp;
+ RexNode leftComparand = (RexNode)binaryExpression.operands.get(0);
+ RexNode rightComparand = (RexNode)binaryExpression.operands.get(1);
+ if(!(leftComparand instanceof RexInputRef)) {
+ return false;
+ } else if(!(rightComparand instanceof RexInputRef)) {
+ return false;
+ } else {
+ int leftFieldCount = join.getLeft().getRowType().getFieldCount();
+ int rightFieldCount = join.getRight().getRowType().getFieldCount();
+ RexInputRef leftFieldAccess = (RexInputRef)leftComparand;
+ RexInputRef rightFieldAccess = (RexInputRef)rightComparand;
+ if(leftFieldAccess.getIndex() >= leftFieldCount+rightFieldCount ||
+ rightFieldAccess.getIndex() >= leftFieldCount+rightFieldCount) {
+ return false;
+ }
+ /* Both columns reference same table */
+ if((leftFieldAccess.getIndex() >= leftFieldCount &&
+ rightFieldAccess.getIndex() >= leftFieldCount) ||
+ (leftFieldAccess.getIndex() < leftFieldCount &&
+ rightFieldAccess.getIndex() < leftFieldCount)) {
+ return false;
+ } else {
+ if (leftFieldAccess.getIndex() < leftFieldCount) {
+ joinFieldOrdinals[0] = leftFieldAccess.getIndex();
+ joinFieldOrdinals[1] = rightFieldAccess.getIndex() - leftFieldCount;
+ } else {
+ joinFieldOrdinals[0] = rightFieldAccess.getIndex();
+ joinFieldOrdinals[1] = leftFieldAccess.getIndex() - leftFieldCount;
+ }
+ return true;
+ }
+ }
+ }
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
index 6409c8e6c..88b8d30bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
@@ -26,9 +26,12 @@ import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
import org.apache.drill.common.exceptions.DrillRuntimeException;
import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.drill.exec.util.Utilities;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
/**
* Base class for logical/physical scan rel implemented in Drill.
@@ -72,4 +75,15 @@ public abstract class DrillScanRelBase extends TableScan implements DrillRelNode
public GroupScan getGroupScan() {
return groupScan;
}
+
+ @Override public double estimateRowCount(RelMetadataQuery mq) {
+ return mq.getRowCount(this);
+ }
+
+ @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+ double dRows = estimateRowCount(mq);
+ double dCpu = dRows + 1; // ensure non-zero cost
+ double dIo = 0;
+ return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
index a22552b3e..668f1d2be 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,115 +17,158 @@
*/
package org.apache.drill.exec.planner.common;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import com.fasterxml.jackson.annotation.JsonGetter;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSetter;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Map;
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.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.planner.logical.DrillTable;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.FormatPlugin;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
/**
- * Wraps the stats table info including schema and tableName. Also materializes stats from storage and keeps them in
- * memory.
+ * 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";
-
+ // All the statistics versions till date
+ public enum STATS_VERSION {V0, V1};
+ // The current version
+ public static final STATS_VERSION CURRENT_VERSION = STATS_VERSION.V1;
+ private final FileSystem fs;
+ private final Path tablePath;
private final String schemaName;
private final String tableName;
-
private final Map<String, Long> ndv = Maps.newHashMap();
private double rowCount = -1;
-
private boolean materialized = false;
+ private TableStatistics statistics = null;
- public DrillStatsTable(String schemaName, String tableName) {
+ public DrillStatsTable(String schemaName, String tableName, Path tablePath, FileSystem fs) {
this.schemaName = schemaName;
this.tableName = tableName;
+ this.tablePath = tablePath;
+ this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf());
}
+ public String getSchemaName() {
+ return schemaName;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ /*
+ * Returns whether statistics have materialized or not i.e. were the table statistics successfully read
+ * from the persistent store?
+ */
+ public boolean isMaterialized() { return materialized; }
/**
- * Get number of distinct values of given column. If stats are not present for the given column, a null is returned.
+ * Get the approximate 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.
+ * Note: returned data may not be accurate. Accuracy depends on whether the table data has changed
+ * after the stats are computed.
*
- * @param col
- * @return
+ * @param col - column for which approximate count distinct is desired
+ * @return approximate count distinct of the column, if available. NULL otherwise.
*/
public Double getNdv(String col) {
- Preconditions.checkState(materialized, "Stats are not yet materialized.");
-
+ // Stats might not have materialized because of errors.
+ if (!materialized) {
+ return null;
+ }
final String upperCol = col.toUpperCase();
- final Long ndvCol = ndv.get(upperCol);
+ Long ndvCol = ndv.get(upperCol);
+ if (ndvCol == null) {
+ ndvCol = ndv.get(SchemaPath.getSimplePath(upperCol).toString());
+ }
+ // Ndv estimation techniques like HLL may over-estimate, hence cap it at rowCount
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.
+ * Note: returned data may not be accurate. Accuracy depends on whether the table data has
+ * changed after the stats are computed.
*
- * @return
+ * @return rowcount for the table, if available. NULL otherwise.
*/
public Double getRowCount() {
- Preconditions.checkState(materialized, "Stats are not yet materialized.");
+ // Stats might not have materialized because of errors.
+ if (!materialized) {
+ return null;
+ }
return rowCount > 0 ? rowCount : null;
}
/**
* Read the stats from storage and keep them in memory.
- * @param context
+ * @param table - Drill table for which we require stats
+ * @param context - Query context
* @throws Exception
*/
- public void materialize(final QueryContext context) throws Exception {
+ public void materialize(final DrillTable table, final QueryContext context) throws IOException {
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())) {
- /*final Listener listener = new Listener(dc.getAllocator());
-
- client.connect();
- client.runQuery(UserBitShared.QueryType.SQL, sql, listener);
-
- listener.waitForCompletion();
-
- for (Map<String, String> r : listener.results) {
- ndv.put(r.get(COL_COLUMN).toUpperCase(), Long.valueOf(r.get(COL_NDV)));
- rowCount = Math.max(rowCount, Long.valueOf(r.get(COL_STATCOUNT)));
- }*/
+ // Deserialize statistics from JSON
+ try {
+ this.statistics = readStatistics(table, tablePath);
+ // Handle based on the statistics version read from the file
+ if (statistics instanceof Statistics_v0) {
+ // Do nothing
+ } else if (statistics instanceof Statistics_v1) {
+ for (DirectoryStatistics_v1 ds : ((Statistics_v1) statistics).getDirectoryStatistics()) {
+ for (ColumnStatistics_v1 cs : ds.getColumnStatistics()) {
+ ndv.put(cs.getName().toUpperCase(), cs.getNdv());
+ rowCount = Math.max(rowCount, cs.getCount());
+ }
+ }
+ }
+ if (statistics != null) { // See stats are available before setting materialized
+ materialized = true;
+ }
+ } catch (IOException ex) {
+ logger.warn("Failed to read the stats file.", ex);
+ throw ex;
}
-
- materialized = true;
}
/**
- * materialize on nodes that have an attached stats table
+ * Materialize on nodes that have an attached stats table
*/
public static class StatsMaterializationVisitor extends RelVisitor {
private QueryContext context;
@@ -145,7 +188,11 @@ public class DrillStatsTable {
final DrillTable drillTable = node.getTable().unwrap(DrillTable.class);
final DrillStatsTable statsTable = drillTable.getStatsTable();
if (statsTable != null) {
- statsTable.materialize(context);
+ statsTable.materialize(drillTable, context);
+ } else {
+ throw new DrillRuntimeException(
+ String.format("Failed to find the stats for table [%s] in schema [%s]",
+ node.getTable().getQualifiedName(), node.getTable().getRelOptSchema()));
}
} catch (Exception e) {
// Log a warning and proceed. We don't want to fail a query.
@@ -155,4 +202,223 @@ public class DrillStatsTable {
super.visit(node, ordinal, parent);
}
}
+
+ /* Each change to the format SHOULD increment the default and/or the max values of the option
+ * exec.statistics.capability_version
+ */
+ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY,
+ property = "statistics_version")
+ @JsonSubTypes({
+ @JsonSubTypes.Type(value = DrillStatsTable.Statistics_v1.class, name="v0"),
+ @JsonSubTypes.Type(value = DrillStatsTable.Statistics_v1.class, name="v1")
+ })
+ public static abstract class TableStatistics {
+ @JsonIgnore public abstract List<? extends DirectoryStatistics> getDirectoryStatistics();
+ }
+
+ public static abstract class DirectoryStatistics {
+ }
+
+ public static abstract class ColumnStatistics {
+ }
+
+ @JsonTypeName("v0")
+ public static class Statistics_v0 extends TableStatistics {
+ @JsonProperty ("directories") List<DirectoryStatistics_v0> directoryStatistics;
+ // Default constructor required for deserializer
+ public Statistics_v0 () { }
+ @JsonGetter ("directories")
+ public List<DirectoryStatistics_v0> getDirectoryStatistics() {
+ return directoryStatistics;
+ }
+ @JsonSetter ("directories")
+ public void setDirectoryStatistics(List<DirectoryStatistics_v0> directoryStatistics) {
+ this.directoryStatistics = directoryStatistics;
+ }
+ }
+
+ public static class DirectoryStatistics_v0 extends DirectoryStatistics {
+ @JsonProperty private double computed;
+ // Default constructor required for deserializer
+ public DirectoryStatistics_v0() { }
+ @JsonGetter ("computed")
+ public double getComputedTime() {
+ return this.computed;
+ }
+ @JsonSetter ("computed")
+ public void setComputedTime(double computed) {
+ this.computed = computed;
+ }
+ }
+
+ /**
+ * Struct which contains the statistics for the entire directory structure
+ */
+ @JsonTypeName("v1")
+ public static class Statistics_v1 extends TableStatistics {
+ @JsonProperty ("directories")
+ List<DirectoryStatistics_v1> directoryStatistics;
+ // Default constructor required for deserializer
+ public Statistics_v1 () { }
+ @JsonGetter ("directories")
+ public List<DirectoryStatistics_v1> getDirectoryStatistics() {
+ return directoryStatistics;
+ }
+ @JsonSetter ("directories")
+ public void setDirectoryStatistics(List<DirectoryStatistics_v1> directoryStatistics) {
+ this.directoryStatistics = directoryStatistics;
+ }
+ }
+
+ public static class DirectoryStatistics_v1 extends DirectoryStatistics {
+ @JsonProperty private String computed;
+ @JsonProperty ("columns") private List<ColumnStatistics_v1> columnStatistics;
+ // Default constructor required for deserializer
+ public DirectoryStatistics_v1() { }
+ @JsonGetter ("computed")
+ public String getComputedTime() {
+ return this.computed;
+ }
+ @JsonSetter ("computed")
+ public void setComputedTime(String computed) {
+ this.computed = computed;
+ }
+ @JsonGetter ("columns")
+ public List<ColumnStatistics_v1> getColumnStatistics() {
+ return this.columnStatistics;
+ }
+ @JsonSetter ("columns")
+ public void setColumnStatistics(List<ColumnStatistics_v1> columnStatistics) {
+ this.columnStatistics = columnStatistics;
+ }
+ }
+
+ public static class ColumnStatistics_v1 extends ColumnStatistics {
+ @JsonProperty ("column") private String name = null;
+ @JsonProperty ("majortype") private TypeProtos.MajorType type = null;
+ @JsonProperty ("schema") private long schema = 0;
+ @JsonProperty ("rowcount") private long count = 0;
+ @JsonProperty ("nonnullrowcount") private long nonNullCount = 0;
+ @JsonProperty ("ndv") private long ndv = 0;
+ @JsonProperty ("avgwidth") private double width = 0;
+
+ public ColumnStatistics_v1() {}
+ @JsonGetter ("column")
+ public String getName() { return this.name; }
+ @JsonSetter ("column")
+ public void setName(String name) {
+ this.name = name;
+ }
+ @JsonGetter ("majortype")
+ public TypeProtos.MajorType getType() { return this.type; }
+ @JsonSetter ("type")
+ public void setType(TypeProtos.MajorType type) {
+ this.type = type;
+ }
+ @JsonGetter ("schema")
+ public double getSchema() {
+ return this.schema;
+ }
+ @JsonSetter ("schema")
+ public void setSchema(long schema) {
+ this.schema = schema;
+ }
+ @JsonGetter ("rowcount")
+ public double getCount() {
+ return this.count;
+ }
+ @JsonSetter ("rowcount")
+ public void setCount(long count) {
+ this.count = count;
+ }
+ @JsonGetter ("nonnullrowcount")
+ public double getNonNullCount() {
+ return this.nonNullCount;
+ }
+ @JsonSetter ("nonnullrowcount")
+ public void setNonNullCount(long nonNullCount) {
+ this.nonNullCount = nonNullCount;
+ }
+ @JsonGetter ("ndv")
+ public long getNdv() {
+ return this.ndv;
+ }
+ @JsonSetter ("ndv")
+ public void setNdv(long ndv) { this.ndv = ndv; }
+ @JsonGetter ("avgwidth")
+ public double getAvgWidth() {
+ return this.width;
+ }
+ @JsonSetter ("avgwidth")
+ public void setAvgWidth(double width) { this.width = width; }
+ }
+
+ private TableStatistics readStatistics(DrillTable drillTable, Path path) throws IOException {
+ final Object selection = drillTable.getSelection();
+
+ if (selection instanceof FormatSelection) {
+ StoragePlugin storagePlugin = drillTable.getPlugin();
+ FormatSelection formatSelection = (FormatSelection) selection;
+ FormatPluginConfig formatConfig = formatSelection.getFormat();
+
+ if (storagePlugin instanceof FileSystemPlugin
+ && (formatConfig instanceof ParquetFormatConfig)) {
+ FormatPlugin fmtPlugin = storagePlugin.getFormatPlugin(formatConfig);
+ if (fmtPlugin.supportsStatistics()) {
+ return fmtPlugin.readStatistics(fs, path);
+ }
+ }
+ }
+ return null;
+ }
+
+ public static TableStatistics generateDirectoryStructure(String dirComputedTime,
+ List<ColumnStatistics> columnStatisticsList) {
+ // TODO: Split up columnStatisticsList() based on directory names. We assume only
+ // one directory right now but this WILL change in the future
+ // HashMap<String, Boolean> dirNames = new HashMap<String, Boolean>();
+ TableStatistics statistics = new Statistics_v1();
+ List<DirectoryStatistics_v1> dirStats = new ArrayList<DirectoryStatistics_v1>();
+ List<ColumnStatistics_v1> columnStatisticsV1s = new ArrayList<DrillStatsTable.ColumnStatistics_v1>();
+ // Create dirStats
+ DirectoryStatistics_v1 dirStat = new DirectoryStatistics_v1();
+ // Add columnStats corresponding to this dirStats
+ for (ColumnStatistics colStats : columnStatisticsList) {
+ columnStatisticsV1s.add((ColumnStatistics_v1) colStats);
+ }
+ dirStat.setComputedTime(dirComputedTime);
+ dirStat.setColumnStatistics(columnStatisticsV1s);
+ // Add this dirStats to the list of dirStats
+ dirStats.add(dirStat);
+ // Add list of dirStats to tableStats
+ ((Statistics_v1) statistics).setDirectoryStatistics(dirStats);
+ return statistics;
+ }
+
+ public static PhysicalPlan direct(QueryContext context, boolean outcome, String message, Object... values) {
+ return DirectPlan.createDirectPlan(context, outcome, String.format(message, values));
+ }
+
+ /* Helper function to generate error - statistics not supported on non-parquet tables */
+ public static PhysicalPlan notSupported(QueryContext context, String tbl) {
+ return direct(context, false, "Table %s is not supported by ANALYZE."
+ + " Support is currently limited to directory-based Parquet tables.", tbl);
+ }
+
+ public static PhysicalPlan notRequired(QueryContext context, String tbl) {
+ return direct(context, false, "Table %s has not changed since last ANALYZE!", tbl);
+ }
+
+ /**
+ * This method returns the statistics (de)serializer which can be used to (de)/serialize the
+ * {@link TableStatistics} from/to JSON
+ */
+ public static ObjectMapper getMapper() {
+ ObjectMapper mapper = new ObjectMapper();
+ SimpleModule deModule = new SimpleModule("StatisticsSerDeModule")
+ .addSerializer(TypeProtos.MajorType.class, new MajorTypeSerDe.Se())
+ .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+ mapper.registerModule(deModule);
+ return mapper;
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
index 13f16008c..6380b523c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
@@ -17,23 +17,35 @@
*/
package org.apache.drill.exec.planner.cost;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Window;
import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
-import org.apache.calcite.rel.metadata.RelColumnOrigin;
import org.apache.calcite.rel.metadata.RelMdDistinctRowCount;
+import org.apache.calcite.rel.metadata.RelMdUtil;
import org.apache.calcite.rel.metadata.RelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
import org.apache.calcite.util.BuiltInMethod;
import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
import org.apache.drill.exec.planner.common.DrillStatsTable;
-import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
+import java.util.ArrayList;
import java.util.List;
-public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
+public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
private static final DrillRelMdDistinctRowCount INSTANCE =
new DrillRelMdDistinctRowCount();
@@ -61,15 +73,35 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
return getDistinctRowCount((RelNode) rel, mq, groupKey, predicate);
}
- public Double getDistinctRowCount(DrillScanRel scan, RelMetadataQuery mq,
- ImmutableBitSet groupKey, RexNode predicate) {
- // Consistent with the estimation of Aggregate row count in RelMdRowCount : distinctRowCount = rowCount * 10%.
- return scan.estimateRowCount(mq) * 0.1;
- }
-
+ @Override
public Double getDistinctRowCount(RelNode rel, RelMetadataQuery mq, ImmutableBitSet groupKey, RexNode predicate) {
- if (rel instanceof DrillScanRel) {
- return getDistinctRowCount((DrillScanRel) rel, mq, groupKey);
+ if (rel instanceof TableScan && !DrillRelOptUtil.guessRows(rel)) {
+ return getDistinctRowCount((TableScan) rel, mq, groupKey, predicate);
+ } else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
+ if (rel instanceof Window) {
+ int childFieldCount = ((Window)rel).getInput().getRowType().getFieldCount();
+ // For window aggregates delegate ndv to parent
+ for (int bit : groupKey) {
+ if (bit >= childFieldCount) {
+ return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+ }
+ }
+ }
+ return mq.getDistinctRowCount(((SingleRel) rel).getInput(), groupKey, predicate);
+ } else if (rel instanceof DrillJoinRelBase) {
+ if (DrillRelOptUtil.guessRows(rel)) {
+ return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+ }
+ //Assume ndv is unaffected by the join
+ return getDistinctRowCount(((DrillJoinRelBase) rel), mq, groupKey, predicate);
+ } else if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
+ if (((RelSubset) rel).getBest() != null) {
+ return mq.getDistinctRowCount(((RelSubset)rel).getBest(), groupKey, predicate);
+ } else if (((RelSubset) rel).getOriginal() != null) {
+ return mq.getDistinctRowCount(((RelSubset)rel).getOriginal(), groupKey, predicate);
+ } else {
+ return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+ }
} else {
return super.getDistinctRowCount(rel, mq, groupKey, predicate);
}
@@ -80,37 +112,120 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
* set of columns indicated by groupKey.
* column").
*/
- private Double getDistinctRowCount(DrillScanRel scan, RelMetadataQuery mq, ImmutableBitSet groupKey) {
- if (scan.getDrillTable() == null || scan.getDrillTable().getStatsTable() == null) {
- // If there is no table or metadata (stats) table associated with scan, estimate the distinct row count.
- // Consistent with the estimation of Aggregate row count in RelMdRowCount : distinctRowCount = rowCount * 10%.
- return scan.getRows() * 0.1;
+ private Double getDistinctRowCount(TableScan scan, RelMetadataQuery mq, ImmutableBitSet groupKey,
+ RexNode predicate) {
+ DrillTable table = scan.getTable().unwrap(DrillTable.class);
+ if (table == null) {
+ table = scan.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
+ }
+ return getDistinctRowCountInternal(scan, mq, table, groupKey, scan.getRowType(), predicate);
+ }
+
+ private Double getDistinctRowCountInternal(RelNode scan, RelMetadataQuery mq, DrillTable table,
+ ImmutableBitSet groupKey, RelDataType type, RexNode predicate) {
+ double selectivity, rowCount;
+ // If guessing, return NDV as 0.1 * rowCount
+ if (DrillRelOptUtil.guessRows(scan)) {
+ /* If there is no table or metadata (stats) table associated with scan, estimate the
+ * distinct row count. Consistent with the estimation of Aggregate row count in
+ * RelMdRowCount: distinctRowCount = rowCount * 10%.
+ */
+ return scan.estimateRowCount(mq) * 0.1;
}
- // TODO: may be we should get the column origin of each group by key before we look up it in metadata table?
- List<RelColumnOrigin> cols = Lists.newArrayList();
+ /* If predicate is present, determine its selectivity to estimate filtered rows.
+ * Thereafter, compute the number of distinct rows.
+ */
+ selectivity = mq.getSelectivity(scan, predicate);
+ rowCount = mq.getRowCount(scan);
if (groupKey.length() == 0) {
- return new Double(0);
+ return selectivity*rowCount;
}
- DrillStatsTable md = scan.getDrillTable().getStatsTable();
-
- final double rc = mq.getRowCount(scan);
+ /* If predicate is present, determine its selectivity to estimate filtered rows. Thereafter,
+ * compute the number of distinct rows
+ */
+ selectivity = mq.getSelectivity(scan, predicate);
+ DrillStatsTable md = table.getStatsTable();
double s = 1.0;
+
for (int i = 0; i < groupKey.length(); i++) {
- final String colName = scan.getRowType().getFieldNames().get(i);
- if (!groupKey.get(i) && colName.equals("*")) {
+ final String colName = type.getFieldNames().get(i);
+ // Skip NDV, if not available
+ if (!groupKey.get(i)) {
continue;
}
-
Double d = md.getNdv(colName);
if (d == null) {
continue;
}
+ s *= 1 - d / rowCount;
+ }
+ if (s > 0 && s < 1.0) {
+ return (1 - s) * selectivity * rowCount;
+ } else if (s == 1.0) {
+ // Could not get any NDV estimate from stats - probably stats not present for GBY cols. So Guess!
+ return scan.estimateRowCount(mq) * 0.1;
+ } else {
+ /* rowCount maybe less than NDV(different source), sanity check OR NDV not used at all */
+ return selectivity * rowCount;
+ }
+ }
+
+ public Double getDistinctRowCount(DrillJoinRelBase joinRel, RelMetadataQuery mq, ImmutableBitSet groupKey,
+ RexNode predicate) {
+ if (DrillRelOptUtil.guessRows(joinRel)) {
+ return super.getDistinctRowCount(joinRel, mq, groupKey, predicate);
+ }
+ // Assume NDV is unaffected by the join when groupKey comes from one side of the join
+ // Alleviates NDV over-estimates
+ ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
+ ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
+ JoinRelType joinType = joinRel.getJoinType();
+ RelNode left = joinRel.getInputs().get(0);
+ RelNode right = joinRel.getInputs().get(1);
+ RelMdUtil.setLeftRightBitmaps(groupKey, leftMask, rightMask,
+ left.getRowType().getFieldCount());
+ RexNode leftPred = null;
+ RexNode rightPred = null;
- s *= 1 - d / rc;
+ // Identify predicates which can be pushed onto the left and right sides of the join
+ if (predicate != null) {
+ ArrayList leftFilters = new ArrayList();
+ ArrayList rightFilters = new ArrayList();
+ ArrayList joinFilters = new ArrayList();
+ List predList = RelOptUtil.conjunctions(predicate);
+ RelOptUtil.classifyFilters(joinRel, predList, joinType, joinType == JoinRelType.INNER,
+ !joinType.generatesNullsOnLeft(), !joinType.generatesNullsOnRight(), joinFilters,
+ leftFilters, rightFilters);
+ RexBuilder rexBuilder = joinRel.getCluster().getRexBuilder();
+ leftPred = RexUtil.composeConjunction(rexBuilder, leftFilters, true);
+ rightPred = RexUtil.composeConjunction(rexBuilder, rightFilters, true);
+ }
+
+ Double leftDistRowCount = null;
+ Double rightDistRowCount = null;
+ double distRowCount = 1;
+ ImmutableBitSet lmb = leftMask.build();
+ ImmutableBitSet rmb = rightMask.build();
+ // Get NDV estimates for the left and right side predicates, if applicable
+ if (lmb.length() > 0) {
+ leftDistRowCount = mq.getDistinctRowCount(left, lmb, leftPred);
+ if (leftDistRowCount != null) {
+ distRowCount = leftDistRowCount.doubleValue();
+ }
+ }
+ if (rmb.length() > 0) {
+ rightDistRowCount = mq.getDistinctRowCount(right, rmb, rightPred);
+ if (rightDistRowCount != null) {
+ distRowCount = rightDistRowCount.doubleValue();
+ }
+ }
+ // Use max of NDVs from both sides of the join, if applicable
+ if (leftDistRowCount != null && rightDistRowCount != null) {
+ distRowCount = Math.max(leftDistRowCount, rightDistRowCount);
}
- return new Double((1 - s) * rc);
+ return RelMdUtil.numDistinctVals(distRowCount, mq.getRowCount(joinRel));
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
index 343affb91..f85448040 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
@@ -17,14 +17,17 @@
*/
package org.apache.drill.exec.planner.cost;
+import java.io.IOException;
+
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.SingleRel;
import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.Join;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.core.Sort;
import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMdRowCount;
import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -32,10 +35,14 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.calcite.util.BuiltInMethod;
import org.apache.calcite.util.ImmutableBitSet;
import org.apache.drill.exec.planner.common.DrillLimitRelBase;
-import org.apache.drill.exec.planner.common.DrillStatsTable;
-import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.PrelUtil;
-public class DrillRelMdRowCount extends RelMdRowCount {
+
+public class DrillRelMdRowCount extends RelMdRowCount{
private static final DrillRelMdRowCount INSTANCE = new DrillRelMdRowCount();
public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.ROW_COUNT.method, INSTANCE);
@@ -51,11 +58,6 @@ public class DrillRelMdRowCount extends RelMdRowCount {
}
}
- @Override
- public Double getRowCount(Filter rel, RelMetadataQuery mq) {
- return rel.estimateRowCount(mq);
- }
-
public double getRowCount(DrillLimitRelBase rel, RelMetadataQuery mq) {
return rel.estimateRowCount(mq);
}
@@ -86,18 +88,45 @@ public class DrillRelMdRowCount extends RelMdRowCount {
}
public Double getRowCount(RelNode rel, RelMetadataQuery mq) {
- if (rel instanceof DrillScanRel) {
- return getRowCount((DrillScanRel)rel, mq);
+ if (rel instanceof TableScan) {
+ return getRowCountInternal((TableScan)rel, mq);
}
return super.getRowCount(rel, mq);
}
- private Double getRowCount(DrillScanRel scanRel, RelMetadataQuery mq) {
- final DrillStatsTable md = scanRel.getDrillTable().getStatsTable();
- if (md != null) {
- return md.getRowCount();
- }
+ @Override
+ public Double getRowCount(Filter rel, RelMetadataQuery mq) {
+ // Need capped selectivity estimates. See the Filter getRows() method
+ return rel.getRows();
+ }
- return super.getRowCount(scanRel, mq);
+ private Double getRowCountInternal(TableScan rel, RelMetadataQuery mq) {
+ DrillTable table;
+ PlannerSettings settings = PrelUtil.getSettings(rel.getCluster());
+ // If guessing, return selectivity from RelMDRowCount
+ if (DrillRelOptUtil.guessRows(rel)) {
+ return super.getRowCount(rel, mq);
+ }
+ table = rel.getTable().unwrap(DrillTable.class);
+ if (table == null) {
+ table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
+ }
+ // Return rowcount from statistics, if available. Otherwise, delegate to parent.
+ try {
+ if (table != null
+ && table.getStatsTable() != null
+ && table.getStatsTable().isMaterialized()
+ /* For GroupScan rely on accurate count from the scan, if available, instead of
+ * statistics since partition pruning/filter pushdown might have occurred.
+ * e.g. ParquetGroupScan returns accurate rowcount. The other way would be to
+ * iterate over the rowgroups present in the GroupScan to compute the rowcount.
+ */
+ && !(table.getGroupScan().getScanStats(settings).getGroupScanProperty().hasExactRowCount())) {
+ return table.getStatsTable().getRowCount();
+ }
+ } catch (IOException ex) {
+ return super.getRowCount(rel, mq);
+ }
+ return super.getRowCount(rel, mq);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
index da4222949..e074be47e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
@@ -17,55 +17,75 @@
*/
package org.apache.drill.exec.planner.cost;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.plan.volcano.RelSubset;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMdSelectivity;
import org.apache.calcite.rel.metadata.RelMdUtil;
import org.apache.calcite.rel.metadata.RelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.exec.physical.base.DbGroupScan;
import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.planner.physical.PrelUtil;
import org.apache.drill.exec.planner.physical.ScanPrel;
-import java.util.List;
-
public class DrillRelMdSelectivity extends RelMdSelectivity {
private static final DrillRelMdSelectivity INSTANCE = new DrillRelMdSelectivity();
-
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRelMdSelectivity.class);
public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.SELECTIVITY.method, INSTANCE);
-
- public Double getSelectivity(RelNode rel, RexNode predicate) {
- if (rel instanceof RelSubset) {
- return getSelectivity((RelSubset) rel, predicate);
- } else if (rel instanceof DrillScanRel) {
- return getScanSelectivity(rel, predicate);
- } else if (rel instanceof ScanPrel) {
- return getScanSelectivity(rel, predicate);
- }else {
- return super.getSelectivity(rel, RelMetadataQuery.instance(), predicate);
+ @Override
+ public Double getSelectivity(RelNode rel, RelMetadataQuery mq, RexNode predicate) {
+ if (rel instanceof RelSubset && !DrillRelOptUtil.guessRows(rel)) {
+ return getSubsetSelectivity((RelSubset) rel, mq, predicate);
+ } else if (rel instanceof TableScan) {
+ return getScanSelectivity(rel, mq, predicate);
+ } else if (rel instanceof DrillJoinRelBase) {
+ return getJoinSelectivity(((DrillJoinRelBase) rel), mq, predicate);
+ } /*else if (rel instanceof SingleRel && !DrillRelOptUtil.guessRows(rel)) {
+ return getSelectivity(((SingleRel)rel).getInput(), mq, predicate);
+ }*/ else {
+ return super.getSelectivity(rel, mq, predicate);
}
}
- private Double getSelectivity(RelSubset rel, RexNode predicate) {
+ private Double getSubsetSelectivity(RelSubset rel, RelMetadataQuery mq, RexNode predicate) {
if (rel.getBest() != null) {
- return getSelectivity(rel.getBest(), predicate);
+ return getSelectivity(rel.getBest(), mq, predicate);
} else {
List<RelNode> list = rel.getRelList();
if (list != null && list.size() > 0) {
- return getSelectivity(list.get(0), predicate);
+ return getSelectivity(list.get(0), mq, predicate);
}
}
+ //TODO: Not required? return mq.getSelectivity(((RelSubset)rel).getOriginal(), predicate);
return RelMdUtil.guessSelectivity(predicate);
}
- private Double getScanSelectivity(RelNode rel, RexNode predicate) {
+ private Double getScanSelectivity(RelNode rel, RelMetadataQuery mq, RexNode predicate) {
double ROWCOUNT_UNKNOWN = -1.0;
GroupScan scan = null;
PlannerSettings settings = PrelUtil.getPlannerSettings(rel.getCluster().getPlanner());
@@ -85,6 +105,151 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
}
}
}
- return super.getSelectivity(rel, RelMetadataQuery.instance(), predicate);
+ // Do not mess with statistics used for DBGroupScans.
+ if (rel instanceof TableScan) {
+ if (DrillRelOptUtil.guessRows(rel)) {
+ return super.getSelectivity(rel, mq, predicate);
+ }
+ DrillTable table = rel.getTable().unwrap(DrillTable.class);
+ if (table == null) {
+ table = rel.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
+ }
+ if (table != null && table.getStatsTable() != null && table.getStatsTable().isMaterialized()) {
+ if (rel instanceof DrillScanRelBase) {
+ List<String> fieldNames = new ArrayList<>();
+ for (SchemaPath fieldPath : ((DrillScanRelBase)rel).getGroupScan().getColumns()) {
+ fieldNames.add(fieldPath.toString());
+ }
+ return getScanSelectivityInternal(table, predicate, fieldNames);
+ } else {
+ return getScanSelectivityInternal(table, predicate, rel.getRowType().getFieldNames());
+ }
+ }
+ }
+ return super.getSelectivity(rel, mq, predicate);
+ }
+
+ private double getScanSelectivityInternal(DrillTable table, RexNode predicate, List<String> fieldNames) {
+ double sel = 1.0;
+ if ((predicate == null) || predicate.isAlwaysTrue()) {
+ return sel;
+ }
+ for (RexNode pred : RelOptUtil.conjunctions(predicate)) {
+ double orSel = 0;
+ for (RexNode orPred : RelOptUtil.disjunctions(pred)) {
+ //CALCITE guess
+ Double guess = RelMdUtil.guessSelectivity(pred);
+ if (orPred.isA(SqlKind.EQUALS)) {
+ if (orPred instanceof RexCall) {
+ int colIdx = -1;
+ RexInputRef op = findRexInputRef(orPred);
+ if (op != null) {
+ colIdx = op.hashCode();
+ }
+ if (colIdx != -1 && colIdx < fieldNames.size()) {
+ String col = fieldNames.get(colIdx);
+ if (table.getStatsTable() != null
+ && table.getStatsTable().getNdv(col) != null) {
+ orSel += 1.00 / table.getStatsTable().getNdv(col);
+ } else {
+ orSel += guess;
+ }
+ } else {
+ orSel += guess;
+ if (logger.isDebugEnabled()) {
+ logger.warn(String.format("No input reference $[%s] found for predicate [%s]",
+ Integer.toString(colIdx), orPred.toString()));
+ }
+ }
+ }
+ } else {
+ //Use the CALCITE guess. TODO: Use histograms for COMPARISON operator
+ orSel += guess;
+ }
+ }
+ sel *= orSel;
+ }
+ // Cap selectivity if it exceeds 1.0
+ return (sel > 1.0) ? 1.0 : sel;
+ }
+
+ private Double getJoinSelectivity(DrillJoinRelBase rel, RelMetadataQuery mq, RexNode predicate) {
+ double sel = 1.0;
+ // determine which filters apply to the left vs right
+ RexNode leftPred, rightPred;
+ JoinRelType joinType = rel.getJoinType();
+ final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+ int[] adjustments = new int[rel.getRowType().getFieldCount()];
+
+ if (DrillRelOptUtil.guessRows(rel)) {
+ return super.getSelectivity(rel, mq, predicate);
+ }
+
+ if (predicate != null) {
+ RexNode pred;
+ List<RexNode> leftFilters = new ArrayList<>();
+ List<RexNode> rightFilters = new ArrayList<>();
+ List<RexNode> joinFilters = new ArrayList<>();
+ List<RexNode> predList = RelOptUtil.conjunctions(predicate);
+
+ RelOptUtil.classifyFilters(
+ rel,
+ predList,
+ joinType,
+ joinType == JoinRelType.INNER,
+ !joinType.generatesNullsOnLeft(),
+ !joinType.generatesNullsOnRight(),
+ joinFilters,
+ leftFilters,
+ rightFilters);
+ leftPred =
+ RexUtil.composeConjunction(rexBuilder, leftFilters, true);
+ rightPred =
+ RexUtil.composeConjunction(rexBuilder, rightFilters, true);
+ for (RelNode child : rel.getInputs()) {
+ RexNode modifiedPred = null;
+
+ if (child == rel.getLeft()) {
+ pred = leftPred;
+ } else {
+ pred = rightPred;
+ }
+ if (pred != null) {
+ // convert the predicate to reference the types of the children
+ modifiedPred =
+ pred.accept(new RelOptUtil.RexInputConverter(
+ rexBuilder,
+ null,
+ child.getRowType().getFieldList(),
+ adjustments));
+ }
+ sel *= mq.getSelectivity(child, modifiedPred);
+ }
+ sel *= RelMdUtil.guessSelectivity(RexUtil.composeConjunction(rexBuilder, joinFilters, true));
+ }
+ return sel;
+ }
+
+ public static RexInputRef findRexInputRef(final RexNode node) {
+ try {
+ RexVisitor<Void> visitor =
+ new RexVisitorImpl<Void>(true) {
+ public Void visitCall(RexCall call) {
+ for (RexNode child : call.getOperands()) {
+ child.accept(this);
+ }
+ return super.visitCall(call);
+ }
+
+ public Void visitInputRef(RexInputRef inputRef) {
+ throw new Util.FoundOne(inputRef);
+ }
+ };
+ node.accept(visitor);
+ return null;
+ } catch (Util.FoundOne e) {
+ Util.swallow(e, null);
+ return (RexInputRef) e.getNode();
+ }
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
index 5d570f3f2..9688cb2e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.planner.logical;
+import java.util.List;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
@@ -31,15 +32,16 @@ import org.apache.drill.common.logical.data.LogicalOperator;
import org.apache.drill.exec.planner.cost.DrillCostBase;
import org.apache.drill.exec.planner.torel.ConversionContext;
-import java.util.List;
-
/**
* Drill logical node for "Analyze".
*/
public class DrillAnalyzeRel extends SingleRel implements DrillRel {
- public DrillAnalyzeRel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+ double samplePercent; // sampling percentage between 0-100
+
+ public DrillAnalyzeRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, double samplePercent) {
super(cluster, traits, child);
+ this.samplePercent = samplePercent;
}
@Override
@@ -52,7 +54,7 @@ public class DrillAnalyzeRel extends SingleRel implements DrillRel {
@Override
public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
- return new DrillAnalyzeRel(getCluster(), traitSet, sole(inputs));
+ return new DrillAnalyzeRel(getCluster(), traitSet, sole(inputs), samplePercent);
}
@Override
@@ -60,12 +62,16 @@ public class DrillAnalyzeRel extends SingleRel implements DrillRel {
final LogicalOperator inputOp = implementor.visitChild(this, 0, getInput());
final Analyze rel = new Analyze();
rel.setInput(inputOp);
-
return rel;
}
- public static DrillAnalyzeRel convert(Analyze analyze, ConversionContext context) throws InvalidRelException {
+ public double getSamplePercent() {
+ return samplePercent;
+ }
+
+ public static DrillAnalyzeRel convert(Analyze analyze, ConversionContext context)
+ throws InvalidRelException {
RelNode input = context.toRel(analyze.getInput());
- return new DrillAnalyzeRel(context.getCluster(), context.getLogicalTraits(), input);
+ return new DrillAnalyzeRel(context.getCluster(), context.getLogicalTraits(), input, analyze.getSamplePercent());
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index afddbfc43..4463d183e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -46,9 +46,10 @@ public abstract class DrillTable implements Table {
private final Object selection;
private final StoragePlugin plugin;
private final String userName;
- private DrillStatsTable statsTable;
private GroupScan scan;
private SessionOptionManager options;
+ // Stores the statistics(rowcount, NDV etc.) associated with the table
+ private DrillStatsTable statsTable;
/**
* Creates a DrillTable instance for a @{code TableType#Table} table.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
index 6869616c8..23ea23fd5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
@@ -46,7 +46,6 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
private FileSystemConfig storageConfig;
private FormatPlugin formatPlugin;
private String location;
- private boolean append;
private final List<String> partitionColumns;
private final StorageStrategy storageStrategy;
@@ -54,7 +53,6 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
public FileSystemCreateTableEntry(@JsonProperty("storageConfig") FileSystemConfig storageConfig,
@JsonProperty("formatConfig") FormatPluginConfig formatConfig,
@JsonProperty("location") String location,
- @JsonProperty("append") boolean append,
@JsonProperty("partitionColumn") List<String> partitionColumns,
@JsonProperty("storageStrategy") StorageStrategy storageStrategy,
@JacksonInject StoragePluginRegistry engineRegistry)
@@ -69,13 +67,11 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
public FileSystemCreateTableEntry(FileSystemConfig storageConfig,
FormatPlugin formatPlugin,
String location,
- boolean append,
List<String> partitionColumns,
StorageStrategy storageStrategy) {
this.storageConfig = storageConfig;
this.formatPlugin = formatPlugin;
this.location = location;
- this.append = append;
this.partitionColumns = partitionColumns;
this.storageStrategy = storageStrategy;
}
@@ -98,7 +94,7 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
formatPlugin.getName())).build(logger);
}
- AbstractWriter writer = formatPlugin.getWriter(child, location, append, partitionColumns);
+ AbstractWriter writer = formatPlugin.getWriter(child, location, partitionColumns);
writer.setStorageStrategy(storageStrategy);
return writer;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
index 44ea9856a..b373ff449 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
@@ -46,7 +46,7 @@ public abstract class AggPruleBase extends Prule {
groupByFields.add(field);
if (!allFields && groupByFields.size() == 1) {
- // if we are only interested in 1 grouping field, pick the first one for now..
+ // TODO: if we are only interested in 1 grouping field, pick the first one for now..
// but once we have num distinct values (NDV) statistics, we should pick the one
// with highest NDV.
break;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
index 4cac5d946..cf41cf8aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,48 +17,115 @@
*/
package org.apache.drill.exec.planner.physical;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import java.math.BigDecimal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.SingleRel;
-import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.statistics.Statistic;
import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
import org.apache.drill.exec.planner.logical.DrillRel;
import org.apache.drill.exec.planner.logical.RelOptHelper;
-
-import java.util.List;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
public class AnalyzePrule extends Prule {
+
public static final RelOptRule INSTANCE = new AnalyzePrule();
+ // List of output functions (from StatsAggBatch)
+ private static final List<String> PHASE_1_FUNCTIONS = ImmutableList.of(
+ Statistic.ROWCOUNT, // total number of entries in table fragment
+ Statistic.NNROWCOUNT, // total number of non-null entries in table fragment
+ Statistic.SUM_WIDTH, // total column width across all entries in table fragment
+ Statistic.CNT_DUPS, // total count of non-singletons in table fragment
+ Statistic.HLL // total distinct values in table fragment
+ );
+
+ // Mapping between output functions (from StatsMergeBatch) and
+ // input functions (from StatsAggBatch)
+ private static Map<String, String> PHASE_2_FUNCTIONS = new HashMap<>();
+ static {
+ PHASE_2_FUNCTIONS.put(Statistic.ROWCOUNT, Statistic.ROWCOUNT);
+ PHASE_2_FUNCTIONS.put(Statistic.NNROWCOUNT, Statistic.NNROWCOUNT);
+ PHASE_2_FUNCTIONS.put(Statistic.AVG_WIDTH, Statistic.SUM_WIDTH);
+ PHASE_2_FUNCTIONS.put(Statistic.SUM_DUPS, Statistic.CNT_DUPS);
+ PHASE_2_FUNCTIONS.put(Statistic.HLL_MERGE, Statistic.HLL);
+ PHASE_2_FUNCTIONS.put(Statistic.NDV, Statistic.HLL);
+ }
- private static final List<String> FUNCTIONS = ImmutableList.of(
- "statcount", // total number of entries in the table
- "nonnullstatcount", // total number of non-null entries in the table
- "ndv", // total distinctive values in table
- "hll" // HyperLogLog
+ // List of input functions (from StatsMergeBatch) to UnpivotMapsBatch
+ private static final List<String> UNPIVOT_FUNCTIONS = ImmutableList.of(
+ Statistic.ROWCOUNT, // total number of entries in the table
+ Statistic.NNROWCOUNT, // total number of non-null entries in the table
+ Statistic.AVG_WIDTH, // average column width across all entries in the table
+ Statistic.HLL_MERGE, // total distinct values(computed using hll) in the table
+ Statistic.SUM_DUPS, // total count of duplicate values across all entries in the table
+ Statistic.NDV // total distinct values across all entries in the table
);
public AnalyzePrule() {
- super(RelOptHelper.some(DrillAnalyzeRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "Prel.AnalyzePrule");
+ super(RelOptHelper.some(DrillAnalyzeRel.class, DrillRel.DRILL_LOGICAL,
+ RelOptHelper.any(RelNode.class)), "Prel.AnalyzePrule");
}
@Override
public void onMatch(RelOptRuleCall call) {
- final DrillAnalyzeRel analyze = (DrillAnalyzeRel) call.rel(0);
+ final DrillAnalyzeRel analyze = call.rel(0);
final RelNode input = call.rel(1);
+ final SingleRel newAnalyze;
+ final RelTraitSet singleDistTrait = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL)
+ .plus(DrillDistributionTrait.SINGLETON);
- final RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
- final RelNode convertedInput = convert(input, traits);
-
- final StatsAggPrel statsAggPrel = new StatsAggPrel(convertedInput, analyze.getCluster(), FUNCTIONS);
-
- final List<String> mapFileds = Lists.newArrayList(FUNCTIONS);
- mapFileds.add(DrillStatsTable.COL_COLUMN);
- final SingleRel newAnalyze = new UnpivotMapsPrel(statsAggPrel, analyze.getCluster(), mapFileds);
+ // Generate parallel ANALYZE plan:
+ // Writer<-Unpivot<-StatsAgg(Phase2)<-Exchange<-StatsAgg(Phase1)<-Scan
+ final RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).
+ plus(DrillDistributionTrait.DEFAULT);
+ RelNode convertedInput = convert(input, traits);
+ final List<String> mapFields1 = Lists.newArrayList(PHASE_1_FUNCTIONS);
+ final Map<String, String> mapFields2 = Maps.newHashMap(PHASE_2_FUNCTIONS);
+ final List<String> mapFields3 = Lists.newArrayList(UNPIVOT_FUNCTIONS);
+ mapFields1.add(0, Statistic.COLNAME);
+ mapFields1.add(1, Statistic.COLTYPE);
+ mapFields2.put(Statistic.COLNAME, Statistic.COLNAME);
+ mapFields2.put(Statistic.COLTYPE, Statistic.COLTYPE);
+ mapFields3.add(0, Statistic.COLNAME);
+ mapFields3.add(1, Statistic.COLTYPE);
+ // Now generate the two phase plan physical operators bottom-up:
+ // STATSAGG->EXCHANGE->STATSMERGE->UNPIVOT
+ if (analyze.getSamplePercent() < 100.0) {
+ // If a sample samplePercent is specified add a filter for Bernoulli sampling
+ RexBuilder builder = convertedInput.getCluster().getRexBuilder();
+ RexNode sampleCondition;
+ if (PrelUtil.getSettings(convertedInput.getCluster()).getOptions().getOption(ExecConstants.DETERMINISTIC_SAMPLING_VALIDATOR)) {
+ sampleCondition = builder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+ builder.makeCall(SqlStdOperatorTable.RAND, builder.makeExactLiteral(BigDecimal.valueOf(1))),
+ builder.makeExactLiteral(BigDecimal.valueOf(analyze.getSamplePercent()/100.0)));
+ } else {
+ sampleCondition = builder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+ builder.makeCall(SqlStdOperatorTable.RAND),
+ builder.makeExactLiteral(BigDecimal.valueOf(analyze.getSamplePercent()/100.0)));
+ }
+ convertedInput = new FilterPrel(convertedInput.getCluster(), convertedInput.getTraitSet(),
+ convertedInput, sampleCondition);
+ }
+ final StatsAggPrel statsAggPrel = new StatsAggPrel(analyze.getCluster(), traits,
+ convertedInput, PHASE_1_FUNCTIONS);
+ UnionExchangePrel exch = new UnionExchangePrel(statsAggPrel.getCluster(), singleDistTrait,
+ statsAggPrel);
+ final StatsMergePrel statsMergePrel = new StatsMergePrel(exch.getCluster(), singleDistTrait,
+ exch, mapFields2, analyze.getSamplePercent());
+ newAnalyze = new UnpivotMapsPrel(statsMergePrel.getCluster(), singleDistTrait, statsMergePrel,
+ mapFields3);
call.transformTo(newAnalyze);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
index d90316533..51db2868c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
@@ -128,8 +128,8 @@ public class ConvertCountToDirectScan extends Prule {
final ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, 1, 1, scanRowType.getFieldCount());
final GroupScan directScan = new MetadataDirectGroupScan(reader, oldGrpScan.getFiles(), scanStats);
- final DirectScanPrel newScan = new DirectScanPrel(scan.getCluster(),
- scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), directScan, scanRowType);
+ final DirectScanPrel newScan = DirectScanPrel.create(scan, scan.getTraitSet().plus(Prel.DRILL_PHYSICAL)
+ .plus(DrillDistributionTrait.SINGLETON), directScan, scanRowType);
final ProjectPrel newProject = new ProjectPrel(agg.getCluster(), agg.getTraitSet().plus(Prel.DRILL_PHYSICAL)
.plus(DrillDistributionTrait.SINGLETON), newScan, prepareFieldExpressions(scanRowType), agg.getRowType());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
index d03225594..dac84082e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrel.java
@@ -17,6 +17,11 @@
*/
package org.apache.drill.exec.planner.physical;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptCost;
import org.apache.calcite.plan.RelOptPlanner;
@@ -31,19 +36,14 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.physical.base.GroupScan;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.planner.cost.DrillCostBase;
import org.apache.drill.exec.planner.fragment.DistributionAffinity;
import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
import org.apache.drill.exec.record.BatchSchema;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
+import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
public class DirectScanPrel extends AbstractRelNode implements Prel, HasDistributionAffinity {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
- .getLogger(DirectScanPrel.class);
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectScanPrel.class);
private final GroupScan groupScan;
private final RelDataType rowType;
@@ -115,6 +115,12 @@ public class DirectScanPrel extends AbstractRelNode implements Prel, HasDistribu
return Collections.emptyIterator();
}
+ public static DirectScanPrel create(RelNode old, RelTraitSet traitSets,
+ GroupScan scan, RelDataType rowType) {
+ return new DirectScanPrel(old.getCluster(), traitSets,
+ getCopy(scan), rowType);
+ }
+
@Override
public RelDataType deriveRowType() {
return this.rowType;
@@ -147,15 +153,24 @@ public class DirectScanPrel extends AbstractRelNode implements Prel, HasDistribu
double rowCount = stats.getRecordCount();
-
+ // As DRILL-4083 points out, when columnCount == 0, cpuCost becomes zero,
+ // which makes the costs of HiveScan and HiveDrillNativeParquetScan the same
+ // For now, assume cpu cost is proportional to row count.
double cpuCost = rowCount * Math.max(columnCount, 1);
+ // If a positive value for CPU cost is given multiply the default CPU cost by given CPU cost.
if (stats.getCpuCost() > 0) {
cpuCost *= stats.getCpuCost();
}
+ // Even though scan is reading from disk, in the currently generated plans all plans will
+ // need to read the same amount of data, so keeping the disk io cost 0 is ok for now.
+ // In the future we might consider alternative scans that go against projections or
+ // different compression schemes etc that affect the amount of data read. Such alternatives
+ // would affect both cpu and io cost.
+ //double ioCost = 0;
double ioCost = stats.getDiskCost();
- DrillCostBase.DrillCostFactory costFactory = (DrillCostBase.DrillCostFactory)planner.getCostFactory();
+ DrillCostFactory costFactory = (DrillCostFactory)planner.getCostFactory();
return costFactory.makeCost(rowCount, cpuCost, ioCost, 0);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
index 19e9fa7e4..ac56c34af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DirectScanPrule.java
@@ -37,9 +37,6 @@ public class DirectScanPrule extends Prule {
final RelTraitSet traits = scan.getTraitSet().plus(Prel.DRILL_PHYSICAL);
final DirectScanPrel newScan = new DirectScanPrel(scan.getCluster(), traits, scan.getGroupScan(), scan.getRowType());
-
- call.transformTo(newScan);
-
call.transformTo(newScan);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
index 6caea820d..5eadab24e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
@@ -18,20 +18,18 @@
package org.apache.drill.exec.planner.physical;
import java.util.List;
-
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.trace.CalciteTrace;
import org.apache.drill.exec.physical.impl.join.JoinUtils;
import org.apache.drill.exec.physical.impl.join.JoinUtils.JoinCategory;
import org.apache.drill.exec.planner.logical.DrillJoin;
import org.apache.drill.exec.planner.logical.DrillJoinRel;
import org.apache.drill.exec.planner.logical.RelOptHelper;
-import org.apache.calcite.rel.InvalidRelException;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.util.trace.CalciteTrace;
-
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.slf4j.Logger;
@@ -81,8 +79,8 @@ public class NestedLoopJoinPrule extends JoinPruleBase {
if (!settings.isNestedLoopJoinEnabled()) {
return;
}
-
- final DrillJoinRel join = call.rel(0);
+ int[] joinFields = new int[2];
+ DrillJoinRel join = (DrillJoinRel) call.rel(0);
final RelNode left = join.getLeft();
final RelNode right = join.getRight();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index 95d77faf7..0a1812615 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -227,6 +227,8 @@ public class PlannerSettings implements Context{
public static final BooleanValidator FORCE_2PHASE_AGGR = new BooleanValidator(FORCE_2PHASE_AGGR_KEY,
new OptionDescription("Forces the cost-based query planner to generate a two phase aggregation for an aggregate operator."));
+ public static final BooleanValidator STATISTICS_USE = new BooleanValidator("planner.statistics.use", null);
+
public OptionManager options = null;
public FunctionImplementationRegistry functionImplementationRegistry = null;
@@ -467,6 +469,10 @@ public class PlannerSettings implements Context{
return options.getOption(INDEX_STATS_ROWCOUNT_SCALING_FACTOR);
}
+ public boolean useStatistics() {
+ return options.getOption(STATISTICS_USE);
+ }
+
@Override
public <T> T unwrap(Class<T> clazz) {
if(clazz == PlannerSettings.class){
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index 8a46f86b7..760a6ba5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -21,26 +21,25 @@ import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
-
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
import org.apache.drill.common.exceptions.DrillRuntimeException;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.physical.base.GroupScan;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
import org.apache.drill.exec.planner.fragment.DistributionAffinity;
import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.planner.common.DrillScanRelBase;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.type.RelDataType;
public class ScanPrel extends DrillScanRelBase implements Prel, HasDistributionAffinity {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
@@ -80,6 +79,12 @@ public class ScanPrel extends DrillScanRelBase implements Prel, HasDistributionA
return creator.addMetadata(this, this.getGroupScan());
}
+ public static ScanPrel create(RelNode old, RelTraitSet traitSets,
+ GroupScan scan, RelDataType rowType) {
+ return new ScanPrel(old.getCluster(), traitSets,
+ getCopy(scan), rowType, old.getTable());
+ }
+
@Override
public RelWriter explainTerms(RelWriter pw) {
return super.explainTerms(pw).item("groupscan", this.getGroupScan().getDigest());
@@ -110,8 +115,8 @@ public class ScanPrel extends DrillScanRelBase implements Prel, HasDistributionA
return planner.getCostFactory().makeCost(stats.getRecordCount() * columnCount, stats.getCpuCost(), stats.getDiskCost());
}
- // double rowCount = RelMetadataQuery.getRowCount(this);
- double rowCount = stats.getRecordCount();
+ double rowCount = mq.getRowCount(this);
+ //double rowCount = stats.getRecordCount();
// As DRILL-4083 points out, when columnCount == 0, cpuCost becomes zero,
// which makes the costs of HiveScan and HiveDrillNativeParquetScan the same
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
index 124246baf..3c3aeda92 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,7 +17,9 @@
*/
package org.apache.drill.exec.planner.physical;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
@@ -27,34 +29,28 @@ import org.apache.drill.exec.physical.config.StatisticsAggregate;
import org.apache.drill.exec.planner.common.DrillRelNode;
import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
public class StatsAggPrel extends SingleRel implements DrillRelNode, Prel {
private List<String> functions;
- public StatsAggPrel(RelNode child, RelOptCluster cluster, List<String> functions) {
- super(cluster, child.getTraitSet(), child);
+ public StatsAggPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<String> functions) {
+ super(cluster, traits, child);
this.functions = ImmutableList.copyOf(functions);
}
@Override
public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
- return new StatsAggPrel(sole(inputs), getCluster(), ImmutableList.copyOf(functions));
+ return new StatsAggPrel(getCluster(), traitSet, sole(inputs), ImmutableList.copyOf(functions));
}
@Override
public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
throws IOException {
Prel child = (Prel) this.getInput();
-
PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-
StatisticsAggregate g = new StatisticsAggregate(childPOP, functions);
-
return creator.addMetadata(this, g);
}
@@ -83,4 +79,5 @@ public class StatsAggPrel extends SingleRel implements DrillRelNode, Prel {
public boolean needsFinalColumnReordering() {
return true;
}
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsMergePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsMergePrel.java
new file mode 100644
index 000000000..5c8d4f9e2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsMergePrel.java
@@ -0,0 +1,89 @@
+/*
+ * 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.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class StatsMergePrel extends SingleRel implements DrillRelNode, Prel {
+
+ private Map<String, String> functions;
+ // Percent will be used to extrapolate statistics
+ private double samplePercent;
+
+ public StatsMergePrel(RelOptCluster cluster, RelTraitSet traits, RelNode child,
+ Map<String, String> functions, double samplePercent) {
+ super(cluster, traits, child);
+ this.functions = ImmutableMap.copyOf(functions);
+ this.samplePercent = samplePercent;
+ }
+
+ @Override
+ public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ return new StatsMergePrel(getCluster(), traitSet, sole(inputs), ImmutableMap.copyOf(functions), samplePercent);
+ }
+
+ @Override
+ public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
+ throws IOException {
+ Prel child = (Prel) this.getInput();
+ PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+ StatisticsMerge g = new StatisticsMerge(childPOP, functions, samplePercent);
+ return creator.addMetadata(this, g);
+ }
+
+ @Override
+ public Iterator<Prel> iterator() {
+ return PrelUtil.iter(getInput());
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value)
+ throws E {
+ return logicalVisitor.visitPrel(this, value);
+ }
+
+ @Override
+ public SelectionVectorMode[] getSupportedEncodings() {
+ return SelectionVectorMode.ALL;
+ }
+
+ @Override
+ public SelectionVectorMode getEncoding() {
+ return SelectionVectorMode.NONE;
+ }
+
+ @Override
+ public boolean needsFinalColumnReordering() {
+ return true;
+ }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
index 4fc7aaec0..3615d1d1e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -35,14 +35,14 @@ public class UnpivotMapsPrel extends SingleRel implements Prel, DrillRelNode {
private List<String> mapFieldsNames;
- public UnpivotMapsPrel(RelNode child, RelOptCluster cluster, List<String> mapFieldsNames) {
- super(cluster, child.getTraitSet(), child);
+ public UnpivotMapsPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<String> mapFieldsNames) {
+ super(cluster, traits, child);
this.mapFieldsNames = mapFieldsNames;
}
@Override
public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
- return new UnpivotMapsPrel(sole(inputs), getCluster(), mapFieldsNames);
+ return new UnpivotMapsPrel(getCluster(), traitSet, sole(inputs), mapFieldsNames);
}
@Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
index 818ec3532..b260080d8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.planner.physical.visitor;
+import org.apache.drill.exec.planner.physical.DirectScanPrel;
import org.apache.drill.exec.planner.physical.ExchangePrel;
import org.apache.drill.exec.planner.physical.JoinPrel;
import org.apache.drill.exec.planner.physical.Prel;
@@ -36,6 +37,11 @@ public class BasePrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> implements
}
@Override
+ public RETURN visitScan(DirectScanPrel prel, EXTRA value) throws EXCEP {
+ return visitPrel(prel, value);
+ }
+
+ @Override
public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP {
return visitPrel(prel, value);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
index 5b3168331..c138c832b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.planner.physical.visitor;
+import org.apache.drill.exec.planner.physical.DirectScanPrel;
import org.apache.drill.exec.planner.physical.ExchangePrel;
import org.apache.drill.exec.planner.physical.JoinPrel;
import org.apache.drill.exec.planner.physical.Prel;
@@ -35,6 +36,7 @@ public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
RETURN visitScreen(ScreenPrel prel, EXTRA value) throws EXCEP;
RETURN visitWriter(WriterPrel prel, EXTRA value) throws EXCEP;
RETURN visitScan(ScanPrel prel, EXTRA value) throws EXCEP;
+ RETURN visitScan(DirectScanPrel prel, EXTRA value) throws EXCEP;
RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
RETURN visitProject(ProjectPrel prel, EXTRA value) throws EXCEP;
RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
index 0bef3a772..f02ad6f04 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisualizerVisitor.java
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.planner.physical.visitor;
+import org.apache.drill.exec.planner.physical.DirectScanPrel;
import org.apache.drill.exec.planner.physical.ExchangePrel;
import org.apache.drill.exec.planner.physical.JoinPrel;
import org.apache.drill.exec.planner.physical.Prel;
@@ -205,6 +206,13 @@ public class PrelVisualizerVisitor
}
@Override
+ public Void visitScan(DirectScanPrel prel, VisualizationState value) throws Exception {
+ visitBasePrel(prel, value);
+ endNode(prel, value);
+ return null;
+ }
+
+ @Override
public Void visitJoin(JoinPrel prel, VisualizationState value)
throws Exception {
visitBasePrel(prel, value);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 3dfab6728..7b9050fd1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -27,16 +27,20 @@ import org.apache.calcite.tools.ValidationException;
import org.apache.drill.common.exceptions.UserException;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.AnalyzeTableHandler;
import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
import org.apache.drill.exec.planner.sql.handlers.DescribeSchemaHandler;
import org.apache.drill.exec.planner.sql.handlers.DescribeTableHandler;
import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
+import org.apache.drill.exec.planner.sql.handlers.RefreshMetadataHandler;
import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
+import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
import org.apache.drill.exec.testing.ControlsInjector;
import org.apache.drill.exec.testing.ControlsInjectorFactory;
import org.apache.drill.exec.util.Pointer;
@@ -142,18 +146,22 @@ public class DrillSqlWorker {
switch(sqlNode.getKind()) {
case EXPLAIN:
handler = new ExplainHandler(config, textPlan);
+ context.setSQLStatementType(SqlStatementType.EXPLAIN);
break;
case SET_OPTION:
handler = new SetOptionHandler(context);
+ context.setSQLStatementType(SqlStatementType.SETOPTION);
break;
case DESCRIBE_TABLE:
if (sqlNode instanceof DrillSqlDescribeTable) {
handler = new DescribeTableHandler(config);
+ context.setSQLStatementType(SqlStatementType.DESCRIBE_TABLE);
break;
}
case DESCRIBE_SCHEMA:
if (sqlNode instanceof SqlDescribeSchema) {
handler = new DescribeSchemaHandler(config);
+ context.setSQLStatementType(SqlStatementType.DESCRIBE_SCHEMA);
break;
}
case CREATE_TABLE:
@@ -164,13 +172,25 @@ public class DrillSqlWorker {
case DROP_VIEW:
case OTHER_DDL:
case OTHER:
+ if(sqlNode instanceof SqlCreateTable) {
+ handler = ((DrillSqlCall)sqlNode).getSqlHandler(config, textPlan);
+ context.setSQLStatementType(SqlStatementType.CTAS);
+ break;
+ }
+
if (sqlNode instanceof DrillSqlCall) {
handler = ((DrillSqlCall) sqlNode).getSqlHandler(config);
+ if (handler instanceof AnalyzeTableHandler) {
+ context.setSQLStatementType(SqlStatementType.ANALYZE);
+ } else if (handler instanceof RefreshMetadataHandler) {
+ context.setSQLStatementType(SqlStatementType.REFRESH);
+ }
break;
}
// fallthrough
default:
handler = new DefaultSqlHandler(config, textPlan);
+ context.setSQLStatementType(SqlStatementType.OTHER);
}
// Determines whether result set should be returned for the query based on return result set option and sql node kind.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
index e07fb1589..fb7a68341 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
@@ -201,6 +201,25 @@ public class SchemaUtilites {
* @return mutable schema, exception otherwise
*/
public static AbstractSchema resolveToMutableDrillSchema(final SchemaPlus defaultSchema, List<String> schemaPath) {
+ return resolveToDrillSchemaInternal(defaultSchema, schemaPath, true);
+ }
+
+ /**
+ * Given reference to default schema in schema tree, search for schema with given <i>schemaPath</i>. Once a schema is
+ * found resolve it into a mutable <i>AbstractDrillSchema</i> instance. A {@link UserException} is throws when:
+ * <li>No schema for given <i>schemaPath</i> is found.</li>
+ * <li>Schema found for given <i>schemaPath</i> is a root schema.</li>
+ *
+ * @param defaultSchema
+ * @param schemaPath
+ * @return schema, if found. Otherwise, throws an {@link UserException}
+ */
+ public static AbstractSchema resolveToDrillSchema(final SchemaPlus defaultSchema, List<String> schemaPath) {
+ return resolveToDrillSchemaInternal(defaultSchema, schemaPath, false);
+ }
+
+ private static AbstractSchema resolveToDrillSchemaInternal (SchemaPlus defaultSchema, List<String> schemaPath,
+ boolean checkMutable) {
final SchemaPlus schema = findSchema(defaultSchema, schemaPath);
if (schema == null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
index cdfe31bbc..f1acc71ba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,8 +17,16 @@
*/
package org.apache.drill.exec.planner.sql.handlers;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.schema.Table;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlNodeList;
@@ -27,23 +35,35 @@ import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.tools.RelConversionException;
import org.apache.calcite.tools.ValidationException;
import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.exec.dotdrill.DotDrillType;
import org.apache.drill.exec.physical.PhysicalPlan;
import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
import org.apache.drill.exec.planner.logical.DrillScreenRel;
import org.apache.drill.exec.planner.logical.DrillStoreRel;
+import org.apache.drill.exec.planner.logical.DrillTable;
import org.apache.drill.exec.planner.logical.DrillWriterRel;
import org.apache.drill.exec.planner.physical.Prel;
import org.apache.drill.exec.planner.sql.SchemaUtilites;
import org.apache.drill.exec.planner.sql.parser.SqlAnalyzeTable;
import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
+import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
import org.apache.drill.exec.util.Pointer;
import org.apache.drill.exec.work.foreman.ForemanSetupException;
import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
-
-import java.io.IOException;
-import java.util.List;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
public class AnalyzeTableHandler extends DefaultSqlHandler {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnalyzeTableHandler.class);
@@ -61,36 +81,72 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
SqlIdentifier tableIdentifier = sqlAnalyzeTable.getTableIdentifier();
SqlSelect scanSql = new SqlSelect(
- SqlParserPos.ZERO, /* position */
- SqlNodeList.EMPTY, /* keyword list */
- getColumnList(sqlAnalyzeTable), /*select list */
- tableIdentifier, /* from */
- null, /* where */
- null, /* group by */
- null, /* having */
- null, /* windowDecls */
- null, /* orderBy */
- null, /* offset */
- null /* fetch */
+ SqlParserPos.ZERO, /* position */
+ SqlNodeList.EMPTY, /* keyword list */
+ getColumnList(sqlAnalyzeTable), /* select list */
+ tableIdentifier, /* from */
+ null, /* where */
+ null, /* group by */
+ null, /* having */
+ null, /* windowDecls */
+ null, /* orderBy */
+ null, /* offset */
+ null /* fetch */
);
final ConvertedRelNode convertedRelNode = validateAndConvert(rewrite(scanSql));
final RelDataType validatedRowType = convertedRelNode.getValidatedRowType();
final RelNode relScan = convertedRelNode.getConvertedNode();
-
final String tableName = sqlAnalyzeTable.getName();
- final AbstractSchema drillSchema = SchemaUtilites.resolveToMutableDrillSchema(
+ final AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
config.getConverter().getDefaultSchema(), sqlAnalyzeTable.getSchemaPath());
+ Table table = SqlHandlerUtil.getTableFromSchema(drillSchema, tableName);
- if (SqlHandlerUtil.getTableFromSchema(drillSchema, tableName) == null) {
+ if (table == null) {
throw UserException.validationError()
- .message("No table with given name [%s] exists in schema [%s]", tableName, drillSchema.getFullSchemaName())
+ .message("No table with given name [%s] exists in schema [%s]", tableName,
+ drillSchema.getFullSchemaName())
.build(logger);
}
+ if(! (table instanceof DrillTable)) {
+ return DrillStatsTable.notSupported(context, tableName);
+ }
+
+ if (table instanceof DrillTable) {
+ DrillTable drillTable = (DrillTable) table;
+ final Object selection = drillTable.getSelection();
+ if (!(selection instanceof FormatSelection)) {
+ return DrillStatsTable.notSupported(context, tableName);
+ }
+ // Do not support non-parquet tables
+ FormatSelection formatSelection = (FormatSelection) selection;
+ FormatPluginConfig formatConfig = formatSelection.getFormat();
+ if (!((formatConfig instanceof ParquetFormatConfig)
+ || ((formatConfig instanceof NamedFormatPluginConfig)
+ && ((NamedFormatPluginConfig) formatConfig).name.equals("parquet")))) {
+ return DrillStatsTable.notSupported(context, tableName);
+ }
+
+ FileSystemPlugin plugin = (FileSystemPlugin) drillTable.getPlugin();
+ DrillFileSystem fs = new DrillFileSystem(plugin.getFormatPlugin(
+ formatSelection.getFormat()).getFsConf());
+
+ String selectionRoot = formatSelection.getSelection().getSelectionRoot();
+ if (!selectionRoot.contains(tableName)
+ || !fs.getFileStatus(new Path(selectionRoot)).isDirectory()) {
+ return DrillStatsTable.notSupported(context, tableName);
+ }
+ // Do not recompute statistics, if stale
+ Path statsFilePath = new Path(new Path(selectionRoot), DotDrillType.STATS.getEnding());
+ if (fs.exists(statsFilePath)
+ && !isStatsStale(fs, statsFilePath)) {
+ return DrillStatsTable.notRequired(context, tableName);
+ }
+ }
// Convert the query to Drill Logical plan and insert a writer operator on top.
- DrillRel drel = convertToDrel(relScan, drillSchema, tableName);
+ DrillRel drel = convertToDrel(relScan, drillSchema, tableName, sqlAnalyzeTable.getSamplePercent());
Prel prel = convertToPrel(drel, validatedRowType);
logAndSetTextPlan("Drill Physical", prel, logger);
PhysicalOperator pop = convertToPop(prel);
@@ -100,34 +156,98 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
return plan;
}
- private SqlNodeList getColumnList(final SqlAnalyzeTable sqlAnalyzeTable) {
- final SqlNodeList columnList = new SqlNodeList(SqlParserPos.ZERO);
+ /* Determines if the table was modified after computing statistics based on
+ * directory/file modification timestamps
+ */
+ private boolean isStatsStale(DrillFileSystem fs, Path statsFilePath)
+ throws IOException {
+ long statsFileModifyTime = fs.getFileStatus(statsFilePath).getModificationTime();
+ Path parentPath = statsFilePath.getParent();
+ FileStatus directoryStatus = fs.getFileStatus(parentPath);
+ // Parent directory modified after stats collection?
+ if (directoryStatus.getModificationTime() > statsFileModifyTime) {
+ return true;
+ }
+ if (tableModified(fs, parentPath, statsFileModifyTime)) {
+ return true;
+ }
+ return false;
+ }
+ /* Determines if the table was modified after computing statistics based on
+ * directory/file modification timestamps. Recursively checks sub-directories.
+ */
+ private boolean tableModified(DrillFileSystem fs, Path parentPath,
+ long statsModificationTime) throws IOException {
+ for (final FileStatus file : fs.listStatus(parentPath)) {
+ // If directory or files within it are modified
+ if (file.getModificationTime() > statsModificationTime) {
+ return true;
+ }
+ // For a directory, we should recursively check sub-directories
+ if (file.isDirectory()) {
+ if (tableModified(fs, file.getPath(), statsModificationTime)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /* Generates the column list specified in the ANALYZE statement */
+ private SqlNodeList getColumnList(final SqlAnalyzeTable sqlAnalyzeTable) {
+ SqlNodeList columnList = sqlAnalyzeTable.getFieldList();
+ if (columnList == null || columnList.size() <= 0) {
+ columnList = new SqlNodeList(SqlParserPos.ZERO);
+ columnList.add(new SqlIdentifier(SchemaPath.STAR_COLUMN.rootName(), SqlParserPos.ZERO));
+ }
+ /*final SqlNodeList columnList = new SqlNodeList(SqlParserPos.ZERO);
final List<String> fields = sqlAnalyzeTable.getFieldNames();
if (fields == null || fields.size() <= 0) {
- columnList.add(new SqlIdentifier("*", SqlParserPos.ZERO));
+ columnList.add(new SqlIdentifier(SchemaPath.STAR_COLUMN.rootName(), SqlParserPos.ZERO));
} else {
for(String field : fields) {
columnList.add(new SqlIdentifier(field, SqlParserPos.ZERO));
}
- }
-
+ }*/
return columnList;
}
- protected DrillRel convertToDrel(RelNode relNode, AbstractSchema schema, String analyzeTableName)
+ /* Converts to Drill logical plan */
+ protected DrillRel convertToDrel(RelNode relNode, AbstractSchema schema, String analyzeTableName,
+ double samplePercent)
throws RelConversionException, SqlUnsupportedException {
- final DrillRel convertedRelNode = convertToDrel(relNode);
+ DrillRel convertedRelNode = convertToRawDrel(relNode);
if (convertedRelNode instanceof DrillStoreRel) {
throw new UnsupportedOperationException();
}
+ if (convertedRelNode instanceof DrillProjectRel) {
+ DrillProjectRel projectRel = (DrillProjectRel) convertedRelNode;
+ DrillScanRel scanRel = findScan(projectRel);
+ List<RelDataTypeField> fields = Lists.newArrayList();
+ RexBuilder b = projectRel.getCluster().getRexBuilder();
+ List<RexNode> projections = Lists.newArrayList();
+ // Get the original scan column names - after projection pushdown they should refer to the full col names
+ List<String> fieldNames = new ArrayList<>();
+ List<RelDataTypeField> fieldTypes = projectRel.getRowType().getFieldList();
+ for (SchemaPath colPath : scanRel.getGroupScan().getColumns()) {
+ fieldNames.add(colPath.toString());
+ }
+ for (int i =0; i < fieldTypes.size(); i++) {
+ projections.add(b.makeInputRef(projectRel, i));
+ }
+ // Get the projection row-types
+ RelDataType newRowType = RexUtil.createStructType(projectRel.getCluster().getTypeFactory(),
+ projections, fieldNames, null);
+ DrillProjectRel renamedProject = DrillProjectRel.create(convertedRelNode.getCluster(),
+ convertedRelNode.getTraitSet(), convertedRelNode, projections, newRowType);
+ convertedRelNode = renamedProject;
+ }
+
final RelNode analyzeRel = new DrillAnalyzeRel(
- convertedRelNode.getCluster(),
- convertedRelNode.getTraitSet(),
- convertedRelNode
- );
+ convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode, samplePercent);
final RelNode writerRel = new DrillWriterRel(
analyzeRel.getCluster(),
@@ -139,7 +259,14 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
return new DrillScreenRel(writerRel.getCluster(), writerRel.getTraitSet(), writerRel);
}
- // make sure no unsupported features in ANALYZE statement are used
+ private DrillScanRel findScan(RelNode rel) {
+ if (rel instanceof DrillScanRel) {
+ return (DrillScanRel) rel;
+ } else {
+ return findScan(rel.getInput(0));
+ }
+ }
+ // Make sure no unsupported features in ANALYZE statement are used
private static void verifyNoUnsupportedFunctions(final SqlAnalyzeTable analyzeTable) {
// throw unsupported error for functions that are not yet implemented
if (analyzeTable.getEstimate()) {
@@ -148,9 +275,9 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
.build(logger);
}
- if (analyzeTable.getPercent() != 100) {
+ if (analyzeTable.getSamplePercent() <= 0 && analyzeTable.getSamplePercent() > 100.0) {
throw UserException.unsupportedError()
- .message("Statistics from sampling is not yet supported.")
+ .message("Valid sampling percent between 0-100 is not specified.")
.build(logger);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 52ae7b1a3..0a1b99564 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -231,6 +231,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
}
try {
+ // Materialize the statistics, if available.
StatsMaterializationVisitor.materialize(relNode, context);
// HEP for rules, which are failed at the LOGICAL_PLANNING stage for Volcano planner
@@ -571,7 +572,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
/*
* 4.)
* If two fragments are both estimated to be parallelization one, remove the exchange
- * separating them
+ * separating them.
*/
phyRelNode = ExcessiveExchangeIdentifier.removeExcessiveEchanges(phyRelNode, targetSliceSize);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
index e92637f08..3671bc4fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
@@ -19,6 +19,9 @@ package org.apache.drill.exec.planner.sql.handlers;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelShuttle;
@@ -299,12 +302,15 @@ public class FindLimit0Visitor extends RelShuttleImpl {
/**
* Reader for column names and types.
*/
+ @JsonTypeName("RelDataTypeRecordReader")
public static class RelDataTypeReader extends AbstractRecordReader {
public final List<String> columnNames;
public final List<TypeProtos.MajorType> columnTypes;
- public RelDataTypeReader(List<String> columnNames, List<TypeProtos.MajorType> columnTypes) {
+ @JsonCreator
+ public RelDataTypeReader(@JsonProperty("columnNames") List<String> columnNames,
+ @JsonProperty("columnTypes") List<TypeProtos.MajorType> columnTypes) {
Preconditions.checkArgument(columnNames.size() == columnTypes.size(), "Number of columns and their types should match");
this.columnNames = columnNames;
this.columnTypes = columnTypes;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index 210f43b89..56985e2fe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -86,13 +86,18 @@ public class CompoundIdentifierConverter extends SqlShuttle {
}
private boolean enableComplex = true;
+ private boolean allowNoTableRefCompoundIdentifier = false;
+
+ public CompoundIdentifierConverter(boolean allowNoTableRefCompoundIdentifier) {
+ this.allowNoTableRefCompoundIdentifier = allowNoTableRefCompoundIdentifier;
+ }
@Override
public SqlNode visit(SqlIdentifier id) {
if (id instanceof DrillCompoundIdentifier) {
DrillCompoundIdentifier compoundIdentifier = (DrillCompoundIdentifier) id;
if (enableComplex) {
- return compoundIdentifier.getAsSqlNode();
+ return compoundIdentifier.getAsSqlNode(allowNoTableRefCompoundIdentifier);
} else {
return compoundIdentifier.getAsCompoundIdentifier();
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
index ba6413b59..7da5d7297 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
@@ -74,7 +74,7 @@ public class DrillCompoundIdentifier extends SqlIdentifier {
}
}
- public SqlNode getAsSqlNode() {
+ public SqlNode getAsSqlNode(boolean allowNoTableRefCompoundIdentifier) {
if (ids.size() == 1) {
return new SqlIdentifier(Collections.singletonList(ids.get(0).value), ids.get(0).parserPos);
}
@@ -91,13 +91,24 @@ public class DrillCompoundIdentifier extends SqlIdentifier {
ids.get(0).parserPos,
ImmutableList.of(ids.get(0).parserPos));
} else {
- // handle everything post two index as item operator.
- startIndex = 2;
- node = new SqlIdentifier(
- // Replaces star by empty string. See SqlIdentifier#isStar()
- ImmutableList.of(ids.get(0).value, STAR_TO_EMPTY.apply(ids.get(1).value)), null,
- ids.get(0).parserPos,
- ImmutableList.of(ids.get(0).parserPos, ids.get(1).parserPos));
+ if (allowNoTableRefCompoundIdentifier) {
+ // For certain statements e.g. ANALYZE which only reference one table, compound column names may be referenced
+ // without the table reference. For such cases, handle everything post one index as item operator.
+ startIndex = 1;
+ node = new SqlIdentifier(
+ // Replaces star by empty string. See SqlIdentifier#isStar()
+ ImmutableList.of(STAR_TO_EMPTY.apply(ids.get(0).value)), null,
+ ids.get(0).parserPos,
+ ImmutableList.of(ids.get(0).parserPos));
+ } else {
+ // handle everything post two index as item operator.
+ startIndex = 2;
+ node = new SqlIdentifier(
+ // Replaces star by empty string. See SqlIdentifier#isStar()
+ ImmutableList.of(ids.get(0).value, STAR_TO_EMPTY.apply(ids.get(1).value)), null,
+ ids.get(0).parserPos,
+ ImmutableList.of(ids.get(0).parserPos, ids.get(1).parserPos));
+ }
}
for (int i = startIndex; i < ids.size(); i++) {
node = ids.get(i).getNode(node);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
index 91f83bf72..c5bbc5e9b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -56,15 +56,15 @@ public class SqlAnalyzeTable extends DrillSqlCall {
private final SqlIdentifier tblName;
private final SqlLiteral estimate;
private final SqlNodeList fieldList;
- private final SqlNumericLiteral percent;
+ private final SqlNumericLiteral samplePercent;
public SqlAnalyzeTable(SqlParserPos pos, SqlIdentifier tblName, SqlLiteral estimate,
- SqlNodeList fieldList, SqlNumericLiteral percent) {
+ SqlNodeList fieldList, SqlNumericLiteral samplePercent) {
super(pos);
this.tblName = tblName;
this.estimate = estimate;
this.fieldList = fieldList;
- this.percent = percent;
+ this.samplePercent = samplePercent;
}
@Override
@@ -78,7 +78,7 @@ public class SqlAnalyzeTable extends DrillSqlCall {
operands.add(tblName);
operands.add(estimate);
operands.add(fieldList);
- operands.add(percent);
+ operands.add(samplePercent);
return operands;
}
@@ -89,10 +89,8 @@ public class SqlAnalyzeTable extends DrillSqlCall {
tblName.unparse(writer, leftPrec, rightPrec);
writer.keyword(estimate.booleanValue() ? "ESTIMATE" : "COMPUTE");
writer.keyword("STATISTICS");
- writer.keyword("FOR");
if (fieldList != null && fieldList.size() > 0) {
- writer.keyword("COLUMNS");
writer.keyword("(");
fieldList.get(0).unparse(writer, leftPrec, rightPrec);
for (int i = 1; i < fieldList.size(); i++) {
@@ -100,12 +98,9 @@ public class SqlAnalyzeTable extends DrillSqlCall {
fieldList.get(i).unparse(writer, leftPrec, rightPrec);
}
writer.keyword(")");
- } else {
- writer.keyword("ALL");
- writer.keyword("COLUMNS");
}
writer.keyword("SAMPLE");
- percent.unparse(writer, leftPrec, rightPrec);
+ samplePercent.unparse(writer, leftPrec, rightPrec);
writer.keyword("PERCENT");
}
@@ -147,11 +142,15 @@ public class SqlAnalyzeTable extends DrillSqlCall {
return columnNames;
}
+ public SqlNodeList getFieldList() {
+ return fieldList;
+ }
+
public boolean getEstimate() {
return estimate.booleanValue();
}
- public int getPercent() {
- return percent.intValue(true);
+ public int getSamplePercent() {
+ return samplePercent.intValue(true);
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java
index 9e7d85195..a80710742 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
import org.apache.calcite.sql.parser.SqlParserImplFactory;
import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.drill.exec.planner.sql.parser.SqlAnalyzeTable;
public class DrillParserWithCompoundIdConverter extends DrillParserImpl {
@@ -45,7 +46,11 @@ public class DrillParserWithCompoundIdConverter extends DrillParserImpl {
}
protected SqlVisitor<SqlNode> createConverter() {
- return new CompoundIdentifierConverter();
+ return createConverter(false);
+ }
+
+ private SqlVisitor<SqlNode> createConverter(boolean allowNoTableRefCompoundIdentifier) {
+ return new CompoundIdentifierConverter(allowNoTableRefCompoundIdentifier);
}
@Override
@@ -57,6 +62,6 @@ public class DrillParserWithCompoundIdConverter extends DrillParserImpl {
@Override
public SqlNode parseSqlStmtEof() throws Exception {
SqlNode originalSqlNode = super.parseSqlStmtEof();
- return originalSqlNode.accept(createConverter());
+ return originalSqlNode.accept(createConverter(originalSqlNode instanceof SqlAnalyzeTable));
}
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index d5463efea..1a9eff3f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -121,6 +121,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
new OptionDefinition(PlannerSettings.JOIN_OPTIMIZATION),
new OptionDefinition(PlannerSettings.ENABLE_UNNEST_LATERAL),
new OptionDefinition(PlannerSettings.FORCE_2PHASE_AGGR), // for testing
+ new OptionDefinition(PlannerSettings.STATISTICS_USE),
new OptionDefinition(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR),
new OptionDefinition(ExecConstants.HASHJOIN_MAX_MEMORY_VALIDATOR, new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, true)),
new OptionDefinition(ExecConstants.HASHJOIN_NUM_ROWS_IN_BATCH_VALIDATOR),
@@ -271,7 +272,11 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
new OptionDefinition(ExecConstants.FRAG_RUNNER_RPC_TIMEOUT_VALIDATOR, new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, true)),
new OptionDefinition(ExecConstants.LIST_FILES_RECURSIVELY_VALIDATOR),
new OptionDefinition(ExecConstants.QUERY_ROWKEYJOIN_BATCHSIZE),
- new OptionDefinition(ExecConstants.RETURN_RESULT_SET_FOR_DDL_VALIDATOR)
+ new OptionDefinition(ExecConstants.RETURN_RESULT_SET_FOR_DDL_VALIDATOR),
+ new OptionDefinition(ExecConstants.HLL_ACCURACY_VALIDATOR),
+ new OptionDefinition(ExecConstants.DETERMINISTIC_SAMPLING_VALIDATOR),
+ new OptionDefinition(ExecConstants.NDV_BLOOM_FILTER_ELEMENTS_VALIDATOR),
+ new OptionDefinition(ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB_VALIDATOR)
};
CaseInsensitiveMap<OptionDefinition> map = Arrays.stream(definitions)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
index 3f8f8a4b4..4992ac3f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
@@ -30,6 +30,25 @@ import org.apache.drill.exec.util.ImpersonationUtil;
public class TypeValidators {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeValidators.class);
+ public static class NonNegativeLongValidator extends LongValidator {
+ private final long max;
+
+ public NonNegativeLongValidator(String name, long max, OptionDescription description) {
+ super(name, description);
+ this.max = max;
+ }
+
+ @Override
+ public void validate(final OptionValue v, final OptionMetaData metaData, final OptionSet manager) {
+ super.validate(v, metaData, manager);
+ if (v.num_val > max || v.num_val < 0) {
+ throw UserException.validationError()
+ .message(String.format("Option %s must be between %d and %d.", getOptionName(), 0, max))
+ .build(logger);
+ }
+ }
+ }
+
public static class PositiveLongValidator extends LongValidator {
protected final long max;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
index 33b361c7b..210b0a0e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
@@ -19,12 +19,23 @@ package org.apache.drill.exec.store;
import java.util.Map;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.exception.OutOfMemoryException;
import org.apache.drill.exec.ops.OperatorContext;
import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.planner.sql.handlers.FindLimit0Visitor;
+import org.apache.drill.exec.store.pojo.PojoRecordReader;
import org.apache.drill.exec.vector.ValueVector;
+@JsonTypeInfo(
+ use = JsonTypeInfo.Id.NAME,
+ include = JsonTypeInfo.As.PROPERTY,
+ property = "type")
+@JsonSubTypes({
+ @JsonSubTypes.Type(value = PojoRecordReader.class, name = "PojoRecordReader"),
+ @JsonSubTypes.Type(value = FindLimit0Visitor.RelDataTypeReader.class, name = "RelDataTypeRecordReader") })
public interface RecordReader extends AutoCloseable {
long ALLOCATOR_INITIAL_RESERVATION = 1 * 1024 * 1024;
long ALLOCATOR_MAX_RESERVATION = 20L * 1000 * 1000 * 1000;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
index 6f25eb35b..dc2046a0d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
@@ -25,6 +25,7 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.planner.logical.DrillTable;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
import org.apache.drill.exec.server.DrillbitContext;
@@ -45,6 +46,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
/**
* Format plugin for Avro data files.
@@ -94,6 +97,21 @@ public class AvroFormatPlugin extends EasyFormatPlugin<AvroFormatConfig> {
return this.matcher;
}
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
private static class AvroFormatMatcher extends BasicFormatMatcher {
public AvroFormatMatcher(AvroFormatPlugin plugin) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
index bf258c2d9..13017fadc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
@@ -27,10 +27,13 @@ import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
import org.apache.drill.exec.physical.base.AbstractWriter;
import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.server.options.OptionManager;
import org.apache.drill.exec.store.StoragePluginOptimizerRule;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
/**
* Similar to a storage engine but built specifically to work within a FileSystem context.
@@ -49,7 +52,7 @@ public interface FormatPlugin {
FormatMatcher getMatcher();
- public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException;
+ public AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) throws IOException;
Set<StoragePluginOptimizerRule> getOptimizerRules();
@@ -59,6 +62,12 @@ public interface FormatPlugin {
return getGroupScan(userName, selection, columns);
}
+ public boolean supportsStatistics();
+
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException;
+
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException;
+
FormatPluginConfig getConfig();
StoragePluginConfig getStorageConfig();
Configuration getFsConf();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index fc512212f..2c629ff94 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -423,11 +423,15 @@ public class WorkspaceSchemaFactory {
private final ExpandingConcurrentMap<TableInstance, DrillTable> tables = new ExpandingConcurrentMap<>(this);
private final SchemaConfig schemaConfig;
private DrillFileSystem fs;
+ // Drill Process User file-system
+ private DrillFileSystem dpsFs;
public WorkspaceSchema(List<String> parentSchemaPath, String wsName, SchemaConfig schemaConfig, DrillFileSystem fs) throws IOException {
super(parentSchemaPath, wsName);
this.schemaConfig = schemaConfig;
this.fs = fs;
+ //this.fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), fsConf);
+ this.dpsFs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fsConf);
}
DrillTable getDrillTable(TableInstance key) {
@@ -583,28 +587,59 @@ public class WorkspaceSchemaFactory {
try {
if (table.getStatsTable() == null) {
- Table statsTable = getStatsTable(tableName);
- if (statsTable != null) {
- table.setStatsTable(new DrillStatsTable(getFullSchemaName(), getStatsTableName(tableName)));
- }
+ String statsTableName = getStatsTableName(tableName);
+ Path statsTableFilePath = getStatsTableFilePath(tableName);
+ table.setStatsTable(new DrillStatsTable(getFullSchemaName(), statsTableName,
+ statsTableFilePath, fs));
}
} catch (final Exception e) {
- logger.warn("Failed to find the stats table for table [{}] in schema [{}]", tableName, getFullSchemaName());
+ logger.warn("Failed to find the stats table for table [{}] in schema [{}]",
+ tableName, getFullSchemaName());
}
}
// Get stats table name for a given table name.
private String getStatsTableName(final String tableName) {
+ // Access stats file as DRILL process user (not impersonated user)
final Path tablePath = new Path(config.getLocation(), tableName);
try {
- if (fs.isDirectory(tablePath)) {
- return tableName + Path.SEPARATOR + STATS.getEnding();
+ String name;
+ if (dpsFs.isDirectory(tablePath)) {
+ name = tableName + Path.SEPARATOR + STATS.getEnding();
+ if (dpsFs.isDirectory(new Path(name))) {
+ return name;
+ }
} else {
- return tableName + STATS.getEnding();
+ //TODO: Not really useful. Remove?
+ name = tableName + STATS.getEnding();
+ if (dpsFs.isFile(new Path(name))) {
+ return name;
+ }
+ }
+ return name;
+ } catch (final Exception e) {
+ throw new DrillRuntimeException(
+ String.format("Failed to find the stats for table [%s] in schema [%s]",
+ tableName, getFullSchemaName()));
+ }
+ }
+
+ // Get stats table file (JSON) path for the given table name.
+ private Path getStatsTableFilePath(final String tableName) {
+ // Access stats file as DRILL process user (not impersonated user)
+ final Path tablePath = new Path(config.getLocation(), tableName);
+ try {
+ Path stFPath = null;
+ if (dpsFs.isDirectory(tablePath)) {
+ stFPath = new Path(tablePath, STATS.getEnding()+ Path.SEPARATOR + "0_0.json");
+ if (dpsFs.isFile(stFPath)) {
+ return stFPath;
+ }
}
+ return stFPath;
} catch (final Exception e) {
throw new DrillRuntimeException(
- String.format("Failed to find the location of the stats for table [%s] in schema [%s]",
+ String.format("Failed to find the the stats for table [%s] in schema [%s]",
tableName, getFullSchemaName()));
}
}
@@ -626,7 +661,8 @@ public class WorkspaceSchemaFactory {
public CreateTableEntry createNewTable(String tableName, List<String> partitionColumns, StorageStrategy storageStrategy) {
String storage = schemaConfig.getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
- return createOrAppendToTable(tableName, false, formatPlugin, partitionColumns, storageStrategy);
+
+ return createOrAppendToTable(tableName, formatPlugin, partitionColumns, storageStrategy);
}
@Override
@@ -634,7 +670,7 @@ public class WorkspaceSchemaFactory {
ensureNotStatsTable(tableName);
final String statsTableName = getStatsTableName(tableName);
FormatPlugin formatPlugin = plugin.getFormatPlugin(JSONFormatPlugin.DEFAULT_NAME);
- return createOrAppendToTable(statsTableName, false, formatPlugin, ImmutableList.<String>of(),
+ return createOrAppendToTable(statsTableName, formatPlugin, ImmutableList.<String>of(),
StorageStrategy.DEFAULT);
}
@@ -643,7 +679,7 @@ public class WorkspaceSchemaFactory {
ensureNotStatsTable(tableName);
final String statsTableName = getStatsTableName(tableName);
FormatPlugin formatPlugin = plugin.getFormatPlugin(JSONFormatPlugin.DEFAULT_NAME);
- return createOrAppendToTable(statsTableName, true, formatPlugin, ImmutableList.<String>of(),
+ return createOrAppendToTable(statsTableName, formatPlugin, ImmutableList.<String>of(),
StorageStrategy.DEFAULT);
}
@@ -652,8 +688,8 @@ public class WorkspaceSchemaFactory {
return getTable(getStatsTableName(tableName));
}
- private CreateTableEntry createOrAppendToTable(String tableName, boolean append, FormatPlugin formatPlugin,
- List<String> partitonColumns, StorageStrategy storageStrategy) {
+ private CreateTableEntry createOrAppendToTable(String tableName, FormatPlugin formatPlugin,
+ List<String> partitionColumns, StorageStrategy storageStrategy) {
if (formatPlugin == null) {
throw new UnsupportedOperationException(
String.format("Unsupported format '%s' in workspace '%s'", config.getDefaultInputFormat(),
@@ -664,8 +700,7 @@ public class WorkspaceSchemaFactory {
(FileSystemConfig) plugin.getConfig(),
formatPlugin,
config.getLocation() + Path.SEPARATOR + tableName,
- append,
- partitonColumns,
+ partitionColumns,
storageStrategy);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index ed1651e0f..d3bed8feb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -38,6 +38,7 @@ import org.apache.drill.exec.physical.base.ScanStats;
import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
import org.apache.drill.exec.physical.impl.ScanBatch;
import org.apache.drill.exec.physical.impl.WriterRecordBatch;
+import org.apache.drill.exec.physical.impl.StatisticsWriterRecordBatch;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.record.CloseableRecordBatch;
import org.apache.drill.exec.record.RecordBatch;
@@ -45,6 +46,7 @@ import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.ColumnExplorer;
import org.apache.drill.exec.store.RecordReader;
import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.StatisticsRecordWriter;
import org.apache.drill.exec.store.StoragePluginOptimizerRule;
import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -171,12 +173,25 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
return new ScanBatch(context, oContext, readers, implicitColumns);
}
+ public boolean isStatisticsRecordWriter(FragmentContext context, EasyWriter writer) {
+ return false;
+ }
+
public abstract RecordWriter getRecordWriter(FragmentContext context, EasyWriter writer) throws IOException;
+ public StatisticsRecordWriter getStatisticsRecordWriter(FragmentContext context, EasyWriter writer) throws IOException
+ {
+ return null;
+ }
+
public CloseableRecordBatch getWriterBatch(FragmentContext context, RecordBatch incoming, EasyWriter writer)
throws ExecutionSetupException {
try {
- return new WriterRecordBatch(writer, incoming, context, getRecordWriter(context, writer));
+ if (isStatisticsRecordWriter(context, writer)) {
+ return new StatisticsWriterRecordBatch(writer, incoming, context, getStatisticsRecordWriter(context, writer));
+ } else {
+ return new WriterRecordBatch(writer, incoming, context, getRecordWriter(context, writer));
+ }
} catch(IOException e) {
throw new ExecutionSetupException(String.format("Failed to create the WriterRecordBatch. %s", e.getMessage()), e);
}
@@ -193,8 +208,8 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
}
@Override
- public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException {
- return new EasyWriter(child, location, append, partitionColumns, this);
+ public AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) throws IOException {
+ return new EasyWriter(child, location, partitionColumns, this);
}
@Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index aae7a4be4..759d07ff9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -162,6 +162,7 @@ public class EasyGroupScan extends AbstractFileGroupScan {
return selection.getFiles();
}
+ @Override
@JsonProperty("columns")
public List<SchemaPath> getColumns() {
return columns;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
index 9f4120689..379e2c93e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
@@ -40,7 +40,6 @@ public class EasyWriter extends AbstractWriter {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyWriter.class);
private final String location;
- private final boolean append;
private final List<String> partitionColumns;
private final EasyFormatPlugin<?> formatPlugin;
@@ -48,7 +47,6 @@ public class EasyWriter extends AbstractWriter {
public EasyWriter(
@JsonProperty("child") PhysicalOperator child,
@JsonProperty("location") String location,
- @JsonProperty("append") boolean append,
@JsonProperty("partitionColumns") List<String> partitionColumns,
@JsonProperty("storageStrategy") StorageStrategy storageStrategy,
@JsonProperty("storage") StoragePluginConfig storageConfig,
@@ -59,21 +57,18 @@ public class EasyWriter extends AbstractWriter {
this.formatPlugin = (EasyFormatPlugin<?>) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
this.location = location;
- this.append = append;
this.partitionColumns = partitionColumns;
setStorageStrategy(storageStrategy);
}
public EasyWriter(PhysicalOperator child,
String location,
- boolean append,
List<String> partitionColumns,
EasyFormatPlugin<?> formatPlugin) {
super(child);
this.formatPlugin = formatPlugin;
this.location = location;
- this.append = append;
this.partitionColumns = partitionColumns;
}
@@ -82,11 +77,6 @@ public class EasyWriter extends AbstractWriter {
return location;
}
- @JsonProperty("append")
- public boolean getAppend() {
- return append;
- }
-
@JsonProperty("storage")
public StoragePluginConfig getStorageConfig(){
return formatPlugin.getStorageConfig();
@@ -104,7 +94,7 @@ public class EasyWriter extends AbstractWriter {
@Override
protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
- EasyWriter writer = new EasyWriter(child, location, append, partitionColumns, formatPlugin);
+ EasyWriter writer = new EasyWriter(child, location, partitionColumns, formatPlugin);
writer.setStorageStrategy(getStorageStrategy());
return writer;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
index af609954e..89b0ef6e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
@@ -17,22 +17,22 @@
*/
package org.apache.drill.exec.store.direct;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.drill.exec.physical.base.AbstractSubScan;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
import org.apache.drill.exec.store.RecordReader;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.annotation.JsonTypeName;
import static com.fasterxml.jackson.annotation.JsonTypeInfo.Id.NAME;
import static com.fasterxml.jackson.annotation.JsonTypeInfo.As.WRAPPER_OBJECT;
@JsonTypeName("direct-sub-scan")
public class DirectSubScan extends AbstractSubScan {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectSubScan.class);
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectSubScan.class);
@JsonTypeInfo(use=NAME, include=WRAPPER_OBJECT)
private final RecordReader reader;
@@ -43,6 +43,7 @@ public class DirectSubScan extends AbstractSubScan {
}
@JsonProperty
+ //@JsonGetter("reader")
public RecordReader getReader() {
return reader;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index ab90cda5c..fea350a7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -17,36 +17,46 @@
*/
package org.apache.drill.exec.store.easy.json;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
-import java.util.HashMap;
+import java.io.OutputStream;
import java.util.List;
import java.util.Map;
-
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.FormatPluginConfig;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.RecordReader;
import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.StatisticsRecordWriter;
import org.apache.drill.exec.store.dfs.DrillFileSystem;
import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
import org.apache.drill.exec.store.dfs.easy.EasyWriter;
import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.drill.exec.store.easy.json.JSONFormatPlugin.JSONFormatConfig;
-import org.apache.hadoop.conf.Configuration;
-
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONFormatPlugin.class);
+ public static final String DEFAULT_NAME = "json";
private static final boolean IS_COMPRESSIBLE = true;
- public static final String DEFAULT_NAME = "json";
public JSONFormatPlugin(String name, DrillbitContext context, Configuration fsConf, StoragePluginConfig storageConfig) {
this(name, context, fsConf, storageConfig, new JSONFormatConfig());
@@ -66,11 +76,40 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
}
@Override
+ public boolean isStatisticsRecordWriter(FragmentContext context, EasyWriter writer) {
+ if (context.getSQLStatementType() == SqlStatementType.ANALYZE) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public StatisticsRecordWriter getStatisticsRecordWriter(FragmentContext context, EasyWriter writer)
+ throws IOException {
+ StatisticsRecordWriter recordWriter;
+ //ANALYZE statement requires the special statistics writer
+ if (!isStatisticsRecordWriter(context, writer)) {
+ return null;
+ }
+ Map<String, String> options = setupOptions(context, writer, true);
+ recordWriter = new JsonStatisticsRecordWriter(getFsConf(), this);
+ recordWriter.init(options);
+ return recordWriter;
+ }
+
+ @Override
public RecordWriter getRecordWriter(FragmentContext context, EasyWriter writer) throws IOException {
- Map<String, String> options = new HashMap<>();
+ RecordWriter recordWriter;
+ Map<String, String> options = setupOptions(context, writer, true);
+ recordWriter = new JsonRecordWriter(writer.getStorageStrategy(), getFsConf());
+ recordWriter.init(options);
+ return recordWriter;
+ }
+ private Map<String, String> setupOptions(FragmentContext context, EasyWriter writer, boolean statsOptions) {
+ Map<String, String> options = Maps.newHashMap();
options.put("location", writer.getLocation());
- options.put("append", Boolean.toString(writer.getAppend()));
FragmentHandle handle = context.getHandle();
String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
@@ -81,11 +120,48 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
options.put("uglify", Boolean.toString(context.getOptions().getOption(ExecConstants.JSON_WRITER_UGLIFY)));
options.put("skipnulls", Boolean.toString(context.getOptions().getOption(ExecConstants.JSON_WRITER_SKIPNULLFIELDS)));
options.put("enableNanInf", Boolean.toString(context.getOptions().getOption(ExecConstants.JSON_WRITER_NAN_INF_NUMBERS_VALIDATOR)));
+ if (statsOptions) {
+ options.put("queryid", context.getQueryIdString());
+ }
+ return options;
+ }
- RecordWriter recordWriter = new JsonRecordWriter(writer.getStorageStrategy(), getFsConf());
- recordWriter.init(options);
+ @Override
+ public boolean supportsStatistics() {
+ return true;
+ }
- return recordWriter;
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ FSDataOutputStream stream = null;
+ JsonGenerator generator = null;
+ try {
+ JsonFactory factory = new JsonFactory();
+ stream = fs.create(statsTablePath);
+ ObjectMapper mapper = DrillStatsTable.getMapper();
+ generator = factory.createGenerator((OutputStream) stream).useDefaultPrettyPrinter().setCodec(mapper);
+ mapper.writeValue(generator, statistics);
+ } catch (com.fasterxml.jackson.core.JsonGenerationException ex) {
+ logger.error("Unable to create file (JSON generation error): " + statsTablePath.getName(), ex);
+ throw ex;
+ } catch (com.fasterxml.jackson.databind.JsonMappingException ex) {
+ logger.error("Unable to create file (JSON mapping error): " + statsTablePath.getName(), ex);
+ throw ex;
+ } catch (IOException ex) {
+ logger.error("Unable to create file " + statsTablePath.getName(), ex);
+ } finally {
+ if (generator != null) {
+ generator.flush();
+ }
+ if (stream != null) {
+ stream.close();
+ }
+ }
}
@JsonTypeName("json")
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
index d533c0a19..4a3d431d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
@@ -75,7 +75,6 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
@Override
public void init(Map<String, String> writerOptions) throws IOException {
this.location = writerOptions.get("location");
- this.append = writerOptions.get("append").equalsIgnoreCase("true") ? true : false;
this.prefix = writerOptions.get("prefix");
this.fieldDelimiter = writerOptions.get("separator");
this.extension = writerOptions.get("extension");
@@ -85,11 +84,7 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
this.fs = FileSystem.get(fsConf);
- Path fileName;
- do {
- fileName = new Path(location, prefix + "_" + (index++) + "." + extension);
- } while (append && fs.exists(fileName));
-
+ Path fileName = new Path(location, prefix + "_" + index + "." + extension);
try {
// json writer does not support partitions, so only one file can be created
// and thus only one location should be deleted in case of abort
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
new file mode 100644
index 000000000..6fb866713
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
@@ -0,0 +1,482 @@
+/*
+ * 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.drill.exec.store.easy.json;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.common.DrillStatsTable.STATS_VERSION;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.store.JSONBaseStatisticsRecordWriter;
+import org.apache.drill.exec.store.dfs.FormatPlugin;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter {
+
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonStatisticsRecordWriter.class);
+ private static final String LINE_FEED = String.format("%n");
+ private String location;
+ private String prefix;
+ private String fieldDelimiter;
+ private String extension;
+ private boolean useExtendedOutput;
+ private FileSystem fs = null;
+ private STATS_VERSION statisticsVersion;
+ private final JsonFactory factory = new JsonFactory();
+ private String lastDirectory = null;
+ private Configuration fsConf = null;
+ private FormatPlugin formatPlugin = null;
+ private String nextField = null;
+ private DrillStatsTable.TableStatistics statistics;
+ private List<DrillStatsTable.ColumnStatistics> columnStatisticsList = new ArrayList<DrillStatsTable.ColumnStatistics>();
+ private DrillStatsTable.ColumnStatistics columnStatistics;
+ private LocalDate dirComputedTime = null;
+ private Path fileName = null;
+ private String queryId = null;
+ private long recordsWritten = -1;
+ private boolean errStatus = false;
+
+ public JsonStatisticsRecordWriter(Configuration fsConf, FormatPlugin formatPlugin){
+ this.fsConf = fsConf;
+ this.formatPlugin = formatPlugin;
+ }
+
+ @Override
+ public void init(Map<String, String> writerOptions) throws IOException {
+ this.location = writerOptions.get("location");
+ this.prefix = writerOptions.get("prefix");
+ this.fieldDelimiter = writerOptions.get("separator");
+ this.extension = writerOptions.get("extension");
+ this.useExtendedOutput = Boolean.parseBoolean(writerOptions.get("extended"));
+ this.skipNullFields = Boolean.parseBoolean(writerOptions.get("skipnulls"));
+ this.statisticsVersion = DrillStatsTable.CURRENT_VERSION;
+ this.queryId = writerOptions.get("queryid");
+ //Write as DRILL process user
+ this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fsConf);
+
+ fileName = new Path(location, prefix + "." + extension + ".tmp." + queryId);
+ // Delete .tmp file if exists. Unexpected error in cleanup during last ANALYZE
+ try {
+ if (fs.exists(fileName)) {
+ fs.delete(fileName, false);
+ }
+ } catch (IOException ex) {
+ logger.error("Unable to delete tmp file (corrupt): " + fileName, ex);
+ throw ex;
+ }
+ try {
+ // Delete the tmp file and .stats.drill on exit. After writing out the permanent file
+ // we cancel the deleteOnExit. This ensures that if prior to writing out the stats
+ // file the process is killed, we perform the cleanup.
+ fs.deleteOnExit(fileName);
+ fs.deleteOnExit(new Path(location));
+ logger.debug("Created file: {}", fileName);
+ } catch (IOException ex) {
+ logger.error("Unable to create file: " + fileName, ex);
+ throw ex;
+ }
+ }
+
+ @Override
+ public void updateSchema(VectorAccessible batch) throws IOException {
+ // no op
+ }
+
+ @Override
+ public boolean isBlockingWriter() {
+ return true;
+ }
+
+ @Override
+ public void checkForNewPartition(int index) {
+ // no op
+ }
+
+ @Override
+ public FieldConverter getNewBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new BigIntJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class BigIntJsonConverter extends FieldConverter {
+
+ public BigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (fieldName.equals(Statistic.SCHEMA)) {
+ nextField = fieldName;
+ } else if (fieldName.equals(Statistic.ROWCOUNT)
+ || fieldName.equals(Statistic.NNROWCOUNT)
+ || fieldName.equals(Statistic.NDV)
+ || fieldName.equals(Statistic.AVG_WIDTH)
+ || fieldName.equals(Statistic.SUM_DUPS)) {
+ nextField = fieldName;
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.SCHEMA)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setSchema(reader.readLong());
+ } else if (nextField.equals(Statistic.ROWCOUNT)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setCount(reader.readLong());
+ } else if (nextField.equals(Statistic.NNROWCOUNT)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNonNullCount(reader.readLong());
+ } else if (nextField.equals(Statistic.NDV)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNdv(reader.readLong());
+ } else if (nextField.equals(Statistic.AVG_WIDTH)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setAvgWidth(reader.readLong());
+ } else if (nextField.equals(Statistic.SUM_DUPS)) {
+ // Ignore Count_Approx_Dups statistic
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewIntConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new IntJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class IntJsonConverter extends FieldConverter {
+
+ public IntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (fieldName.equals(Statistic.COLTYPE)) {
+ nextField = fieldName;
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.COLTYPE)) {
+ // Do not write out the type
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewDateConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new DateJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class DateJsonConverter extends FieldConverter {
+
+ public DateJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (fieldName.equals(Statistic.COMPUTED)) {
+ nextField = fieldName;
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals((Statistic.COMPUTED))) {
+ LocalDate computedTime = reader.readLocalDate();
+ if (dirComputedTime == null
+ || computedTime.compareTo(dirComputedTime) > 0) {
+ dirComputedTime = computedTime;
+ }
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewVarCharConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new VarCharJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class VarCharJsonConverter extends FieldConverter {
+
+ public VarCharJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (fieldName.equals(Statistic.COLNAME)) {
+ nextField = fieldName;
+ } else if (fieldName.equals(Statistic.COLTYPE)) {
+ nextField = fieldName;
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.COLNAME)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setName(reader.readText().toString());
+ } else if (nextField.equals(Statistic.COLTYPE)) {
+ MajorType fieldType = DrillStatsTable.getMapper().readValue(reader.readText().toString(), MajorType.class);
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setType(fieldType);
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewNullableBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new NullableBigIntJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class NullableBigIntJsonConverter extends FieldConverter {
+
+ public NullableBigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (fieldName.equals(Statistic.ROWCOUNT)
+ || fieldName.equals(Statistic.NNROWCOUNT)
+ || fieldName.equals(Statistic.NDV)
+ || fieldName.equals(Statistic.SUM_DUPS)) {
+ nextField = fieldName;
+ }
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.ROWCOUNT)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setCount(reader.readLong());
+ } else if (nextField.equals(Statistic.NNROWCOUNT)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNonNullCount(reader.readLong());
+ } else if (nextField.equals(Statistic.NDV)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNdv(reader.readLong());
+ } else if (nextField.equals(Statistic.SUM_DUPS)) {
+ // Ignore Count_Approx_Dups statistic
+ }
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewNullableVarBinaryConverter(int fieldId, String fieldName, FieldReader reader) {
+ return new NullableVarBinaryJsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class NullableVarBinaryJsonConverter extends FieldConverter {
+
+ public NullableVarBinaryJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (fieldName.equals(Statistic.HLL)
+ || fieldName.equals(Statistic.HLL_MERGE)) {
+ nextField = fieldName;
+ }
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.HLL)
+ || nextField.equals(Statistic.HLL_MERGE)) {
+ // Do NOT write out the HLL output, since it is not used yet for computing statistics for a
+ // subset of partitions in the query OR for computing NDV with incremental statistics.
+ }
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public FieldConverter getNewNullableFloat8Converter(int fieldId, String fieldName, FieldReader reader) {
+ return new NullableFloat8JsonConverter(fieldId, fieldName, reader);
+ }
+
+ public class NullableFloat8JsonConverter extends FieldConverter {
+
+ public NullableFloat8JsonConverter(int fieldId, String fieldName, FieldReader reader) {
+ super(fieldId, fieldName, reader);
+ }
+
+ @Override
+ public void startField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (fieldName.equals(Statistic.AVG_WIDTH)) {
+ nextField = fieldName;
+ }
+ }
+ }
+
+ @Override
+ public void writeField() throws IOException {
+ if (!skipNullFields || this.reader.isSet()) {
+ if (nextField == null) {
+ errStatus = true;
+ throw new IOException("Statistics writer encountered unexpected field");
+ }
+ if (nextField.equals(Statistic.AVG_WIDTH)) {
+ ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setAvgWidth(reader.readDouble());
+ }
+ }
+ }
+
+ @Override
+ public void endField() throws IOException {
+ nextField = null;
+ }
+ }
+
+ @Override
+ public void startStatisticsRecord() throws IOException {
+ columnStatistics = new DrillStatsTable.ColumnStatistics_v1();
+ }
+
+ @Override
+ public void endStatisticsRecord() throws IOException {
+ columnStatisticsList.add(columnStatistics);
+ ++recordsWritten;
+ }
+
+ @Override
+ public void flushBlockingWriter() throws IOException {
+ Path permFileName = new Path(location, prefix + "." + extension);
+ try {
+ if (errStatus) {
+ // Encountered some error
+ throw new IOException("Statistics writer encountered unexpected field");
+ } else if (recordsWritten < 0) {
+ throw new IOException("Statistics writer did not have data");
+ }
+ // Generated the statistics data structure to be serialized
+ statistics = DrillStatsTable.generateDirectoryStructure(dirComputedTime.toString(),
+ columnStatisticsList);
+ if (formatPlugin.supportsStatistics()) {
+ // Invoke the format plugin stats API to write out the stats
+ formatPlugin.writeStatistics(statistics, fs, fileName);
+ // Delete existing permanent file and rename .tmp file to permanent file
+ // If failed to do so then delete the .tmp file
+ fs.delete(permFileName, false);
+ fs.rename(fileName, permFileName);
+ // Cancel delete once perm file is created
+ fs.cancelDeleteOnExit(fileName);
+ fs.cancelDeleteOnExit(new Path(location));
+ }
+ logger.debug("Created file: {}", permFileName);
+ } catch(IOException ex) {
+ logger.error("Unable to create file: " + permFileName, ex);
+ throw ex;
+ }
+ }
+
+ @Override
+ public void abort() throws IOException {
+ // Invoke cleanup to clear any .tmp files and/or empty statistics directory
+ cleanup();
+ }
+
+ @Override
+ public void cleanup() throws IOException {
+ Path permFileName = new Path(location, prefix + "." + extension);
+ try {
+ // Remove the .tmp file, if any
+ if (fs.exists(fileName)) {
+ fs.delete(fileName, false);
+ logger.debug("Deleted file: {}", fileName);
+ }
+ // Also delete the .stats.drill directory if no permanent file exists.
+ if (!fs.exists(permFileName)) {
+ fs.delete(new Path(location), false);
+ logger.debug("Deleted directory: {}", location);
+ }
+ } catch (IOException ex) {
+ logger.error("Unable to delete tmp file: " + fileName, ex);
+ throw ex;
+ }
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
index 2da44753a..9dbe715b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.RecordReader;
import org.apache.drill.exec.store.RecordWriter;
@@ -33,6 +34,7 @@ import org.apache.drill.exec.store.dfs.easy.EasyGroupScan;
import org.apache.drill.exec.store.dfs.easy.EasyWriter;
import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileSplit;
@@ -65,6 +67,21 @@ public class SequenceFileFormatPlugin extends EasyFormatPlugin<SequenceFileForma
}
@Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
public RecordReader getRecordReader(FragmentContext context,
DrillFileSystem dfs,
FileWork fileWork,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index 2ac24d8d0..1c53a3774 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.physical.base.AbstractGroupScan;
import org.apache.drill.exec.physical.base.ScanStats;
import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.planner.physical.PlannerSettings;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
@@ -49,6 +50,7 @@ import org.apache.drill.exec.store.schedule.CompleteFileWork;
import org.apache.drill.exec.store.text.DrillTextRecordReader;
import org.apache.drill.exec.store.text.DrillTextRecordWriter;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileSplit;
@@ -100,6 +102,21 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
}
@Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
protected ScanStats getScanStats(final PlannerSettings settings, final EasyGroupScan scan) {
long data = 0;
for (final CompleteFileWork work : scan.getWorkIterable()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
index f43bb8813..3958a3270 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
@@ -35,6 +35,7 @@ import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.ops.OperatorContext;
import org.apache.drill.exec.physical.impl.OutputMutator;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.AbstractRecordReader;
import org.apache.drill.exec.store.RecordWriter;
@@ -45,6 +46,7 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
@@ -75,6 +77,21 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
Lists.newArrayList(PLUGIN_EXTENSION), PLUGIN_EXTENSION);
}
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
/**
* This class is a POJO to hold the configuration for the HttpdLogFormat Parser. This is automatically
* serialized/deserialized from JSON format.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
index e26ae23b2..15ea1b462 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/image/ImageFormatPlugin.java
@@ -26,6 +26,7 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.RecordReader;
@@ -35,6 +36,8 @@ import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
import org.apache.drill.exec.store.dfs.easy.EasyWriter;
import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class ImageFormatPlugin extends EasyFormatPlugin<ImageFormatConfig> {
@@ -80,4 +83,19 @@ public class ImageFormatPlugin extends EasyFormatPlugin<ImageFormatConfig> {
public boolean supportsPushDown() {
return true;
}
+
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ return null;
+ }
+
+ @Override
+ public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
index 796e1398a..3f381c857 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/log/LogFormatPlugin.java
@@ -18,6 +18,8 @@
package org.apache.drill.exec.store.log;
+import java.io.IOException;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.SchemaPath;
@@ -34,6 +36,8 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.hadoop.conf.Configuration;
import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
@@ -81,4 +85,19 @@ public class LogFormatPlugin extends EasyFormatPlugin<LogFormatConfig> {
public int getWriterOperatorType() {
throw new UnsupportedOperationException("unimplemented");
}
+
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ return null;
+ }
+
+ @Override
+ public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index a3663391b..e2d356953 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -116,6 +116,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
this.matchAllRowGroups = that.matchAllRowGroups;
}
+ @Override
@JsonProperty
public List<SchemaPath> getColumns() {
return columns;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index 876cd5b72..df84a9d1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -17,13 +17,16 @@
*/
package org.apache.drill.exec.store.parquet;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
+import java.io.InputStream;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
-
import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
@@ -35,6 +38,8 @@ import org.apache.drill.exec.physical.base.AbstractWriter;
import org.apache.drill.exec.physical.base.PhysicalOperator;
import org.apache.drill.exec.physical.base.SchemalessScan;
import org.apache.drill.exec.physical.impl.WriterRecordBatch;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
import org.apache.drill.exec.planner.logical.DrillTable;
import org.apache.drill.exec.planner.logical.DynamicDrillTable;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
@@ -48,26 +53,28 @@ import org.apache.drill.exec.store.dfs.BasicFormatMatcher;
import org.apache.drill.exec.store.dfs.DrillFileSystem;
import org.apache.drill.exec.store.dfs.FileSelection;
import org.apache.drill.exec.store.dfs.FileSystemPlugin;
-import org.apache.drill.exec.store.parquet.metadata.Metadata;
-import org.apache.drill.exec.util.DrillFileSystemUtil;
import org.apache.drill.exec.store.dfs.FormatMatcher;
import org.apache.drill.exec.store.dfs.FormatPlugin;
import org.apache.drill.exec.store.dfs.FormatSelection;
import org.apache.drill.exec.store.dfs.MagicString;
import org.apache.drill.exec.store.dfs.MetadataContext;
import org.apache.drill.exec.store.mock.MockStorageEngine;
+import org.apache.drill.exec.store.parquet.metadata.Metadata;
import org.apache.drill.exec.store.parquet.metadata.ParquetTableMetadataDirs;
+import org.apache.drill.exec.util.DrillFileSystemUtil;
+import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileWriter;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-
public class ParquetFormatPlugin implements FormatPlugin {
+
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
public static final ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
@@ -126,15 +133,14 @@ public class ParquetFormatPlugin implements FormatPlugin {
}
@Override
- public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException {
- return new ParquetWriter(child, location, append, partitionColumns, this);
+ public AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) throws IOException {
+ return new ParquetWriter(child, location, partitionColumns, this);
}
public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter writer) throws IOException, OutOfMemoryException {
Map<String, String> options = new HashMap<>();
options.put("location", writer.getLocation());
- options.put("append", Boolean.toString(writer.getAppend()));
FragmentHandle handle = context.getHandle();
String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
@@ -194,6 +200,28 @@ public class ParquetFormatPlugin implements FormatPlugin {
}
@Override
+ public boolean supportsStatistics() {
+ return true;
+ }
+
+ @Override
+ public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ Stopwatch timer = Stopwatch.createStarted();
+ ObjectMapper mapper = DrillStatsTable.getMapper();
+ mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+ FSDataInputStream is = fs.open(statsTablePath);
+ TableStatistics statistics = mapper.readValue((InputStream) is, TableStatistics.class);
+ logger.info("Took {} ms to read statistics from {} format plugin", timer.elapsed(TimeUnit.MILLISECONDS), name);
+ timer.stop();
+ return statistics;
+ }
+
+ @Override
+ public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+ throw new UnsupportedOperationException("unimplemented");
+ }
+
+ @Override
public StoragePluginConfig getStorageConfig() {
return storageConfig;
}
@@ -262,9 +290,9 @@ public class ParquetFormatPlugin implements FormatPlugin {
new FormatSelection(plugin.getConfig(), selection));
}
}
- if (!super.supportDirectoryReads() && selection.containsDirectories(fs)) {
+ /*if (!super.supportDirectoryReads() && selection.containsDirectories(fs)) {
return null;
- }
+ }*/
return super.isReadable(fs, selection, fsPlugin, storageEngineName, schemaConfig);
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
index 6298c1a8c..aea321859 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
@@ -53,7 +53,6 @@ public class ParquetWriter extends AbstractWriter {
public static final int WRITER_VERSION = 2;
private final String location;
- private final boolean append;
private final List<String> partitionColumns;
private final ParquetFormatPlugin formatPlugin;
@@ -61,7 +60,6 @@ public class ParquetWriter extends AbstractWriter {
public ParquetWriter(
@JsonProperty("child") PhysicalOperator child,
@JsonProperty("location") String location,
- @JsonProperty("append") boolean append,
@JsonProperty("partitionColumns") List<String> partitionColumns,
@JsonProperty("storageStrategy") StorageStrategy storageStrategy,
@JsonProperty("storage") StoragePluginConfig storageConfig,
@@ -71,21 +69,18 @@ public class ParquetWriter extends AbstractWriter {
this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
this.location = location;
- this.append = append;
this.partitionColumns = partitionColumns;
setStorageStrategy(storageStrategy);
}
public ParquetWriter(PhysicalOperator child,
String location,
- boolean append,
List<String> partitionColumns,
ParquetFormatPlugin formatPlugin) {
super(child);
this.formatPlugin = formatPlugin;
this.location = location;
- this.append = append;
this.partitionColumns = partitionColumns;
}
@@ -94,11 +89,6 @@ public class ParquetWriter extends AbstractWriter {
return location;
}
- @JsonProperty("append")
- public boolean getAppend() {
- return append;
- }
-
@JsonProperty("storage")
public StoragePluginConfig getStorageConfig(){
return formatPlugin.getStorageConfig();
@@ -121,7 +111,7 @@ public class ParquetWriter extends AbstractWriter {
@Override
protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
- ParquetWriter writer = new ParquetWriter(child, location, append, partitionColumns, formatPlugin);
+ ParquetWriter writer = new ParquetWriter(child, location, partitionColumns, formatPlugin);
writer.setStorageStrategy(getStorageStrategy());
return writer;
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
index 92f97cf82..164082f5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
@@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.store.pcap;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -44,6 +45,8 @@ import org.apache.hadoop.conf.Configuration;
import java.io.IOException;
import java.util.List;
import java.util.regex.Pattern;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class PcapFormatPlugin extends EasyFormatPlugin<PcapFormatConfig> {
@@ -89,6 +92,21 @@ public class PcapFormatPlugin extends EasyFormatPlugin<PcapFormatConfig> {
return this.matcher;
}
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ return null;
+ }
+
+ @Override
+ public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+
+ }
+
private static class PcapFormatMatcher extends BasicFormatMatcher {
public PcapFormatMatcher(PcapFormatPlugin plugin) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngFormatPlugin.java
index 832c0ec3b..de9a558e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcapng/PcapngFormatPlugin.java
@@ -17,9 +17,11 @@
*/
package org.apache.drill.exec.store.pcapng;
+import java.io.IOException;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.StoragePluginConfig;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
import org.apache.drill.exec.proto.UserBitShared;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.RecordReader;
@@ -31,6 +33,8 @@ import org.apache.drill.exec.store.dfs.easy.FileWork;
import org.apache.hadoop.conf.Configuration;
import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
public class PcapngFormatPlugin extends EasyFormatPlugin<PcapngFormatConfig> {
@@ -73,4 +77,19 @@ public class PcapngFormatPlugin extends EasyFormatPlugin<PcapngFormatConfig> {
public int getWriterOperatorType() {
throw new UnsupportedOperationException("unimplemented");
}
+
+ @Override
+ public boolean supportsStatistics() {
+ return false;
+ }
+
+ @Override
+ public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+ return null;
+ }
+
+ @Override
+ public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+
+ }
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
index 3546c73bf..46784ecd0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
@@ -17,14 +17,16 @@
*/
package org.apache.drill.exec.store.pojo;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
-import java.util.ArrayList;
import java.util.List;
+import java.util.ArrayList;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
/**
* Reads values from the given list of pojo instances.
@@ -32,6 +34,7 @@ import java.util.List;
*
* @param <T> pojo class type
*/
+@JsonTypeName("PojoRecordReader")
public class PojoRecordReader<T> extends AbstractPojoRecordReader<T> {
private final Class<T> pojoClass;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
index f72cc7c14..787e24389 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
@@ -51,6 +51,7 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan {
@JsonCreator
public SystemTableScan(@JsonProperty("table") SystemTable table,
@JsonProperty("maxRecordsToRead") int maxRecordsToRead,
+ @JsonProperty("columns") List<SchemaPath> columns,
@JacksonInject StoragePluginRegistry engineRegistry) throws ExecutionSetupException {
this(table, maxRecordsToRead, (SystemTablePlugin) engineRegistry.getPlugin(SystemTablePluginConfig.INSTANCE));
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
index d3967eb66..2446ba7e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
@@ -17,9 +17,7 @@
*/
package org.apache.drill.exec.util;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.apache.drill.shaded.guava.com.google.common.base.Predicate;
-import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
+import java.util.Collection;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rex.RexLiteral;
import org.apache.drill.common.expression.PathSegment;
@@ -31,8 +29,9 @@ import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
import org.apache.drill.exec.proto.ExecProtos;
import org.apache.drill.exec.proto.helper.QueryIdHelper;
-
-import java.util.Collection;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.base.Predicate;
+import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
public class Utilities {
@@ -60,9 +59,11 @@ public class Utilities {
* QueryContextInformation is derived from the current state of the process.
*
* @param defaultSchemaName
+ * @param sessionId
* @return A {@link org.apache.drill.exec.proto.BitControl.QueryContextInformation} with given <i>defaultSchemaName</i>.
*/
- public static QueryContextInformation createQueryContextInfo(final String defaultSchemaName, final String sessionId) {
+ public static QueryContextInformation createQueryContextInfo(final String defaultSchemaName,
+ final String sessionId) {
final long queryStartTime = System.currentTimeMillis();
final int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
return QueryContextInformation.newBuilder()
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 6b9fe6fbf..386b8c2fa 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -584,6 +584,7 @@ drill.exec.options: {
planner.partitioner_sender_threads_factor: 2,
planner.producer_consumer_queue_size: 10,
planner.slice_target: 100000,
+ planner.statistics.use: false,
planner.store.parquet.rowgroup.filter.pushdown.enabled: true,
planner.store.parquet.rowgroup.filter.pushdown.threshold: 10000,
# Max per node should always be configured as zero and
@@ -651,7 +652,7 @@ drill.exec.options: {
web.display_format.date: "",
web.display_format.time: "",
window.enable: true,
- storage.list_files_recursively: false
+ storage.list_files_recursively: false,
# ============ index plan related options ==============
planner.use_simple_optimizer: false,
planner.enable_index_planning: true,
@@ -668,4 +669,10 @@ drill.exec.options: {
planner.index.max_indexes_to_intersect: 5,
exec.query.rowkeyjoin_batchsize: 128,
exec.query.return_result_set_for_ddl: true
+ exec.return_result_set_for_ddl: true,
+ storage.list_files_recursively: false,
+ exec.statistics.ndv_accuracy: 20,
+ exec.statistics.ndv_extrapolation_bf_elements: 1000000,
+ exec.statistics.ndv_extrapolation_bf_fpprobability: 10,
+ exec.statistics.deterministic_sampling: false
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestSelectivity.java b/exec/java-exec/src/test/java/org/apache/drill/TestSelectivity.java
index 5589b172b..b0840a18f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestSelectivity.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestSelectivity.java
@@ -63,8 +63,8 @@ public class TestSelectivity extends BaseTestQuery {
"Scan.*columns=\\[`employee_id`, `department_id`\\].*rowcount = 463.0.*"};
PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan1, new String[]{});
- test(String.format("alter session set `planner.filter.min_selectivity_estimate_factor` = %f", 0.9));
- final String[] expectedPlan2 = {"Filter\\(condition.*\\).*rowcount = 416.7,.*",
+ test(String.format("alter session set `planner.filter.min_selectivity_estimate_factor` = %f", 0.7));
+ final String[] expectedPlan2 = {"Filter\\(condition.*\\).*rowcount = 324.0.*",
"Scan.*columns=\\[`employee_id`, `department_id`\\].*rowcount = 463.0.*"};
PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan2, new String[]{});
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
index 03754e536..2c6e4eefa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
@@ -17,31 +17,31 @@
*/
package org.apache.drill.exec.impersonation;
-import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import java.util.Map;
import org.apache.drill.categories.SecurityTest;
+import org.apache.drill.categories.SlowTest;
import org.apache.drill.categories.UnlikelyTest;
import org.apache.drill.common.exceptions.UserException;
import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.exec.dotdrill.DotDrillType;
import org.apache.drill.exec.store.dfs.WorkspaceConfig;
-import org.apache.drill.categories.SlowTest;
+import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
+import static org.hamcrest.core.StringContains.containsString;
import org.junit.AfterClass;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
-import java.util.Map;
-
-import static org.hamcrest.core.StringContains.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
/**
* Tests impersonation on metadata related queries as SHOW FILES, SHOW TABLES, CREATE VIEW, CREATE TABLE and DROP TABLE
*/
@@ -377,6 +377,31 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
}
+ @Test
+ public void testAnalyzeTable() throws Exception {
+ final String tableName = "nation1_stats";
+ final String tableWS = "drill_test_grp_1_700";
+
+ updateClient(user1);
+ test("USE " + Joiner.on(".").join(MINI_DFS_STORAGE_PLUGIN_NAME, tableWS));
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE " + tableName + " AS SELECT * FROM cp.`tpch/nation.parquet`;");
+ test("ANALYZE TABLE " + tableName + " COMPUTE STATISTICS;");
+ test("SELECT * FROM " + tableName + ";");
+
+ final Path statsFilePath = new Path(Path.SEPARATOR + tableWS + Path.SEPARATOR + tableName
+ + Path.SEPARATOR + DotDrillType.STATS.getEnding());
+ assertTrue (fs.exists(statsFilePath) && fs.isDirectory(statsFilePath));
+ FileStatus status = fs.getFileStatus(statsFilePath);
+ // Verify process user is the directory owner
+ assert(processUser.equalsIgnoreCase(status.getOwner()));
+
+ fs.mkdirs(new Path(statsFilePath, "tmp5"));
+
+ test("SELECT * from " + tableName + ";");
+ test("DROP TABLE " + tableName);
+ }
+
@AfterClass
public static void removeMiniDfsBasedStorage() {
getDrillbitContext().getStorage().deletePlugin(MINI_DFS_STORAGE_PLUGIN_NAME);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
index ae73087b6..b45acfbc8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
@@ -394,7 +394,8 @@ public class TestLocalExchange extends PlanTestBase {
findFragmentsWithPartitionSender(rootFragment, planningSet, deMuxFragments, htrFragments);
- final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
+ final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName",
+ "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
QueryWorkUnit qwu = PARALLELIZER.getFragments(new OptionList(), drillbitContext.getEndpoint(),
QueryId.getDefaultInstance(),
drillbitContext.getBits(), rootFragment, USER_SESSION, queryContextInfo);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
index d16a85100..0986c75d6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
@@ -207,7 +207,8 @@ public class TestPartitionSender extends PlanTestBase {
RecordBatch incoming, FunctionImplementationRegistry registry, PhysicalPlanReader planReader, PlanningSet planningSet, Fragment rootFragment,
int expectedThreadsCount) throws Exception {
- final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
+ final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName",
+ "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
final QueryWorkUnit qwu = PARALLELIZER.getFragments(options, drillbitContext.getEndpoint(),
QueryId.getDefaultInstance(),
drillbitContext.getBits(), rootFragment, USER_SESSION, queryContextInfo);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
index 68d0daa9f..7b53bb65c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
@@ -63,7 +63,8 @@ public class TestFragmentChecker extends PopUnitTestBase{
endpoints.add(b1);
}
- final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
+ final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName",
+ "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e");
QueryWorkUnit qwu = par.getFragments(new OptionList(), localBit, QueryId.getDefaultInstance(), endpoints, fragmentRoot,
UserSession.Builder.newBuilder().withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()).build(),
queryContextInfo);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
index 0f15fb300..04e9b99aa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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.
@@ -17,41 +17,57 @@
*/
package org.apache.drill.exec.sql;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.drill.PlanTestBase;
import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.test.BaseTestQuery;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Test;
-public class TestAnalyze extends PlanTestBase {
+public class TestAnalyze extends BaseTestQuery {
+
+ @BeforeClass
+ public static void copyData() throws Exception {
+ dirTestWatcher.copyResourceToRoot(Paths.get("multilevel", "parquet"));
+ }
// Analyze for all columns
@Test
public void basic1() throws Exception {
try {
test("ALTER SESSION SET `planner.slice_target` = 1");
- test("CREATE TABLE dfs_test.tmp.region_basic1 AS SELECT * from cp.`region.json`");
- test("ANALYZE TABLE dfs_test.tmp.region_basic1 COMPUTE STATISTICS FOR ALL COLUMNS");
- test("SELECT * FROM dfs_test.tmp.`region_basic1/.stats.drill`");
-
- testBuilder()
- .sqlQuery("SELECT `column`, statcount, nonnullstatcount, ndv FROM dfs_test.tmp.`region_basic1/.stats.drill`")
- .unOrdered()
- .baselineColumns("column", "statcount", "nonnullstatcount", "ndv")
- .baselineValues("region_id", 110L, 110L, 107L)
- .baselineValues("sales_city", 110L, 110L, 111L)
- .baselineValues("sales_state_province", 110L, 110L, 13L)
- .baselineValues("sales_district", 110L, 110L, 22L)
- .baselineValues("sales_region", 110L, 110L, 8L)
- .baselineValues("sales_country", 110L, 110L, 4L)
- .baselineValues("sales_district_id", 110L, 110L, 23L)
- .go();
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.region_basic1 AS SELECT * from cp.`region.json`");
+ test("ANALYZE TABLE dfs.tmp.region_basic1 COMPUTE STATISTICS");
+ test("SELECT * FROM dfs.tmp.`region_basic1/.stats.drill`");
+ test("create table dfs.tmp.flatstats1 as select flatten(`directories`[0].`columns`) as `columns`"
+ + " from dfs.tmp.`region_basic1/.stats.drill`");
- // we can't compare the ndv for correctness as it is an estimate and not accurate
testBuilder()
- .sqlQuery("SELECT statcount FROM dfs_test.tmp.`region_basic1/.stats.drill` WHERE `column` = 'region_id'")
+ .sqlQuery("SELECT tbl.`columns`.`column` as `column`, tbl.`columns`.rowcount as rowcount,"
+ + " tbl.`columns`.nonnullrowcount as nonnullrowcount, tbl.`columns`.ndv as ndv,"
+ + " tbl.`columns`.avgwidth as avgwidth"
+ + " FROM dfs.tmp.flatstats1 tbl")
.unOrdered()
- .sqlBaselineQuery("SELECT count(region_id) AS statcount FROM dfs_test.tmp.region_basic1")
+ .baselineColumns("column", "rowcount", "nonnullrowcount", "ndv", "avgwidth")
+ .baselineValues("`region_id`", 110.0, 110.0, 110L, 8.0)
+ .baselineValues("`sales_city`", 110.0, 110.0, 109L, 8.663636363636364)
+ .baselineValues("`sales_state_province`", 110.0, 110.0, 13L, 2.4272727272727272)
+ .baselineValues("`sales_district`", 110.0, 110.0, 23L, 9.318181818181818)
+ .baselineValues("`sales_region`", 110.0, 110.0, 8L, 10.8)
+ .baselineValues("`sales_country`", 110.0, 110.0, 4L, 3.909090909090909)
+ .baselineValues("`sales_district_id`", 110.0, 110.0, 23L, 8.0)
.go();
-
} finally {
test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
}
@@ -62,26 +78,53 @@ public class TestAnalyze extends PlanTestBase {
public void basic2() throws Exception {
try {
test("ALTER SESSION SET `planner.slice_target` = 1");
- test("CREATE TABLE dfs_test.tmp.employee_basic2 AS SELECT * from cp.`employee.json`");
- test("ANALYZE TABLE dfs_test.tmp.employee_basic2 COMPUTE STATISTICS FOR COLUMNS (employee_id, birth_date)");
- test("SELECT * FROM dfs_test.tmp.`employee_basic2/.stats.drill`");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.employee_basic2 AS SELECT * from cp.`employee.json`");
+ test("ANALYZE TABLE dfs.tmp.employee_basic2 COMPUTE STATISTICS (employee_id, birth_date)");
+ test("SELECT * FROM dfs.tmp.`employee_basic2/.stats.drill`");
+ test("create table dfs.tmp.flatstats2 as select flatten(`directories`[0].`columns`) as `columns`"
+ + " from dfs.tmp.`employee_basic2/.stats.drill`");
testBuilder()
- .sqlQuery("SELECT `column`, statcount, nonnullstatcount, ndv FROM dfs_test.tmp.`employee_basic2/.stats.drill`")
+ .sqlQuery("SELECT tbl.`columns`.`column` as `column`, tbl.`columns`.rowcount as rowcount,"
+ + " tbl.`columns`.nonnullrowcount as nonnullrowcount, tbl.`columns`.ndv as ndv,"
+ + " tbl.`columns`.avgwidth as avgwidth"
+ + " FROM dfs.tmp.flatstats2 tbl")
.unOrdered()
- .baselineColumns("column", "statcount", "nonnullstatcount", "ndv")
- .baselineValues("employee_id", 1155L, 1155L, 1144L)
- .baselineValues("birth_date", 1155L, 1155L, 53L)
+ .baselineColumns("column", "rowcount", "nonnullrowcount", "ndv", "avgwidth")
+ .baselineValues("`employee_id`", 1155.0, 1155.0, 1155L, 8.0)
+ .baselineValues("`birth_date`", 1155.0, 1155.0, 52L, 10.0)
.go();
+ } finally {
+ test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+ }
+ }
- // we can't compare the ndv for correctness as it is an estimate and not accurate
- testBuilder()
- .sqlQuery("SELECT statcount FROM dfs_test.tmp.`employee_basic2/.stats.drill` WHERE `column` = 'birth_date'")
- .unOrdered()
- .sqlBaselineQuery("SELECT count(birth_date) AS statcount FROM dfs_test.tmp.employee_basic2")
- .go();
+ // Analyze with sampling percentage
+ @Test
+ public void basic3() throws Exception {
+ try {
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("ALTER SESSION SET `exec.statistics.deterministic_sampling` = true");
+ test("CREATE TABLE dfs.tmp.employee_basic3 AS SELECT * from cp.`employee.json`");
+ test("ANALYZE TABLE dfs.tmp.employee_basic3 COMPUTE STATISTICS (employee_id, birth_date) SAMPLE 55 PERCENT");
+ test("SELECT * FROM dfs.tmp.`employee_basic3/.stats.drill`");
+ test("create table dfs.tmp.flatstats3 as select flatten(`directories`[0].`columns`) as `columns`"
+ + " from dfs.tmp.`employee_basic3/.stats.drill`");
+ testBuilder()
+ .sqlQuery("SELECT tbl.`columns`.`column` as `column`, tbl.`columns`.rowcount as rowcount,"
+ + " tbl.`columns`.nonnullrowcount as nonnullrowcount, tbl.`columns`.ndv as ndv,"
+ + " tbl.`columns`.avgwidth as avgwidth"
+ + " FROM dfs.tmp.flatstats3 tbl")
+ .unOrdered()
+ .baselineColumns("column", "rowcount", "nonnullrowcount", "ndv", "avgwidth")
+ .baselineValues("`employee_id`", 1138.0, 1138.0, 1138L, 8.00127815945039)
+ .baselineValues("`birth_date`", 1138.0, 1138.0, 38L, 10.001597699312988)
+ .go();
} finally {
+ test("ALTER SESSION SET `exec.statistics.deterministic_sampling` = false");
test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
}
}
@@ -90,16 +133,258 @@ public class TestAnalyze extends PlanTestBase {
public void join() throws Exception {
try {
test("ALTER SESSION SET `planner.slice_target` = 1");
- test("CREATE TABLE dfs_test.tmp.lineitem AS SELECT * FROM cp.`tpch/lineitem.parquet`");
- test("CREATE TABLE dfs_test.tmp.orders AS select * FROM cp.`tpch/orders.parquet`");
- test("ANALYZE TABLE dfs_test.tmp.lineitem COMPUTE STATISTICS FOR ALL COLUMNS");
- test("ANALYZE TABLE dfs_test.tmp.orders COMPUTE STATISTICS FOR ALL COLUMNS");
- test("SELECT * FROM dfs_test.tmp.`lineitem/.stats.drill`");
- test("SELECT * FROM dfs_test.tmp.`orders/.stats.drill`");
-
- test("SELECT * FROM dfs_test.tmp.`lineitem` l JOIN dfs_test.tmp.`orders` o ON l.l_orderkey = o.o_orderkey");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.lineitem AS SELECT * FROM cp.`tpch/lineitem.parquet`");
+ test("CREATE TABLE dfs.tmp.orders AS select * FROM cp.`tpch/orders.parquet`");
+ test("ANALYZE TABLE dfs.tmp.lineitem COMPUTE STATISTICS");
+ test("ANALYZE TABLE dfs.tmp.orders COMPUTE STATISTICS");
+ test("SELECT * FROM dfs.tmp.`lineitem/.stats.drill`");
+ test("SELECT * FROM dfs.tmp.`orders/.stats.drill`");
+ test("ALTER SESSION SET `planner.statistics.use` = true");
+ test("SELECT * FROM dfs.tmp.`lineitem` l JOIN dfs.tmp.`orders` o ON l.l_orderkey = o.o_orderkey");
+ } finally {
+ test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+ }
+ }
+
+ @Test
+ public void testAnalyzeSupportedFormats() throws Exception {
+ //Only allow computing statistics on PARQUET files.
+ try {
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'json'");
+ test("CREATE TABLE dfs.tmp.employee_basic4 AS SELECT * from cp.`employee.json`");
+ //Should display not supported
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.employee_basic4 COMPUTE STATISTICS",
+ "Table employee_basic4 is not supported by ANALYZE. "
+ + "Support is currently limited to directory-based Parquet tables.");
+
+ test("DROP TABLE dfs.tmp.employee_basic4");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.employee_basic4 AS SELECT * from cp.`employee.json`");
+ //Should complete successfully (16 columns in employee.json)
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.employee_basic4 COMPUTE STATISTICS",
+ "16");
+ } finally {
+ test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+ }
+ }
+
+ @Test
+ public void testAnalyzePartitionedTables() throws Exception {
+ //Computing statistics on columns, dir0, dir1
+ try {
+ final String tmpLocation = "/multilevel/parquet";
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.parquet1 AS SELECT * from dfs.`%s`", tmpLocation);
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquet1 COMPUTE STATISTICS", "11");
+ test("SELECT * FROM dfs.tmp.`parquet1/.stats.drill`");
+ test("create table dfs.tmp.flatstats4 as select flatten(`directories`[0].`columns`) as `columns` " +
+ "from dfs.tmp.`parquet1/.stats.drill`");
+ //Verify statistics
+ testBuilder()
+ .sqlQuery("SELECT tbl.`columns`.`column` as `column`, tbl.`columns`.rowcount as rowcount,"
+ + " tbl.`columns`.nonnullrowcount as nonnullrowcount, tbl.`columns`.ndv as ndv,"
+ + " tbl.`columns`.avgwidth as avgwidth"
+ + " FROM dfs.tmp.flatstats4 tbl")
+ .unOrdered()
+ .baselineColumns("column", "rowcount", "nonnullrowcount", "ndv", "avgwidth")
+ .baselineValues("`o_orderkey`", 120.0, 120.0, 119L, 4.0)
+ .baselineValues("`o_custkey`", 120.0, 120.0, 113L, 4.0)
+ .baselineValues("`o_orderstatus`", 120.0, 120.0, 3L, 1.0)
+ .baselineValues("`o_totalprice`", 120.0, 120.0, 120L, 8.0)
+ .baselineValues("`o_orderdate`", 120.0, 120.0, 111L, 4.0)
+ .baselineValues("`o_orderpriority`", 120.0, 120.0, 5L, 8.458333333333334)
+ .baselineValues("`o_clerk`", 120.0, 120.0, 114L, 15.0)
+ .baselineValues("`o_shippriority`", 120.0, 120.0, 1L, 4.0)
+ .baselineValues("`o_comment`", 120.0, 120.0, 120L, 46.333333333333336)
+ .baselineValues("`dir0`", 120.0, 120.0, 3L, 4.0)
+ .baselineValues("`dir1`", 120.0, 120.0, 4L, 2.0)
+ .go();
} finally {
test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
}
}
+
+ @Test
+ public void testStaleness() throws Exception {
+ // copy the data into the temporary location
+ final String tmpLocation = "/multilevel/parquet";
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.parquetStale AS SELECT o_orderkey, o_custkey, o_orderstatus, " +
+ "o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment from dfs.`%s`", tmpLocation);
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS", "9");
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS",
+ "Table parquetStale has not changed since last ANALYZE!");
+ // Verify we recompute statistics once a new file/directory is added. Update the directory some
+ // time after ANALYZE so that the timestamps are different.
+ Thread.sleep(1000);
+ final String Q4 = "/multilevel/parquet/1996/Q4";
+ test("CREATE TABLE dfs.tmp.`parquetStale/1996/Q5` AS SELECT o_orderkey, o_custkey, o_orderstatus, " +
+ "o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment from dfs.`%s`", Q4);
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS", "9");
+ Thread.sleep(1000);
+ test("DROP TABLE dfs.tmp.`parquetStale/1996/Q5`");
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS", "9");
+ }
+
+ @Test
+ public void testUseStatistics() throws Exception {
+ //Test ndv/rowcount for scan
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("CREATE TABLE dfs.tmp.employeeUseStat AS SELECT * from cp.`employee.json`");
+ test("CREATE TABLE dfs.tmp.departmentUseStat AS SELECT * from cp.`department.json`");
+ test("ANALYZE TABLE dfs.tmp.employeeUseStat COMPUTE STATISTICS");
+ test("ANALYZE TABLE dfs.tmp.departmentUseStat COMPUTE STATISTICS");
+ test("ALTER SESSION SET `planner.statistics.use` = true");
+ String query = " select employee_id from dfs.tmp.employeeUseStat where department_id = 2";
+ String[] expectedPlan1 = {"Filter\\(condition.*\\).*rowcount = 96.25,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan1, new String[]{});
+
+ query = " select employee_id from dfs.tmp.employeeUseStat where department_id IN (2, 5)";
+ String[] expectedPlan2 = {"Filter\\(condition.*\\).*rowcount = 192.5,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan2, new String[]{});
+
+ query = "select employee_id from dfs.tmp.employeeUseStat where department_id IN (2, 5) and employee_id = 5";
+ String[] expectedPlan3 = {"Filter\\(condition.*\\).*rowcount = 1.0,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan3, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept"
+ + " on emp.department_id = dept.department_id";
+ String[] expectedPlan4 = {"HashJoin\\(condition.*\\).*rowcount = 1154.9999999999995,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan4, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept"
+ + " on emp.department_id = dept.department_id where dept.department_id = 5";
+ String[] expectedPlan5 = {"HashJoin\\(condition.*\\).*rowcount = 96.24999999999997,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan5, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept"
+ + " on emp.department_id = dept.department_id"
+ + " where dept.department_id = 5 and emp.employee_id = 10";
+ String[] expectedPlan6 = {"MergeJoin\\(condition.*\\).*rowcount = 1.0,.*",
+ "Filter\\(condition=\\[AND\\(=\\(\\$1, 10\\), =\\(\\$0, 5\\)\\)\\]\\).*rowcount = 1.0,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*",
+ "Filter\\(condition=\\[=\\(\\$0, 5\\)\\]\\).*rowcount = 1.0,.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan6, new String[]{});
+
+ query = " select emp.employee_id, count(*)"
+ + " from dfs.tmp.employeeUseStat emp"
+ + " group by emp.employee_id";
+ String[] expectedPlan7 = {"HashAgg\\(group=\\[\\{0\\}\\], EXPR\\$1=\\[COUNT\\(\\)\\]\\).*rowcount = 1155.0,.*",
+ "Scan.*columns=\\[`employee_id`\\].*rowcount = 1155.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan7, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept"
+ + " on emp.department_id = dept.department_id "
+ + " group by emp.employee_id";
+ String[] expectedPlan8 = {"HashAgg\\(group=\\[\\{0\\}\\]\\).*rowcount = 730.0992454469839,.*",
+ "HashJoin\\(condition.*\\).*rowcount = 1154.9999999999995,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan8, new String[]{});
+
+ query = "select emp.employee_id, dept.department_description"
+ + " from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept"
+ + " on emp.department_id = dept.department_id "
+ + " group by emp.employee_id, emp.store_id, dept.department_description "
+ + " having dept.department_description = 'FINANCE'";
+ String[] expectedPlan9 = {"HashAgg\\(group=\\[\\{0, 1, 2\\}\\]\\).*rowcount = 92.3487011031316.*",
+ "HashJoin\\(condition.*\\).*rowcount = 96.24999999999997,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`, `store_id`\\].*rowcount = 1155.0.*",
+ "Filter\\(condition=\\[=\\(\\$1, 'FINANCE'\\)\\]\\).*rowcount = 1.0,.*",
+ "Scan.*columns=\\[`department_id`, `department_description`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan9, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept\n"
+ + " on emp.department_id = dept.department_id "
+ + " group by emp.employee_id, emp.store_id "
+ + " having emp.store_id = 7";
+ String[] expectedPlan10 = {"HashAgg\\(group=\\[\\{0, 1\\}\\]\\).*rowcount = 29.203969817879365.*",
+ "HashJoin\\(condition.*\\).*rowcount = 46.2,.*",
+ "Filter\\(condition=\\[=\\(\\$2, 7\\)\\]\\).*rowcount = 46.2,.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`, `store_id`\\].*rowcount = 1155.0.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan10, new String[]{});
+
+ query = " select emp.employee_id from dfs.tmp.employeeUseStat emp join dfs.tmp.departmentUseStat dept\n"
+ + " on emp.department_id = dept.department_id "
+ + " group by emp.employee_id "
+ + " having emp.employee_id = 7";
+ String[] expectedPlan11 = {"StreamAgg\\(group=\\[\\{0\\}\\]\\).*rowcount = 1.0.*",
+ "HashJoin\\(condition.*\\).*rowcount = 1.0,.*",
+ "Filter\\(condition=\\[=\\(\\$1, 7\\)\\]\\).*rowcount = 1.0.*",
+ "Scan.*columns=\\[`department_id`\\].*rowcount = 12.0.*",
+ "Scan.*columns=\\[`department_id`, `employee_id`\\].*rowcount = 1155.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan11, new String[]{});
+ }
+
+ @Ignore("Fails intermittently. Enable after fixing the issue.")
+ @Test
+ public void testWithMetadataCaching() throws Exception {
+ test("ALTER SESSION SET `planner.slice_target` = 1");
+ test("ALTER SESSION SET `store.format` = 'parquet'");
+ test("ALTER SESSION SET `planner.statistics.use` = true");
+ final String tmpLocation = "/multilevel/parquet";
+ // copy the data into the temporary location
+ test("DROP TABLE dfs.tmp.parquetStale");
+ test("CREATE TABLE dfs.tmp.parquetStale AS SELECT o_orderkey, o_custkey, o_orderstatus, " +
+ "o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment from dfs.`%s`", tmpLocation);
+ String query = "select count(distinct o_orderkey) from dfs.tmp.parquetStale";
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS", "9");
+ test("REFRESH TABLE METADATA dfs.tmp.parquetStale");
+ // Verify we recompute statistics once a new file/directory is added. Update the directory some
+ // time after ANALYZE so that the timestamps are different.
+ Thread.sleep(1000);
+ final String Q4 = "/multilevel/parquet/1996/Q4";
+ test("CREATE TABLE dfs.tmp.`parquetStale/1996/Q5` AS SELECT o_orderkey, o_custkey, o_orderstatus, " +
+ "o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment from dfs.`%s`", Q4);
+ // query should use STALE statistics
+ String[] expectedStalePlan = {"StreamAgg\\(group=\\[\\{0\\}\\]\\).*rowcount = 119.0.*",
+ "Scan.*rowcount = 130.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedStalePlan, new String[]{});
+ // Query should use Parquet Metadata, since statistics not available. In this case, NDV is computed as
+ // 1/10*rowcount (Calcite default). Hence, NDV is 13.0 instead of the correct 119.0
+ test("DROP TABLE dfs.tmp.`parquetStale/.stats.drill`");
+ String[] expectedPlan1 = {"HashAgg\\(group=\\[\\{0\\}\\]\\).*rowcount = 13.0.*",
+ "Scan.*rowcount = 130.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan1, new String[]{});
+ // query should use the new statistics. NDV remains unaffected since we copy the Q4 into Q5
+ verifyAnalyzeOutput("ANALYZE TABLE dfs.tmp.parquetStale COMPUTE STATISTICS", "9");
+ String[] expectedPlan2 = {"StreamAgg\\(group=\\[\\{0\\}\\]\\).*rowcount = 119.0.*",
+ "Scan.*rowcount = 130.0.*"};
+ PlanTestBase.testPlanWithAttributesMatchingPatterns(query, expectedPlan2, new String[]{});
+ test("DROP TABLE dfs.tmp.`parquetStale/1996/Q5`");
+ }
+
+ //Helper function to verify output of ANALYZE statement
+ private void verifyAnalyzeOutput(String query, String message) throws Exception {
+ List<QueryDataBatch>result = testRunAndReturn(QueryType.SQL, query);
+ List<List<String>> output = new ArrayList<>();
+ assertTrue(result.size() == 1);
+ final QueryDataBatch batch = result.get(0);
+ final RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
+ loader.load(batch.getHeader().getDef(), batch.getData());
+ output.add(new ArrayList<String>());
+ for (VectorWrapper<?> vw: loader) {
+ ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
+ Object o = accessor.getObject(0);
+ output.get(0).add(o == null ? null: o.toString());
+ }
+ batch.release();
+ loader.clear();
+ assertTrue(output.get(0).size() == 2);
+ assertEquals(message, output.get(0).get(1));
+ }
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
index 3d071cb85..cc21b43ea 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
@@ -67,7 +67,7 @@ public class TestSqlBracketlessSyntax {
"ORDER BY `x`.`a`['a']['a']['a']['a']";
- SqlNode rewritten = node.accept(new CompoundIdentifierConverter());
+ SqlNode rewritten = node.accept(new CompoundIdentifierConverter(false));
String rewrittenQuery = rewritten.toString();
DrillAssert.assertMultiLineStringEquals(expected, rewrittenQuery);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
index 68442d058..88bb03b90 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
@@ -17,6 +17,8 @@
*/
package org.apache.drill.test;
+import java.math.BigDecimal;
+import java.math.BigInteger;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
@@ -108,6 +110,10 @@ public class DrillTestWrapper {
// queries to run before the baseline or test queries, can be used to set options
private String baselineOptionSettingQueries;
private String testOptionSettingQueries;
+ // allow approximate equality tests for number types
+ private boolean approximateEquality;
+ // tolerance for approximate equality tests defined as |Expected - Actual|/|Expected| <= Tolerance
+ private double tolerance;
// two different methods are available for comparing ordered results, the default reads all of the records
// into giant lists of objects, like one giant on-heap batch of 'vectors'
// this flag enables the other approach which iterates through a hyper batch for the test query results and baseline
@@ -125,7 +131,8 @@ public class DrillTestWrapper {
public DrillTestWrapper(TestBuilder testBuilder, TestServices services, Object query, QueryType queryType,
String baselineOptionSettingQueries, String testOptionSettingQueries,
- QueryType baselineQueryType, boolean ordered, boolean highPerformanceComparison,
+ QueryType baselineQueryType, boolean ordered, boolean approximateEquality, double tolerance,
+ boolean highPerformanceComparison,
String[] baselineColumns, List<Map<String, Object>> baselineRecords, int expectedNumBatches,
int expectedNumRecords) {
this.testBuilder = testBuilder;
@@ -134,6 +141,8 @@ public class DrillTestWrapper {
this.queryType = queryType;
this.baselineQueryType = baselineQueryType;
this.ordered = ordered;
+ this.approximateEquality = approximateEquality;
+ this.tolerance = tolerance;
this.baselineOptionSettingQueries = baselineOptionSettingQueries;
this.testOptionSettingQueries = testOptionSettingQueries;
this.highPerformanceComparison = highPerformanceComparison;
@@ -781,6 +790,11 @@ public class DrillTestWrapper {
}
public static boolean compareValues(Object expected, Object actual, int counter, String column) throws Exception {
+ return compareValues(expected, actual, counter, column, false, 0);
+ }
+
+ public static boolean compareValues(Object expected, Object actual, int counter, String column,
+ boolean approximateEquality, double tolerance) throws Exception {
if (expected == null) {
if (actual == null) {
if (VERBOSE_DEBUG) {
@@ -805,6 +819,24 @@ public class DrillTestWrapper {
}
}
if (!expected.equals(actual)) {
+ if (approximateEquality && expected instanceof Number && actual instanceof Number) {
+ if (expected instanceof BigDecimal && actual instanceof BigDecimal) {
+ if (((((BigDecimal) expected).subtract((BigDecimal) actual)).abs().divide((BigDecimal) expected).abs()).compareTo(BigDecimal.valueOf(tolerance)) <= 0) {
+ return true;
+ }
+ } else if (expected instanceof BigInteger && actual instanceof BigInteger) {
+ BigDecimal expBD = new BigDecimal((BigInteger)expected);
+ BigDecimal actBD = new BigDecimal((BigInteger)actual);
+ if ((expBD.subtract(actBD)).abs().divide(expBD.abs()).compareTo(BigDecimal.valueOf(tolerance)) <= 0) {
+ return true;
+ }
+ } else if (!(expected instanceof BigDecimal || expected instanceof BigInteger) && !(actual instanceof BigDecimal || actual instanceof BigInteger)) {
+ // For all other types cast to double and compare
+ if (Math.abs((double) expected - (double) actual) / Math.abs((double) expected) <= tolerance) {
+ return true;
+ }
+ }
+ }
return false;
} else {
if (VERBOSE_DEBUG) {
@@ -836,7 +868,7 @@ public class DrillTestWrapper {
if (!expectedRecord.containsKey(s)) {
throw new Exception("Unexpected column '" + s + "' returned by query.");
}
- if (! compareValues(expectedRecord.get(s), actualRecord.get(s), counter, s)) {
+ if (! compareValues(expectedRecord.get(s), actualRecord.get(s), counter, s, approximateEquality, tolerance)) {
i++;
continue findMatch;
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index 6c4cdadb4..757b619a8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -63,6 +63,7 @@ public class TestBuilder {
// should the validation enforce ordering
private Boolean ordered;
private boolean approximateEquality;
+ private double tolerance;
private TestServices services;
// Used to pass the type information associated with particular column names rather than relying on the
// ordering of the columns in the CSV file, or the default type inferences when reading JSON, this is used for the
@@ -122,6 +123,7 @@ public class TestBuilder {
query = "";
ordered = null;
approximateEquality = false;
+ tolerance = 0.1;
highPerformanceComparison = false;
testOptionSettingQueries = "";
baselineOptionSettingQueries = "";
@@ -131,7 +133,8 @@ public class TestBuilder {
public DrillTestWrapper build() {
return new DrillTestWrapper(this, services, query, queryType, baselineOptionSettingQueries, testOptionSettingQueries,
- getValidationQueryType(), ordered, highPerformanceComparison, baselineColumns, baselineRecords, expectedNumBatches, expectedNumRecords);
+ getValidationQueryType(), ordered, approximateEquality, tolerance, highPerformanceComparison, baselineColumns,
+ baselineRecords, expectedNumBatches, expectedNumRecords);
}
public List<Pair<SchemaPath, TypeProtos.MajorType>> getExpectedSchema() {
@@ -218,7 +221,12 @@ public class TestBuilder {
}
public TestBuilder approximateEquality() {
+ return approximateEquality(0.1);
+ }
+
+ public TestBuilder approximateEquality(double tolerance) {
approximateEquality = true;
+ this.tolerance = tolerance;
return this;
}
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java b/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
index 711050d70..3b12595f8 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
+++ b/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -25,11 +25,22 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
@JsonTypeName("analyze")
public class Analyze extends SingleInputOperator {
+ double samplePercent = 100.0;
+
@JsonCreator
public Analyze() { }
+ @JsonCreator
+ public Analyze(double samplePercent) {
+ this.samplePercent = samplePercent;
+ }
+
@Override
public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
return logicalVisitor.visitAnalyze(this, value);
}
+
+ public double getSamplePercent() {
+ return samplePercent;
+ }
}
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 635f972d9..2135a1f19 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -609,6 +609,10 @@ public final class UserBitShared {
* <code>UNPIVOT_MAPS = 60;</code>
*/
UNPIVOT_MAPS(60, 60),
+ /**
+ * <code>STATISTICS_MERGE = 61;</code>
+ */
+ STATISTICS_MERGE(61, 61),
;
/**
@@ -855,6 +859,10 @@ public final class UserBitShared {
* <code>UNPIVOT_MAPS = 60;</code>
*/
public static final int UNPIVOT_MAPS_VALUE = 60;
+ /**
+ * <code>STATISTICS_MERGE = 61;</code>
+ */
+ public static final int STATISTICS_MERGE_VALUE = 61;
public final int getNumber() { return value; }
@@ -922,6 +930,7 @@ public final class UserBitShared {
case 58: return SYSLOG_SUB_SCAN;
case 59: return STATISTICS_AGGREGATE;
case 60: return UNPIVOT_MAPS;
+ case 61: return STATISTICS_MERGE;
default: return null;
}
}
@@ -24662,7 +24671,7 @@ public final class UserBitShared {
"entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA" +
"TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN" +
"CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ" +
- "UESTED\020\006*\323\t\n\020CoreOperatorType\022\021\n\rSINGLE_" +
+ "UESTED\020\006*\351\t\n\020CoreOperatorType\022\021\n\rSINGLE_" +
"SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER" +
"\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n" +
"\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006" +
@@ -24693,10 +24702,11 @@ public final class UserBitShared {
"T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT" +
"ER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_SCAN" +
"\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIVOT_" +
- "MAPS\020<*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016" +
- "\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014" +
- "SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.a" +
- "pache.drill.exec.protoB\rUserBitSharedH\001"
+ "MAPS\020<\022\024\n\020STATISTICS_MERGE\020=*g\n\nSaslStat" +
+ "us\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020" +
+ "SASL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013" +
+ "SASL_FAILED\020\004B.\n\033org.apache.drill.exec.p",
+ "rotoB\rUserBitSharedH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
index 051f82f2e..6b277b4ac 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
@@ -82,7 +82,8 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
ROWKEY_JOIN(57),
SYSLOG_SUB_SCAN(58),
STATISTICS_AGGREGATE(59),
- UNPIVOT_MAPS(60);
+ UNPIVOT_MAPS(60),
+ STATISTICS_MERGE(61);
public final int number;
@@ -161,6 +162,7 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
case 58: return SYSLOG_SUB_SCAN;
case 59: return STATISTICS_AGGREGATE;
case 60: return UNPIVOT_MAPS;
+ case 61: return STATISTICS_MERGE;
default: return null;
}
}
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index ca4e27385..30c612ca3 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -353,6 +353,7 @@ enum CoreOperatorType {
SYSLOG_SUB_SCAN = 58;
STATISTICS_AGGREGATE = 59;
UNPIVOT_MAPS = 60;
+ STATISTICS_MERGE = 61;
}
/* Registry that contains list of jars, each jar contains its name and list of function signatures.