This change adds a UDF "utc_timestamp" which returns the current
date and time in UTC. Example query:
select utc_timestamp();
+-------------------------------+
| utc_timestamp() |
+-------------------------------+
| 2017-06-15 17:36:39.290773000 |
+-------------------------------+
Change-Id: I969fc805922f2bb9c8101e84f85ff2cc3b1b6729
Reviewed-on: http://gerrit.cloudera.org:8080/7203
Tested-by: Impala Public Jenkins
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Instead of materializing empty rows when computing count star, we use
the data stored in the Parquet RowGroup.num_rows field. The Parquet
scanner tuple is modified to have one slot into which we will write the
num rows statistic. The aggregate function is changed from count to a
special sum function that gets initialized to 0. We also add a rewrite
rule so that count(<literal>) is rewritten to count(*) in order to make
sure that this optimization is applied in all cases.
Testing:
- Added functional and planner tests
Change-Id: I536b85c014821296aed68a0c68faadae96005e62
Reviewed-on: http://gerrit.cloudera.org:8080/6812
Reviewed-by: Taras Bobrovytsky <tbobrovytsky@cloudera.com>
Tested-by: Impala Public Jenkins
This is similar to the single-node execution optimisation, but applies
to slightly larger queries that should run in a distributed manner but
won't benefit from codegen.
This adds a new query option disable_codegen_rows_threshold that
defaults to 50,000. If fewer than this number of rows are processed
by a plan node per impalad, the cost of codegen almost certainly
outweighs the benefit.
Using rows processed as a threshold is justified by a simple
model that assumes the cost of codegen and execution per row for
the same operation are proportional. E.g. if x is the complexity
of the operation, n is the number of rows processed, C is a
constant factor giving the cost of codegen and Ec/Ei are constant
factor giving the cost of codegen'd and interpreted execution and
d, then the cost of the codegen'd operator is C * x + Ec * x * n
and the cost of the interpreted operator is Ei * x * n. Rearranging
means that interpretation is cheaper if n < C / (Ei - Ec), i.e. that
(at least with the simplified model) it makes sense to choose
interpretation or codegen based on a constant threshold. The
model also implies that it is somewhat safer to choose codegen
because the additional cost of codegen is O(1) but the additional
cost of interpretation is O(n).
I ran some experiments with TPC-H Q1, varying the input table size, to
determine what the cut-over point where codegen was beneficial was.
The cutover was around 150k rows per node for both text and parquet.
At 50k rows per node disabling codegen was very beneficial - around
0.12s versus 0.24s. To be somewhat conservative I set the default
threshold to 50k rows. On more complex queries, e.g. TPC-H Q10, the
cutover tends to be higher because there are plan nodes that process
many fewer than the max rows.
Fix a couple of minor issues in the frontend - the numNodes_
calculation could return 0 for Kudu, and the single node optimization
didn't handle the case where for a scan node with conjuncts, a limit
and missing stats correctly (it considered the estimate still valid.)
Testing:
Updated e2e tests that set disable_codegen to set
disable_codegen_rows_threshold to 0, so that those tests run both
with and without codegen still.
Added an e2e test to make sure that the optimisation is applied in
the backend.
Added planner tests for various cases where codegen should and shouldn't
be disabled.
Perf:
Added a targeted perf test for a join+agg over a small input, which
benefits from this change.
Change-Id: I273bcee58641f5b97de52c0b2caab043c914b32e
Reviewed-on: http://gerrit.cloudera.org:8080/7153
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
Kudu recently added the ability to alter a column's default value
and storage attributes (KUDU-861). This patch adds the ability to
modify these from Impala using ALTER.
It also supports altering a column's comment for non-Kudu tables.
It does not support setting a column to be a primary key or
changing a column's nullability, because those are not supported on
the Kudu side yet.
Syntax:
ALTER TABLE <table> ALTER [COLUMN] <column>
SET <attr> <value> [<attr> <value> [<attr> <value>...]]
where <attr> is one of:
- DEFAULT, BLOCK_SIZE, ENCODING, COMPRESSION (Kudu tables)
- COMMENT (non-Kudu tables)
ALTER TABLE <table> ALTER [COLUMN] <column> DROP DEFAULT
This is similar to the existing CHANGE statement:
ALTER TABLE <table> CHANGE <column> <new_col_name> <type>
[COMMENT <comment>]
but the new syntax is more natural for setting column properties
when the column name and type are not being changed. Both ALTER
COLUMN and CHANGE COLUMN operations use AlterTableAlterColStmt and
are sent to the catalog as ALTER_COLUMN operations.
Testing:
- Added FE tests to ParserTest and AnalyzeDDLTest
- Added EE tests to test_kudu.py
Change-Id: Id2e8bd65342b79644a0fdcd925e6f17797e89ad6
Reviewed-on: http://gerrit.cloudera.org:8080/6955
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Tested-by: Impala Public Jenkins
Remove from TCatalogUpdateResult the deprecated fields that are no longer
used and modify the catalog to always populate the
'updated_catalog_objects' and 'removed_catalog_object' fields with the
updated/removed catalog objects.
Testing: Run core tests.
Change-Id: Ibc80e43392cdc85a841e670030e0988692bdf884
Reviewed-on: http://gerrit.cloudera.org:8080/7248
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
This fixes some issues with Impala compilation on Hive 2:
1. It fixes TCLIService.thrift generation by changing
common/thrift/CMakeLists.txt to create the directory
before generating the file and adding a dependency
to generate TCLIService.thrift before compiling the
thrift files.
2. It modifies CatalogOpExecutor::applyAlterTable to
use the MetastoreShim.
With these changes, Impala builds successfully with
Hive 2 in my environment and on automated builds.
Change-Id: I7e1477993ee3ccddd236609efec7bb23f20a7b66
Reviewed-on: http://gerrit.cloudera.org:8080/7231
Tested-by: Impala Public Jenkins
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
This change separates Expr and ExprContext. This is a preparatory
step for factoring out static data (e.g. Exprs) of plan fragments
to be shared by multiple plan fragment instances.
This change includes the followings:
1. Include aggregate functions (AggFn) as Expr. This separates
AggFn from its evaluator. AggFn is similar to existing Expr
as both are represented as a tree of Expr nodes but it doesn't
really make sense to call Get*Val() on AggFn. This change
restructures the class hierarchy: much of the existing Expr
class is now renamed to ScalarExpr. Expr is the parent class
of both AggFn and ScalarExpr. Expr is defined to be a tree
with root of either AggFn or ScalarExpr and all descendants
being ScalarExpr.
2. ExprContext is renamed to ScalarExprEvaluator which is the
interface for evaluating ScalarExpr; AggFnEvaluator is the
interface for evaluating AggFn. Multiple evaluators can be
instantiated per Expr. Expr contains static states of an
expression while evaluator contains runtime states needed
for execution (i.e. evaluating the expression).
3. Update all exec nodes to instantiate Expr and their evaluators
separately. ExecNode::Init() will be responsible for creating
all the Exprs in an ExecNode while their evaluators are created
in ExecNode::Prepare(). Certain evaluators are also moved into
the data structures which actually utilize them. For instance,
HashTableCtx now owns the build and probe expression evaluators.
Similarly, TupleRowComparator and Sorter also own the evaluators.
ExecNode which utilizes these data structures are only responsible
for creating the expressions used by these data structures.
4. All codegen functions take Exprs instead of evaluators. Also, codegen
functions will not return error status should the IR function fails the
LLVM verification step.
5. The assignment of index into the FunctionContext vector is now done
during the construction of ScalarExpr. Evaluators are only responsible
for allocating and initializing the FunctionContexts.
6. Open(), Prepare() are now removed from Expr classes. The interface
for creating any Expr is via either ScalarExpr::Create() or AggFn::Create()
which will convert a thrift Expr into an initialized Expr object.
Similarly, Create() interface is used for creating evaluators from an
intialized Expr object.
This separation allows the future change to introduce PlanNode data structures.
The plan is to move all ExecNode::Init() logic to PlanNode and call them once
per plan fragment.
Change-Id: Iefdc9aeeba033355cb9497e3a5d2363627dcf2f3
Reviewed-on: http://gerrit.cloudera.org:8080/5483
Reviewed-by: Michael Ho <kwho@cloudera.com>
Tested-by: Impala Public Jenkins
The stream defaults to pages of default_page_len_. If a row doesn't
fit in that page, it will allocate another page up to max_page_len_
bytes and append a single row to that page, then immediately unpin
the page. This means that when writing a stream, the large
page only needs to be kept in memory temporarily, which helps with
memory requirements. E.g. consider a hash join that is repartitioning
1 unpinned stream into 16 unpinned streams. We will need
default_page_len_ * 15 + max_page_len_ * 2 bytes of reservation because
when processing a large row we only need one large write buffer at a
time.
Also switches the stream to lazily allocating write pages, so that
we don't need to allocate a page until we know the size of the row
to go in it. This required a mechanism to "save" reservation in
PrepareForRead()/PrepareForWrite(). A SubReservation APi is added
to BufferPool for this purpose and the stream now saves read and
write reservation for lazy page allocation. It also saves reservation
instead of double-pinning pages in the read/write case.
The large row cases are not as optimised for memory consumption or
performance - queries processing very large numbers of large rows
are an extreme edge case that is likely to hit other performance
bottlenecks first. Pages with large rows can have up to 50%
internal fragmentation.
To avoid duplicating more logic between AddRow() and AllocateRow()
I restructured things so that AddRowSlow() is implemented in terms
of AllocateRowSlow(). AllocateRow() now takes a function as an
argument to populate the row.
Testing:
* Added tests for the case where 0 rows are added to the stream
* Extend BigRow to exercise the new code.
* Also test large strings and read/write streams.
Change-Id: I2861c58efa7bc1aeaa5b7e2f043c97cb3985c8f5
Reviewed-on: http://gerrit.cloudera.org:8080/6638
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
Impala is case insensitive for column names and generally deals
with them in all lower case. Kudu is case sensitive. This can
lead to a problems when a table is created externally in Kudu
with a column name with upper case letters.
This patch solves the problem by having KuduColumn always store
its name in lower case, so that general Impala code that has been
written expecting lower cased column names can use Column.getName()
safely.
It also adds the method KuduColumn.getKuduName(), which returns
the column name in the case that it appears in Kudu. Any code that
passes column names into the Kudu API must call this method first
to get the correct column name.
There are four specific situations fixed by this patch:
- When ordering on a Kudu column, the Analyzer would create
two SlotDescriptors that point to the same column because
registerSlotRef() was being called with inconsistent casing.
It is now always called with the lower cased names.
- 'ADD RANGE PARTITION' would fail to find the range partition
column if it isn't all lower case in Kudu.
- 'ALTER TABLE DROP COLUMN' and 'ALTER TABLE CHANGE' only worked
if the column name was specified in Kudu case.
- 'CREATE EXTERNAL TABLE' called on a Kudu table with column names
that differ only in case now returns an error, since Impala has
no way of handling this situation.
Testing:
- Added e2e tests in test_kudu.py.
- Manually edited functional_kudu to change column names to have
mixed casing and ran the kudu tests.
Change-Id: I14aba88510012174716691b9946e1c7d54d01b44
Reviewed-on: http://gerrit.cloudera.org:8080/6902
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Tested-by: Impala Public Jenkins
Clean pages and free buffers appear as untracked memory in the
MemTracker hierarchy. This was misleading since the memory is tracked
and present in the BufferPool. This change adds two MemTrackers below
the process level that accounts for this memory.
Updating global counters would be very inefficient and negate most of
the effort put into making the buffer allocator scalable. Instead the
values of the metrics are computed on demand by summing values across
all of the arena in BufferAlloctor.
The numbers reported are approximate because we do not lock any of the
BufferAllocator state and therefore don't get a consistent view of the
entire BufferAllocator at any moment in time. However they are accurate
enough to understand the general state of the system.
Also switches over ASAN to use a metric, similar to the regular TCMalloc
build so that the behaviour under ASAN diverges less.
Testing:
Add some checks to unit tests to sanity-check that the numbers computed
are valid.
Manually tested by rebasing my buffer pool dev branch onto this change
and running some spilling queries. The /memz page reported:
Process: Limit=8.35 GB Total=1005.49 MB Peak=1.01 GB
Buffer Pool: Free Buffers: Total=391.50 MB
Buffer Pool: Clean Pages: Total=112.00 MB
Free Disk IO Buffers: Total=247.00 KB Peak=30.23 MB
RequestPool=fe-eval-exprs: Total=0 Peak=4.00 KB
RequestPool=default-pool: Total=374.30 MB Peak=416.55 MB
Query(b9421063d13af70b:ddb9973900000000): Reservation=0 ReservationLimit=6.68 GB OtherMemory=801.09 KB Total=801.09 KB Peak=1.05 MB
<< snip >>
Untracked Memory: Total=127.45 MB
Manually tested the ASAN change by building under ASAN, running some
queries, and inspecting the memz/ page. It reported a value of 100-200MB
untracked memory, similar to the non-ASAN build.
Change-Id: I007eb258377b33fff9f3246580d80fa551837078
Reviewed-on: http://gerrit.cloudera.org:8080/6993
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
This change adds last_day() function.
The function takes exactly one TIMESTAMP argument
and returns a TIMESTAMP that is the last date of the
input date's calendar month.
The function will return NULL when:
1) The input argument cannot be implicitly casted to
a TIMESTAMP.
2) The TIMESTAMP argument is missing a date component.
3) The TIMESTAMP argument is outside of the supported range:
between 1400-01-31 00:00:00 and 9999-12-31 23:59:59
Change-Id: I429c8734bddca3c37a2eedc211a16a4ffcb04370
Reviewed-on: http://gerrit.cloudera.org:8080/6991
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: Impala Public Jenkins
Bug:
When Sentry-based authorization is enabled, a user that isn't authorized
to EXPLAIN a statement that uses a view can still access unauthorized
information, such as view's definition, by running the statement and
asking for the query profile or the execution summary.
Fix:
During query compilation, determine if the user can access the the runtime
profile or the execution summary. Upon request for a runtime profile or
execution summary from a user, determine based on that information and
the user that is asking for the profile if the runtime profile
(or execution summary) will be returned or an authorization error.
The authorization rule enforced is the following:
- User A runs statement S, A asks for profile, A has profile access:
Runtime profile is returned
- User A runs statement S, A asks for profile, A doesn't have profile access:
Authorization error
- User A runs statement S, user B asks for profile:
Authorization error.
This patch doesn't enforce access to the runtime profile or execution summary
through the Web UI.
Change-Id: I2255d587367c2d328590ae8534a5406c4b0c9b15
Reviewed-on: http://gerrit.cloudera.org:8080/7064
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
Adds a new query option DEFAULT_JOIN_DISTRIBUTION_MODE to
control which join distribution mode is chosen when the join
inputs have an unknown cardinality (e.g., missing stats) or when
the expected costs of the different strategies are equal.
Values for DEFAULT_JOIN_DISTRIBUTION_MODE: [BROADCAST, SHUFFLE]
Default: BROADCAST
Note that this change effectively undoes IMPALA-5120.
Testing:
- Added new planner tests
- Core/hdfs run passed
Change-Id: Ibd34442f422129d53bef5493fc9cbe7375a0765c
Reviewed-on: http://gerrit.cloudera.org:8080/7059
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Impala Public Jenkins
While support for TIMESTAMP columns in Kudu tables has been
committed (IMPALA-5137), it does not support TIMESTAMP
column default values.
This supports CREATE TABLE syntax to specify the default
values, but more importantly this fixes the loading of Kudu
tables that may have had default values set on
UNIXTIME_MICROS columns, e.g. if the table was created via
the python client. This involves fixing KuduColumn to hide
the LiteralExpr representing the default value because it
will be a BIGINT if the column type is TIMESTAMP. It is only
needed to call toSql() and toStringValue(), so helper
functions are added to KuduColumn to encapsulate special
logic for TIMESTAMP.
TODO: Add support and tests for ALTER setting the default
value (when IMPALA-4622 is committed).
Change-Id: I655910fb4805bb204a999627fa9f68e43ea8aaf2
Reviewed-on: http://gerrit.cloudera.org:8080/6936
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: Impala Public Jenkins
This change adds a query option to disable reading Parquet statistics.
It provides a workaround when dealing with files that have corrupt
parquet statistics.
Note that Impala handles Parquet files affected by PARQUET-251 correctly
by ignoring statistics for anything but plain numeric types. This query
option is supposed to help with files affected by unknown or errors or
by errors that are yet to be made.
Change-Id: I427f7fde40d0f4b703751e40f3c2109a850643f7
Reviewed-on: http://gerrit.cloudera.org:8080/7001
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins
The main idea of this patch is to use table stats to
extrapolate the row counts for new/modified partitions.
Existing behavior:
- Partitions that lack the row count stat are ignored
when estimating the cardinality of HDFS scans. Such
partitions effectively have an estimated row count
of zero.
- We always use the row count stats for partitions that
have one. The row count may be innaccurate if data in
such partitions has changed significantly.
Summary of changes:
- Enhance COMPUTE STATS to also store the total number
of file bytes in the table.
- Use the table-level row count and file bytes stats
to estimate the number of rows in a scan.
- A new impalad startup flag is added to enable/disable
the extrapolation behavior. The feature is disabled by
default. Note that even with the feature disabled,
COMPUTE STATS stores the file bytes so you can enable
the feature without having to run COMPUTE STATS again.
Testing:
- Added new FE unit test
- Added new EE test
Change-Id: I972c8a03ed70211734631a7dc9085cb33622ebc4
Reviewed-on: http://gerrit.cloudera.org:8080/6840
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Impala Public Jenkins
Before this patch, Impala relied on INVALIDATE METADATA to load
externally added UDFs from HMS. The problem with this approach is that
INVALIDATE METADATA affects all databases and tables in the entire
cluster.
In this patch, we add a REFRESH FUNCTIONS <db> statement that reloads
the functions of a database from HMS. We return a list of updated and
removed db functions to the issuing Impalad in order to update its
local catalog cache.
Testing:
- Ran a private build which passed.
Change-Id: I3625c88bb51cca833f3293c224d3f0feb00e6e0b
Reviewed-on: http://gerrit.cloudera.org:8080/6878
Reviewed-by: Taras Bobrovytsky <tbobrovytsky@cloudera.com>
Tested-by: Impala Public Jenkins
Misc changes to improve usability of the profiles.
* Separate out detailed BufferPool metrics into a "Buffer pool"
sub-profile.
* Only create the limit counter if there is a limit
* Show BufferPool using in query MemTracker (it was accidentally
disabled before because there was no query-level profile).
* Reduce clutter in MemTracker dump by only showing buffer pool
reservation, not usage (the usage was misleading anyway because
it didn't include child usage).
* Remove TotalUnpinnedBytes, which had limited value - WriteIoBytes
and PeakUnpinnedBytes can answer most of the same questions - i.e.
did it unpin any pages, and how many did it need to write to disk.
* Add buffer pool metrics to /memz (if buffer pool is enabled) and
reorder /memz so more useful information is up the top.
Change-Id: I34b7f4d94c3d396ac89026c7559d6b2c6e02697c
Reviewed-on: http://gerrit.cloudera.org:8080/6690
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
The sortby() hint is superseded by the SORT BY SQL clause, which has
been introduced in IMPALA-4166. This changes removes the hint.
Change-Id: I83e1cd6fa7039035973676322deefbce00d3f594
Reviewed-on: http://gerrit.cloudera.org:8080/6885
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Impala Public Jenkins
This change builds on the support for reading and writing
TIMESTAMP columns to Kudu tables (see [1]), adding support
for pushing TIMESTAMP predicates to Kudu for scans.
Binary predicates and IN list predicates are supported.
Testing: Added some planner and EE tests to validate the
behavior.
1: https://gerrit.cloudera.org/#/c/6526/
Change-Id: I08b6c8354a408e7beb94c1a135c23722977246ea
Reviewed-on: http://gerrit.cloudera.org:8080/6789
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: Impala Public Jenkins
The following Hadoop metrics have been added to the /metrics page:
hedgedReadOps - the number of hedged reads that have occurred
hedgedReadOpsWin - the number of times the hedged read returned
faster than the original read
The metrics will be updated only when --use_hdfs_pread is set to
'true'.
This change depends on the following new commit to HDFS:
8c81a16a1f
Testing: Not adding tests since it requires some custom hadoop
configuration. Tested manually by setting the configurations and
verifying that the metrics work.
Change-Id: Id4a5d396abb3373d352ad2df8c2272db018114da
Reviewed-on: http://gerrit.cloudera.org:8080/6886
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Impala Public Jenkins
This change adds support for adding SORT BY (...) clauses to CREATE
TABLE and ALTER TABLE statements. Examples are:
CREATE TABLE t (i INT, j INT, k INT) PARTITIONED BY (l INT) SORT BY (i, j);
CREATE TABLE t SORT BY (int_col,id) LIKE u;
CREATE TABLE t LIKE PARQUET '/foo' SORT BY (id,zip);
ALTER TABLE t SORT BY (int_col,id);
ALTER TABLE t SORT BY ();
Sort columns can only be specified for Hdfs tables and effectiveness may
vary based on storage type; for example TEXT tables will not see
improved compression. The SORT BY clause must not contain clustering
columns. The columns in the SORT BY clause are stored in the
'sort.columns' table property and will result in an additional SORT node
being added to the plan before the final table sink. Specifying sort
columns also enables clustering during inserts, so the SORT node will
contain all partitioning columns first, followed by the sort columns. We
do this because sort columns add a SORT node to the plan and adding the
clustering columns to the SORT node is cheap.
Sort columns supersede the sortby() hint, which we will remove in a
subsequent change (IMPALA-5144). Until then, it is possible to specify
sort columns using both ways at the same time and the column lists
will be concatenated.
Change-Id: I08834f38a941786ab45a4381c2732d929a934f75
Reviewed-on: http://gerrit.cloudera.org:8080/6495
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Impala Public Jenkins
Adds Impala support for TIMESTAMP types stored in Kudu.
Impala stores TIMESTAMP values in 96-bits and has nanosecond
precision. Kudu's timestamp is a 64-bit microsecond delta
from the Unix epoch (called UNIXTIME_MICROS), so a conversion
is necessary.
When writing to Kudu, TIMESTAMP values in nanoseconds are
averaged to the nearest microsecond.
When reading from Kudu, the KuduScanner returns
UNIXTIME_MICROS with 8bytes of padding so Impala can convert
the value to a TimestampValue in-line and copy the entire
row.
Testing:
Updated the functional_kudu schema to use TIMESTAMPs instead
of converting to STRING, so this provides some decent
coverage. Some BE tests were added, and some EE tests as
well.
TODO: Support pushing down TIMESTAMP predicates
TODO: Support TIMESTAMPs in range partitioning expressions
Change-Id: Iae6ccfffb79118a9036fb2227dba3a55356c896d
Reviewed-on: http://gerrit.cloudera.org:8080/6526
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: Impala Public Jenkins
This commit improves the memory requirements for storing file and block
metadata in the catalog and the impalad nodes by using the FlatBuffers
serialization library.
Testing:
Passed an exhaustive tests run.
Benchmark:
Memory requirement for storing an HDFS table with 250K files is reduced
by 2.5X.
Change-Id: I483d3cadc9d459f71a310c35a130d073597b0983
Reviewed-on: http://gerrit.cloudera.org:8080/6406
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
This change adds functionality to write and read parquet::Statistics for
Decimal, String, and Timestamp values. As an exception, we don't read
statistics for CHAR columns, since CHAR support is broken in Impala
(IMPALA-1652).
This change also switches from using the deprecated fields 'min' and
'max' to populate the new fields 'min_value' and 'max_value' in
parquet::Statistics, that were added in parquet-format pull request #46.
The HdfsParquetScanner will preferably read the new fields if they are
populated and if the column order 'TypeDefinedOrder' has been used to
compute the statistics. For columns without a column order set or with
only the deprecated fields populated, the scanner will read them only if
they are of simple numeric type, i.e. boolean, integer, or floating
point.
This change removes the validation of the Parquet Statistics we write to
Hive from the tests, since Hive does not write the new fields. Instead
it adds a parquet file written by Hive that uses the deprecated fields
for its statistics. It uses that file to exercise the fallback logic for
supported types in a test.
This change also cleans up the interface of ParquetPlainEncoder in
parquet-common.h.
Change-Id: I3ef4a5d25a57c82577fd498d6d1c4297ecf39312
Reviewed-on: http://gerrit.cloudera.org:8080/6563
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Lars Volker <lv@cloudera.com>
The Parquet file column reader may fail in the middle
of producing a scratch tuple batch for various reasons
such as exceeding memory limit or cancellation. In which
case, the scratch tuple batch may not have materialized
all the rows in a row group. We shouldn't erroneously
report that the file is corrupted in this case as the
column reader didn't completely read the entire row group.
A new test case is added to verify that we won't see this
error message. A new failpoint phase GETNEXT_SCANNER is
also added to differentiate it from the GETNEXT in the
scan node itself.
Change-Id: I9138039ec60fbe9deff250b8772036e40e42e1f6
Reviewed-on: http://gerrit.cloudera.org:8080/6787
Reviewed-by: Michael Ho <kwho@cloudera.com>
Tested-by: Impala Public Jenkins
Coordinator:
- FragmentInstanceState -> BackendState, which in turn records
FragmentInstanceStats
QueryState
- does query-wide setup in a separate thread (which also launches
the instance exec threads)
- has a query-wide 'prepared' state at which point all static setup
is done and all FragmentInstanceStates are accessible
Also renamed QueryExecState to ClientRequestState.
Simplified handling of execution status (in FragmentInstanceState):
- status only transmitted via ReportExecStatus rpc
- in particular, it's not returned anymore from the Cancel rpc
FIS: Fixed bugs related to partially-prepared state (in Close() and ReleaseThreadToken())
Change-Id: I20769e420711737b6b385c744cef4851cee3facd
Reviewed-on: http://gerrit.cloudera.org:8080/6535
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Impala Public Jenkins
This adds a compatibility shim layer with Hive 1 and Hive 2
implementations. The version-specific code lives in
fe/src/compat-hive-$IMPALA_HIVE_MAJOR_VERSION and
common/thrift/hive-$IMPALA_HIVE_MAJOR_VERSION-api/
The shim adds wrapper methods to handle differing method signatures
and and config variables that changed slightly.
Some thrift classes were also moved from the 'cli' to 'rpc' package.
We work around these by implementing subclasses with the same name
in a different package for compatibility or by implementing shim
methods that operate on the classes. We also need to change the
package in the TCLIService.thrift, which is done with a
search-and-replace.
Also avoid the sticky config variable problem with some of the source
paths by requiring an _OVERRIDE suffix on the variable to override it
from the environment.
Testing:
Made sure that I could build Impala on master as normal, and also
with the following config overrides in bin/impala-config-local.sh:
export IMPALA_HADOOP_VERSION=3.0.0-alpha1-cdh6.x-SNAPSHOT
export IMPALA_HBASE_VERSION=2.0.0-cdh6.x-SNAPSHOT
export IMPALA_HIVE_VERSION=2.1.0-cdh6.x-SNAPSHOT
export IMPALA_SENTRY_VERSION=1.5.1-cdh6.x-SNAPSHOT
export IMPALA_PARQUET_VERSION=1.5.0-cdh6.x-SNAPSHOT
I manually assembled the dependencies by copying the following files
from the Hive 2 source and from a Hadoop 3 build:
$CDH_COMPONENTS_HOME/hive-2.1.0-cdh6.x-SNAPSHOT/src/metastore/if/hive_metastore.thrift
$CDH_COMPONENTS_HOME/hadoop-3.0.0-alpha1-cdh6.x-SNAPSHOT/lib/native/*
$CDH_COMPONENTS_HOME/hadoop-3.0.0-alpha1-cdh6.x-SNAPSHOT/include/hdfs.h
Change-Id: Ifbc265281c04fe3136bc3c920dbac966742ce09a
Reviewed-on: http://gerrit.cloudera.org:8080/5538
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
Before this change:
Hive adjusts timestamps by subtracting the local time zone's offset
from all values when writing data to Parquet files. Hive is internally
inconsistent because it behaves differently for other file formats. As
a result of this adjustment, Impala may read "incorrect" timestamp
values from Parquet files written by Hive.
After this change:
Impala reads Parquet MR timestamp data and adjusts values using a time
zone from a table property (parquet.mr.int96.write.zone), if set, and
will not adjust it if the property is absent. No adjustment will be
applied to data written by Impala.
New HDFS tables created by Impala using CREATE TABLE and CREATE TABLE
LIKE <file> will set the table property to UTC if the global flag
--set_parquet_mr_int96_write_zone_to_utc_on_new_tables is set to true.
HDFS tables created by Impala using CREATE TABLE LIKE <other table>
will copy the property of the table that is copied.
This change also affects the way Impala deals with
--convert_legacy_hive_parquet_utc_timestamps global flag (introduced
in IMPALA-1658). The flag will be taken into account only if
parquet.mr.int96.write.zone table property is not set and ignored
otherwise.
Change-Id: I3f24525ef45a2814f476bdee76655b30081079d6
Reviewed-on: http://gerrit.cloudera.org:8080/5939
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins
Bulk DMLs (INSERT, UPSERT, UPDATE, and DELETE) for Kudu
are currently painful because we just send rows randomly,
which creates a lot of work for Kudu since it partitions
and sorts data before writing, causing writes to be slow
and leading to timeouts.
We can alleviate this by sending the rows to Kudu already
partitioned and sorted. This patch partitions and sorts
rows according to Kudu's partitioning scheme for INSERTs
and UPSERTs. A followup patch will handle UPDATE and DELETE.
It accomplishes this by inserting an exchange node and a sort
node into the plan before the operation. Both the exchange and
the sort are given a KuduPartitionExpr which takes a row and
calls into the Kudu client to return its partition number.
It also disallows INSERT hints for Kudu tables, since the
hints that we support (SHUFFLE, CLUSTER, SORTBY), so longer
make sense.
Testing:
- Updated planner tests.
- Ran the Kudu functional tests.
- Ran performance tests demonstrating that we can now handle much
larger inserts without having timeouts.
Change-Id: I84ce0032a1b10958fdf31faef225372c5c38fdc4
Reviewed-on: http://gerrit.cloudera.org:8080/6559
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Tested-by: Impala Public Jenkins
This commit introduces a new startup option, termed 'is_executor',
that determines whether an impalad process can execute query fragments.
The 'is_executor' option determines if a specific host will be included
in the scheduler's backend configuration and hence included in
scheduling decisions.
Testing:
- Added a customer cluster test.
- Added a new scheduler test.
Change-Id: I5d2ff7f341c9d2b0649e4d14561077e166ad7c4d
Reviewed-on: http://gerrit.cloudera.org:8080/6628
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
Compute the minimum buffer requirement for spilling nodes and
per-host estimates for the entire plan tree.
This builds on top of the existing resource estimation code, which
computes the sets of plan nodes that can execute concurrently. This is
cleaned up so that the process of producing resource requirements is
clearer. It also removes the unused VCore estimates.
Fixes various bugs and other issues:
* computeCosts() was not called for unpartitioned fragments, so
the per-operator memory estimate was not visible.
* Nested loop join was not treated as a blocking join.
* The TODO comment about union was misleading
* Fix the computation for mt_dop > 1 by distinguishing per-instance and
per-host estimates.
* Always generate an estimate instead of unpredictably returning
-1/"unavailable" in many circumstances - there was little rhyme or
reason to when this happened.
* Remove the special "trivial plan" estimates. With the rest of the
cleanup we generate estimates <= 10MB for those trivial plans through
the normal code path.
I left one bug (IMPALA-4862) unfixed because it is subtle, will affect
estimates for many plans and will be easier to review once we have the
test infra in place.
Testing:
Added basic planner tests for resource requirements in both the MT and
non-MT cases.
Re-enabled the explain_level tests, which appears to be the only
coverage for many of these estimates. Removed the complex and
brittle test cases and replaced with a couple of much simpler
end-to-end tests.
Change-Id: I1e358182bcf2bc5fe5c73883eb97878735b12d37
Reviewed-on: http://gerrit.cloudera.org:8080/5847
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
Support allocating with mmap instead of TCMalloc to give more control
over memory usage. Also tell Linux to back larger buffers with huge
pages when possible to reduce TLB pressure. The main complication is
that memory returned by mmap() is not necessarily aligned to a huge
page boundary, so we need to "fix up" the mapping ourselves.
Adds additional memory metrics, since we previously relied on the
assumption that all memory was allocated through TCMalloc.
memory.total-used tracks the total across the buffer pool and
TCMalloc. When the buffer pool is not present, they just report
the TCMalloc values.
This can be enabled with the --mmap_buffers flag. The transparent
huge pages support can be disabled with the --madvise_huge_pages
startup flag.
At some point this should become the default, but it requires
more work to validate perf and resource used (virtual address
space, etc).
Testing:
Added some unit tests to test edge cases and the different supported
flags. Many pre-existing tests also exercise the modified code.
Change-Id: Ifbc748f74adcbbdcfa45f3ec7df98284925acbd6
Reviewed-on: http://gerrit.cloudera.org:8080/6474
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
Adds tests for read errors from permissions (i.e. open() fails),
corrupt data (integrity check fails) and truncated files (read() fails).
Fixes a couple of bugs:
* Truncated reads were not detected in TmpFilemgr
* IoMgr buffers weren't returned on error paths (this isn't a true leak
but results in DCHECKs being hit).
Change-Id: I3f2b93588dd47f70a4863ecad3b5556c3634ccb4
Reviewed-on: http://gerrit.cloudera.org:8080/6562
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins
The union node acts as pass through operator and forwards row batches
from it's children without materializing. This is done in the case
when the child's tuple layout is identical to union node tuple layout
and no functions need to be applied to the child row batches.
Removed operand reordering in the FE because it's simpler and safer to
handle all passthrough children before non-passthrough children in the
BE. The recent improvements to memory management allowed us to remove
this requirement.
Testing:
- Added new planner and end to end tests that cover the new
functionality.
- Updated existing tests to reflect the new behavior.
Perf:
Ran a benchmark on a local 10 GB tpcds dataset. I used an unpartitioned
version of the store_sales table. There was over a 2x performance
improvement for the following query:
SELECT
COUNT(ss_sold_time_sk),
COUNT(ss_item_sk),
COUNT(ss_customer_sk),
COUNT(ss_cdemo_sk),
COUNT(ss_hdemo_sk),
COUNT(ss_addr_sk),
COUNT(ss_store_sk),
COUNT(ss_promo_sk),
COUNT(ss_ticket_number),
COUNT(ss_quantity),
COUNT(ss_wholesale_cost),
COUNT(ss_list_price),
COUNT(ss_sales_price),
COUNT(ss_ext_discount_amt),
COUNT(ss_ext_sales_price),
COUNT(ss_ext_wholesale_cost),
COUNT(ss_ext_list_price),
COUNT(ss_ext_tax),
COUNT(ss_coupon_amt),
COUNT(ss_net_paid),
COUNT(ss_net_paid_inc_tax),
COUNT(ss_net_profit),
COUNT(ss_sold_date_sk)
FROM (
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
union all
select * from tpcds_10_parquet.store_sales_unpartitioned
) t
Before:
Total Time: 43s164ms
Summary:
Operator #Hosts Avg Time Max Time #Rows Est. #Rows Peak Mem Est. Peak Mem Detail
------------------------------------------------------------------------------------------------------------------------------
13:AGGREGATE 1 224.721us 224.721us 1 1 28.00 KB -1.00 B FINALIZE
12:EXCHANGE 1 24.578us 24.578us 3 1 0 -1.00 B UNPARTITIONED
11:AGGREGATE 3 2s402ms 3s060ms 3 1 119.00 KB 10.00 MB
00:UNION 3 35s380ms 37s846ms 288.01M 288.01M 3.08 MB 0
|--02:SCAN HDFS 3 184.197ms 219.931ms 28.80M 28.80M 535.03 MB 1.88 GB store_sales_unpartitioned
|--03:SCAN HDFS 3 131.956ms 153.401ms 28.80M 28.80M 534.98 MB 1.88 GB store_sales_unpartitioned
|--04:SCAN HDFS 3 178.456ms 247.721ms 28.80M 28.80M 534.98 MB 1.88 GB store_sales_unpartitioned
|--05:SCAN HDFS 3 189.398ms 242.251ms 28.80M 28.80M 535.01 MB 1.88 GB store_sales_unpartitioned
|--06:SCAN HDFS 3 122.786ms 156.528ms 28.80M 28.80M 534.98 MB 1.88 GB store_sales_unpartitioned
|--07:SCAN HDFS 3 147.467ms 183.391ms 28.80M 28.80M 535.13 MB 1.88 GB store_sales_unpartitioned
|--08:SCAN HDFS 3 147.502ms 186.273ms 28.80M 28.80M 535.01 MB 1.88 GB store_sales_unpartitioned
|--09:SCAN HDFS 3 130.086ms 154.682ms 28.80M 28.80M 535.04 MB 1.88 GB store_sales_unpartitioned
|--10:SCAN HDFS 3 122.701ms 161.056ms 28.80M 28.80M 534.89 MB 1.88 GB store_sales_unpartitioned
01:SCAN HDFS 3 287.863ms 330.436ms 28.80M 28.80M 534.98 MB 1.88 GB store_sales_unpartitioned
After:
Total Time: 19s139ms
Summary:
Operator #Hosts Avg Time Max Time #Rows Est. #Rows Peak Mem Est. Peak Mem Detail
------------------------------------------------------------------------------------------------------------------------------
13:AGGREGATE 1 166.241us 166.241us 1 1 28.00 KB -1.00 B FINALIZE
12:EXCHANGE 1 71.695us 71.695us 3 1 0 -1.00 B UNPARTITIONED
11:AGGREGATE 3 2s971ms 3s809ms 3 1 3.08 MB 10.00 MB
00:UNION 3 207.956ms 222.846ms 288.01M 288.01M 0 0
|--02:SCAN HDFS 3 1s533ms 1s535ms 28.80M 28.80M 532.28 MB 1.88 GB store_sales_unpartitioned
|--03:SCAN HDFS 3 1s554ms 1s669ms 28.80M 28.80M 525.73 MB 1.88 GB store_sales_unpartitioned
|--04:SCAN HDFS 3 1s568ms 1s716ms 28.80M 28.80M 525.03 MB 1.88 GB store_sales_unpartitioned
|--05:SCAN HDFS 3 1s503ms 1s617ms 28.80M 28.80M 527.43 MB 1.88 GB store_sales_unpartitioned
|--06:SCAN HDFS 3 1s560ms 1s634ms 28.80M 28.80M 528.52 MB 1.88 GB store_sales_unpartitioned
|--07:SCAN HDFS 3 1s489ms 1s643ms 28.80M 28.80M 534.81 MB 1.88 GB store_sales_unpartitioned
|--08:SCAN HDFS 3 1s534ms 1s581ms 28.80M 28.80M 528.10 MB 1.88 GB store_sales_unpartitioned
|--09:SCAN HDFS 3 1s558ms 1s674ms 28.80M 28.80M 526.77 MB 1.88 GB store_sales_unpartitioned
|--10:SCAN HDFS 3 1s504ms 1s692ms 28.80M 28.80M 527.83 MB 1.88 GB store_sales_unpartitioned
01:SCAN HDFS 3 1s682ms 1s911ms 28.80M 28.80M 526.14 MB 1.88 GB store_sales_unpartitioned
Change-Id: Ia8f6d5062724ba5b78174c3227a7a796d10d8416
Reviewed-on: http://gerrit.cloudera.org:8080/5816
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins
This introduces KuduScanNodeMt, the single-threaded version
of KuduScanNode that materializes the tuples in GetNext().
KuduScanNodeMt is enabled by the same condition as
HdfsScanNodeMt: mt_dop is greater than or equal to 1.
To share code between the two implementations, KuduScanNode
and KuduScanNodeMt are now subclasses of KuduScanNodeBase,
which implements the shared code. The KuduScanner is
minimally impacted, as it already had the required GetNext
interface.
Since the KuduClient is a heavy-weight object, it is now
shared at the QueryState level. We try to share the
KuduClient as much as possible, but there are times when
the KuduClient cannot be shared. Each Kudu table has
master addresses stored in the Hive Metastore. We only
share KuduClients for tables that have an identical value
for the master addresses. In the ideal case, every Kudu
table will have the same value, but there is no explicit
guarantee of this.
The testing for this is a modified version of
kudu-scan-node.test run with various mt_dop values.
Change-Id: I6e4593300e376bc508b78acaea64ffdd2c73a67a
Reviewed-on: http://gerrit.cloudera.org:8080/6312
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Impala Public Jenkins
Summary of changes:
Introduces a new query option PARQUET_ARRAY_RESOLUTION to
control the path-resolution behavior for Parquet files
with nested arrays. The values are:
- THREE_LEVEL
Assumes arrays are encoded with the 3-level representation.
Also resolves arrays encoded with a single level.
Does not attempt a 2-level resolution.
- TWO_LEVEL
Assumes arrays are encoded with the 2-level representation.
Also resolves arrays encoded with a single level.
Does not attempt a 3-level resolution.
- TWO_LEVEL_THEN_THREE_LEVEL
First tries to resolve assuming the 2-level representation,
and if unsuccessful, tries the 3-level representation.
Also resolves arrays encoded with a single level.
This is the current Impala behavior and is used as the
default value for compatibility.
Note that 'failure' to resolve a schema path with a given
array-resolution policy does not necessarily mean a warning or
error is returned by the query. A mismatch might be treated
like a missing field which is necessary to support schema
evolution. There is no way to reliably distinguish the
'bad resolution' and 'legitimately missing field' cases.
The new query option is independent of and can be combined
with the existing PARQUET_FALLBACK_SCHEMA_RESOLUTION.
Background:
Arrays can be represented in several ways in Parquet:
- Three Level Encoding (standard)
- Two Level Encoding (legacy)
- One Level Encoding (legacy)
More details are in the "Lists" section of the spec:
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
Unfortunately, there is no reliable metadata within Parquet files
to indicate which encoding was used. There is even the possibility
of having mixed encodings within the same file if there are multiple
arrays.
As a result, Impala currently tries to auto-detect the file encoding
when resolving a schema path in a Parquet file using the
TWO_LEVEL_THEN_THREE_LEVEL policy.
However, regardless of whether a Parquet data file uses the 2-level
or 3-level encoding, the index-based resolution may return incorrect
results if the representation in the Parquet file does not
exactly match the attempted array-resoution policy. Intuitively,
when attempting a 2-level resolution on a 3-level file, the matched
schema node may not be deep enough in the schema tree, but could still
be a scalar node with expected type. Similarly, when attempting a
3-level resolution on a 2-level file a level may be incorrectly
skipped.
The name-based policy generally does not have this problem because it
avoids traversing incorrect schema paths. However, the index-based
resoution allows a different set of schema-evolution operations,
so just using name-based resolution is not an acceptable workaround
in all cases.
Testing:
- Added new Parquet data files that show how incorrect results
can be returned with a mismatched file encoding and resolution
policy. Added both 2-level and 3-level versions of the data.
- Added a new test in test_nested_types.py that shows the behavior
with the new PARQUET_ARRAY_RESOLUTION query option.
- Locally ran test_scanners.py and test_nested_types.py on core.
Change-Id: I4f32e19ec542d4d485154c9d65d0f5e3f9f0a907
Reviewed-on: http://gerrit.cloudera.org:8080/6250
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Impala Public Jenkins
This commit exposes information about JVM threads to the impalad and
catalogd web UIs. This information includes statistics about the number
of threads running in the JVM as well as per-thread stacktraces, monitors and
synchronizers. Total CPU, user CPU and blocked time is also reported per
thread.
Change-Id: Id497043ab33dcf107a562f0b1ccd5c46095d397f
Reviewed-on: http://gerrit.cloudera.org:8080/6013
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
This changes the HdfsParquetTableWriter to populate the
parquet::RowGroup::sorting_columns list with all columns mentioned in a
'sortby()' hint within INSERT statements. The columns are added to the
list in the order in which they appear inside the hint.
The change also adds backports.tempfile to the python requirements to
provide 'tempfile.TemporaryDirectory' on python 2.7.
The change also changes the default ordering for columns mentioned in
'sortby()' hints from descending to ascending.
To test this change, we write a table with a 'sortby()' hint and verify,
that the sorting_columns get populated correctly.
Change-Id: Ib42aab585e9e627796e9510e783652d49d74b56c
Reviewed-on: http://gerrit.cloudera.org:8080/6219
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Impala Public Jenkins
Very often we have to change the logging levels
of Impalads and Catalog server for debugging purposes.
Currently, there is no way to do that without a restart
of the daemons, which is not a viable option in production
deployments.
This patch addresses this supportability gap by exposing
the ability to set dynamic logging levels using a simple
web endpoint on Impalad/Catalog/Statestore web pages.
This includes setting VLOG levels (equivalent to --v flag)
as well as setting log4j levels on the Frontend and the
Catalog JVMs.
Change-Id: I588418e9bcb0b66d33138baf96207a5a35bfbd63
Reviewed-on: http://gerrit.cloudera.org:8080/5792
Reviewed-by: Bharath Vissapragada <bharathv@cloudera.com>
Tested-by: Impala Public Jenkins
Here is a basic summary of the changes:
Frontend looks for conjuncts that operate on a single slot and pass a
map from slot id to the conjunct index through thrift to the backend.
The conjunct indices are the indices into the normal PlanNode conjuncts list.
The conjuncts need to satisfy certain conditions:
1. They are bound on a single slot
2. They are deterministic (no random functions)
3. They evaluate to FALSE on a NULL input. This is because the dictionary
does not include NULLs, so any condition that evaluates to TRUE on NULL
cannot be evaluated by looking only at the dictionary.
The backend converts the indices into ExprContexts. These are cloned in
the scanner threads.
The dictionary read codepath has been removed from ReadDataPage into its
own function, InitDictionary. This has also been turned into its own step
in row group initialization. ReadDataPage will not see any dictionary
pages unless the parquet file is invalid.
For dictionary filtering, we initialize dictionaries only as needed to evaluate
the conjuncts. The Parquet scanner evaluates the dictionary filter conjuncts on the
dictionary to see if any dictionary entry passes. If no entry passes, the row
group is eliminated. If the row group passes the dictionary filtering, then we
initialize all remaining dictionaries.
Dictionary filtering is controlled by a new query option,
parquet_dictionary_filtering, which is on by default.
Since column chunks can have a mixture of encodings, dictionary filtering
uses three tests to determine whether this is purely dictionary encoded:
1. If the encoding_stats is in the parquet file, then use it to determine if
there are only dictionary encoded pages (i.e. there are no data pages with
an encoding other than PLAIN_DICTIONARY).
-OR-
2. If the encoding stats are not present, then look at the encodings. The column
is purely dictionary encoded if:
a) PLAIN_DICTIONARY is present
AND
b) Only PLAIN_DICTIONARY, RLE, or BIT_PACKED encodings are listed
-OR-
3. If this file was written by an older version of Impala, then we know that
dictionary failover happens when the dictionary reaches 40,000 values.
Dictionary filtering can proceed as long as the dictionary is smaller than
that.
parquet-mr writes the encoding list correctly in the current version in our
environment (1.5.0). This means that check #2 works on some existing files
(potentially most existing parquet-mr files).
parquet-mr writes the encoding stats starting in 1.9.0. This is the version
where check #1 will start working.
Impala's parquet writer now implements both, so either check above will work.
Change-Id: I3a7cc3bd0523fbf3c79bd924219e909ef671cfd7
Reviewed-on: http://gerrit.cloudera.org:8080/5904
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Impala Public Jenkins
This change adds support for skipping row groups based on Parquet row
group statistics. With this change we only support reading statistics
from Parquet files for numerical types (bool, integer, floating point)
and for simple predicates of the forms <slot> <op> <constant> or
<constant> <op> <slot>, where <op> is LT, LE, GE, GT, and EQ.
Change-Id: I39b836165756fcf929c801048d91c50c8fdcdae4
Reviewed-on: http://gerrit.cloudera.org:8080/6032
Reviewed-by: Lars Volker <lv@cloudera.com>
Tested-by: Impala Public Jenkins
This turned out to be slightly non-trivial as REPLACE is already a
keyword, and thus the parser needs to be tweaked to allow this,
since function names act as bare identifiers.
It was difficult to get this to match performance of regexp_replace.
For expanding patterns, the fact that regexp_replace copies the
expansion inline means that it may in fact win on large strings
with sparse matches that are > dcache size apart. Let's leave
optimizing that for later.
Testing: Added a full test for maximum size strings and got most
of the boundary conditions I could identify. Manually ran queries
on TPC-H dataset in impala to verify both performance and correctness.
Added large string and exprs.test test clauses and ran the tests to
verify they work as expected.
Change-Id: I1780a7d8fee6d0db9dad148217fb6eb10f773329
Reviewed-on: http://gerrit.cloudera.org:8080/5776
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins
This uses the existing infrastructure for codegening builtin UDAs and
for codegening calls to UDFs. GetUdf() is refactored to support both
UDFs and UDAs.
IR UDAs are still not allowed by the frontend. It's unclear if we want
to enable them going forward because of the difficulties in testing and
supporting IR UDFs/UDAs.
This also fixes some bugs with the Get*Type() methods of
FunctionContext. GetArgType() and related methods now always return the
logical input types of the aggregate function. Getting the tests to pass
required fixing IMPALA-4878 because they called GetIntermediateType().
Testing:
test_udfs.py tests UDAs with codegen enabled and disabled.
Added some asserts to test UDAs to check that the correct types are
passed in via the FunctionContext.
Change-Id: Id1708eaa96eb76fb9bec5eeabf209f81c88eec2f
Reviewed-on: http://gerrit.cloudera.org:8080/5161
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins
The test should allow Unpin() to fail with a scratch allocation error to
handle the case where the first write fails and blacklists the scratch
disk around the same time that the second write starts. Usually either
the second write succeeds because it started before the first write
failed or it fails with CANCELLED because the
BufferedBlockMgr::is_cancelled_ flag is set. There is a small
window for a race after the disk is blacklisted in TmpFileMgr but
before BufferedBlockMgr::WriteComplete() is called.
Testing:
I was able to reproduce the problem locally by adding some delays
to the test. I added a variant of the WriteError test that more reliably
reproduces the bug. Ran both WriteError tests in a loop locally to try
to flush out flakiness.
Change-Id: I9878d7000b03a64ee06c2088a8c30e318fe1d2a3
Reviewed-on: http://gerrit.cloudera.org:8080/5940
Tested-by: Impala Public Jenkins
Reviewed-by: Michael Ho <kwho@cloudera.com>
- Removes the runtime unknown disk ID reporting and instead moves
it to the explain plan as a counter that prints the number of
scan ranges missing disk IDs in the corresponding HDFS scan nodes.
- Adds a warning to the header of query profile/explain plan with a
list of tables missing disk ids.
- Removes reference to enabling dfs block metadata configuration,
since it doesn't apply anymore.
- Removes VolumeId terminology from the runtime profile.
Change-Id: Iddb132ff7ad66f3291b93bf9d8061bd0525ef1b2
Reviewed-on: http://gerrit.cloudera.org:8080/5828
Reviewed-by: Bharath Vissapragada <bharathv@cloudera.com>
Tested-by: Impala Public Jenkins
ADD PARTITION
Just like Hive, Implala should support multiple partitions in ALTER
TABLE ADD PARTITION statements. The syntax is as follows:
ALTER TABLE table_name ADD [IF NOT EXISTS]
PARTITION partition_spec1 [location_spec1] [cache_spec1]
PARTITION partition_spec2 [location_spec2] [cache_spec2]
...
Grammar was modified to handle the new syntax. Introduced PartitionDef
class to capture the repeatable part of the statement. TPartitionDef
is the name of the corresponding thrift class.
AlterTableAddPartitionStmt and CatalogOpExecutor classes were also
modified to work with a list of partitions. Duplicate partition specs
are rejected in AlterTableAddPartitionStmt.analyze().
Added FE, E2E and integration tests.
Change-Id: Iddbc951f2931f488f7048c9780260f6b49100750
Reviewed-on: http://gerrit.cloudera.org:8080/4144
Reviewed-by: Dimitris Tsirogiannis <dtsirogiannis@cloudera.com>
Tested-by: Impala Public Jenkins
This query option is going to be used to change both rounding semantics
and establish more appropriate output types for multiply / divide.
The option won't be supported until all the changes are complete,
but this will eventually become the default behavior for DECIMAL.
Change-Id: I6cf25fe5c1766c86ebfe196d49f646e81e50a24e
Reviewed-on: http://gerrit.cloudera.org:8080/5889
Reviewed-by: Michael Ho <kwho@cloudera.com>
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Impala Public Jenkins