Commit Graph

61 Commits

Author SHA1 Message Date
Joe McDonnell
c5a0ec8bdf IMPALA-11980 (part 1): Put all thrift-generated python code into the impala_thrift_gen package
This puts all of the thrift-generated python code into the
impala_thrift_gen package. This is similar to what Impyla
does for its thrift-generated python code, except that it
uses the impala_thrift_gen package rather than impala._thrift_gen.
This is a preparatory patch for fixing the absolute import
issues.

This patches all of the thrift files to add the python namespace.
This has code to apply the patching to the thirdparty thrift
files (hive_metastore.thrift, fb303.thrift) to do the same.

Putting all the generated python into a package makes it easier
to understand where the imports are getting code. When the
subsequent change rearranges the shell code, the thrift generated
code can stay in a separate directory.

This uses isort to sort the imports for the affected Python files
with the provided .isort.cfg file. This also adds an impala-isort
shell script to make it easy to run.

Testing:
 - Ran a core job

Change-Id: Ie2927f22c7257aa38a78084efe5bd76d566493c0
Reviewed-on: http://gerrit.cloudera.org:8080/20169
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
2025-04-15 17:03:02 +00:00
Xuebin Su
d7ee509e93 IMPALA-12648: Add KILL QUERY statement
To support killing queries programatically, this patch adds a new
type of SQL statements, called the KILL QUERY statement, to cancel and
unregister a query on any coordinator in the cluster.

A KILL QUERY statement looks like
```
KILL QUERY '123:456';
```
where `123:456` is the query id of the query we want to kill. We follow
syntax from HIVE-17483. For backward compatibility, 'KILL' and 'QUERY'
are added as "unreserved keywords", like 'DEFAULT'. This allows the
three keywords to be used as identifiers.

A user is authorized to kill a query only if the user is an admin or is
the owner of the query. KILL QUERY statements are not affected by
admission control.

Implementation:

Since we don't know in advance which impalad is the coordinator of the
query we want to kill, we need to broadcast the kill request to all the
coordinators in the cluster. Upon receiving a kill request, each
coordinator checks whether it is the coordinator of the query:
- If yes, it cancels and unregisters the query,
- If no, it reports "Invalid or unknown query handle".

Currently, a KILL QUERY statement is not interruptible. IMPALA-13663 is
created for this.

For authorization, this patch adds a custom handler of
AuthorizationException for each statement to allow the exception to be
handled by the backend. This is because we don't know whether the user
is the owner of the query until we reach its coordinator.

To support cancelling child queries, this patch changes
ChildQuery::Cancel() to bypass the HS2 layer so that the session of the
child query will not be added to the connection used to execute the
KILL QUERY statement.

Testing:
- A new ParserTest case is added to test using "unreserved keywords" as
  identifiers.
- New E2E test cases are added for the KILL QUERY statement.
- Added a new dimension in TestCancellation to use the KILL QUERY
  statement.
- Added file tests/common/cluster_config.py and made
  CustomClusterTestSuite.with_args() composable so that common cluster
  configs can be reused in custom cluster tests.

Change-Id: If12d6e47b256b034ec444f17c7890aa3b40481c0
Reviewed-on: http://gerrit.cloudera.org:8080/21930
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Michael Smith <michael.smith@cloudera.com>
2025-01-22 22:22:54 +00:00
Peter Rozsa
a0aaf338ae IMPALA-12732: Add support for MERGE statements for Iceberg tables
MERGE statement is a DML command that allows users to perform
conditional insert, update, or delete operations on a target table based
on the results of a join with a source table. This change adds MERGE
statement parsing and an Iceberg-specific semantic analysis, planning,
and execution. The parsing grammar follows the SQL standard, it accepts
the same syntax as Hive, Spark, and Trino by supporting arbitrary number
of WHEN clauses, with conditions or without and accepting inline views
as source.

Example:
'MERGE INTO target t USING source s ON t.id = s.id
WHEN MATCHED AND t.id < 100 THEN UPDATE SET column1 = s.column1
WHEN MATCHED AND t.id > 100 THEN DELETE
WHEN MATCHED THEN UPDATE SET column1 = "value"
WHEN NOT MATCHED THEN INSERT VALUES (s.id, s.column1);'

The Iceberg-specific analysis, planning, and execution are based on a
concept that was previously used for UPDATE: The analyzer creates a
SELECT statement with all target and source columns (including
Iceberg's virtual columns) and a 'row_present' column that defines
whether the source, the target, or both rows are present in the result
set after joining the two table references by the ON clause. The join
condition should be an equi-join, as it is a FULL OUTER JOIN, and Impala
currently supports only equi-joins in this case. The joining order is
forced by a query hint, this guarantees that the target table is always
on the left side.

A new, IcebergMergeNode is added at planning phase, this node does the
row-level filtering for each MATCHED/ NOT MATCHED cases. The
'row_present' column decides which case group will be evaluated; if
both sides are available, the matched cases, if only the source side
matches then the not matched cases and their filter expressions
will be evaluated over the row. If one of the cases match, then the
execution evaluates the result expressions into the output row batch,
and an auxiliary tuple will store the merge action. The merge action is
a flag for the newly added IcebergMergeSink; this sink will route each
incoming row from IcebergMergeNode to their respective destination. Each
row could go to the delete sink, insert sink, or to both sinks.

Target-side duplicate records are filtered during IcebergMergeNode's
execution, if one target table-side duplicate is detected, the whole
statement's execution is stopped and the error is reported back to the
user.

Added tests:
 - Parser tests
 - Analyzer tests
 - Unit test for WHEN NOT MATCHED INSERT column collation
 - Planner tests for partitioned/sorted cases
 - Authorization tests
 - E2E tests

Change-Id: I3416a79740eddc446c87f72bf1a85ed3f71af268
Reviewed-on: http://gerrit.cloudera.org:8080/21423
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2024-09-05 01:01:05 +00:00
Michael Smith
45995e6892 IMPALA-12540: Query Live Table
Defines SystemTable which are in-memory tables that can provide access
to Impala state. Adds the 'impala_query_live' to the database 'sys',
which already exists for 'sys.impala_query_log'.

Implements the 'impala_query_live' table to view active queries across
all coordinators sharing the same statestore. SystemTables create new
SystemTableScanNodes for their scan node implementation. When computing
scan range locations, SystemTableScanNodes creates a scan range for each
in the cluster (identified via ClusterMembershipMgr). This produces a
plan that looks like:

Query: explain select * from sys.impala_query_live
+------------------------------------------------------------+
| Explain String                                             |
+------------------------------------------------------------+
| Max Per-Host Resource Reservation: Memory=4.00MB Threads=2 |
| Per-Host Resource Estimates: Memory=11MB                   |
| WARNING: The following tables are missing relevant table   |
| and/or column statistics.                                  |
| sys.impala_query_live                                      |
|                                                            |
| PLAN-ROOT SINK                                             |
| |                                                          |
| 01:EXCHANGE [UNPARTITIONED]                                |
| |                                                          |
| 00:SCAN SYSTEM_TABLE [sys.impala_query_live]               |
|    row-size=72B cardinality=20                             |
+------------------------------------------------------------+

Impala's scheduler checks for whether the query contains fragments that
can be scheduled on coordinators, and if present includes an
ExecutorGroup containing all coordinators. These are used to schedule
scan ranges that are flagged as 'use_coordinator', allowing
SystemTableScanNodes to be scheduled on dedicated coordinators and
outside the selected executor group.

Execution will pull data from ImpalaServer on the backend via a
SystemTableScanner implementation based on table name.

In the query profile, SYSTEM_TABLE_SCAN_NODE includes
ActiveQueryCollectionTime and PendingQueryCollectionTime to track time
spent collecting QueryState from ImpalaServer.

Grants QueryScanner private access to ImpalaServer, identical to how
ImpalaHttpHandler access internal server state.

Adds custom cluster tests for impala_query_live, and unit tests for
changes to planner and scheduler.

Change-Id: Ie2f9a449f0e5502078931e7f1c5df6e0b762c743
Reviewed-on: http://gerrit.cloudera.org:8080/20762
Reviewed-by: Jason Fehr <jfehr@cloudera.com>
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2024-03-28 16:34:48 +00:00
Noemi Pap-Takacs
47db4fd1f5 IMPALA-12412: Support partition evolution in OPTIMIZE statement
The OPTIMIZE statement is used to execute table maintenance tasks
on Iceberg tables, such as:
 1. compacting small files,
 2. merging delete deltas,
 3. rewriting the table according to the latest schema
    and partition spec.

OptimizeStmt used to serve as an alias for INSERT OVERWRITE.
After this change it works as follows: It creates a source statement
that contains all columns of the table. All table content will be
rewritten to new data files. After the executors finished writing,
the Catalog calls RewriteFiles Iceberg API to commit the changes.
All previous data and delete files will be excluded from,
and all newly written data files will be added to the next
snapshot. The old files remain accessible via time travel
to older snapshots of the table.

By default, Impala has as many file writers as query fragment instances
and therefore can write too many files for unpartitioned tables.
For smaller tables this can be limited by setting the
MAX_FS_WRITERS Query Option.

Authorization: OPTIMIZE TABLE requires ALL privileges.

Limitations:
All limitations about writing Iceberg tables apply.

Testing:
 - E2E tests:
     - schema evolution
     - partition evolution
     - UPDATE/DELETE
     - time travel
     - table history
 - negative tests
 - Ranger tests for authorization
 - FE: Planner test:
     - sorting order
     - MAX_FS_WRITERS
     - partitioned exchange
     - Parser test
Change-Id: I65a0c8529d274afff38ccd582f1b8a857716b1b5
Reviewed-on: http://gerrit.cloudera.org:8080/20866
Reviewed-by: Daniel Becker <daniel.becker@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2024-02-29 18:37:16 +00:00
Michael Smith
f3ac2ddbfe IMPALA-12747: Atomic update of execution state
QueryDriver owns instances of ClientRequestState and TExecRequest. The
ClientRequestState is used to track execution state of the client-facing
side of a query. TExecRequest encapsulates context about the query
produced by the planner.

When a QueryDriver is created, it creates an instance of
ClientRequestState, but has not yet executed planning. It would create
an empty TExecRequest and pass a pointer to it to ClientRequestState,
then update the content of TExecRequest when RunFrontendPlanner is
called from ImpalaServer::ExecuteInternal.

Updating TExecRequest was not atomic, so it was possible other
operations - like producing a QueryStateRecord for /queries in the web
UI - would try to read the content of TExecRequest while updating. This
caused TSAN errors and occasional crashes in internal-server-test, which
runs concurrent requests and examines them through calls to /queries.

Changes ClientRequestState to
- Provide a static placeholder for TExecRequest during creation that
  represents an empty context for an UNKNOWN statement type (default
  initialized in Thrift).
- Make all references to TExecRequest const so its content cannot be
  updated in a non-thread-safe manner.
- ClientRequestState uses an AtomicPtr which is updated atomically when
  the filled TExecRequest is available.

QueryDriver does not publicly expose access to TExecRequest, so we can
ensure its use is thread-safe without atomics.

ClientRequestState::exec_request() will return either a reference to the
static placeholder or the value provided after - which is never changed
- so this reference will always be valid for the lifetime of the
ClientRequestState.

Updates user_has_profile_access to be AtomicBool for the same reason.

Reverts tsan-suppressions for IMPALA-12660 so we get TSAN coverage. Adds
suppression for a lock-order-inversion bug (IMPALA-12757) that was
uncovered after fixing this data race.

Testing:
- InternalServerTest.SimultaneousMultipleQueriesOneSession would fail
  after ~10 test runs. Ran 90 times without failure.
- Passed TSAN run of backend tests.

Change-Id: I9a967c5c84b6a401f8f5764373f6cd7ee807545f
Reviewed-on: http://gerrit.cloudera.org:8080/20956
Reviewed-by: Jason Fehr <jfehr@cloudera.com>
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2024-01-26 23:17:10 +00:00
Zoltan Borok-Nagy
e326b3cc0d IMPALA-12313: (part 2) Limited UPDATE support for Iceberg tables
This patch adds limited UPDATE support for Iceberg tables. The
limitations mean users cannot update Iceberg tables if any of
the following is true:
 * UPDATE value of partitioning column
 * UPDATE table that went through partition evolution
 * Table has SORT BY properties

The above limitations will be resolved by part 3. The usual limitations
like writing non-Parquet files, using copy-on-write, modifying V1 tables
are out of scope of IMPALA-12313.

This patch implements UPDATEs with the merge-on-read technique. This
means the UPDATE statement writes both data files and delete files.
Data files contain the updated records, delete files contain the
position delete records of the old data records that have been
touched.

To achieve the above this patch introduces a new sink: MultiDataSink.
We can configure multiple TableSinks for a single MultiDataSink object.
During execution, the row batches sent to the MultiDataSink will be
forwarded to all the TableSinks that have been registered.

The UPDATE statement for an Iceberg table creates a source select
statement with all table columns and virtual columns INPUT__FILE__NAME
and FILE__POSITION. E.g. imagine we have a table 'tbl' with schema
(i int, s string, k int), and we update the table with:

  UPDATE tbl SET k = 5 WHERE i % 100 = 11;

 The generated source statement will be ==>

  SELECT i, s, 5, INPUT__FILE__NAME, FILE__POSITION
  FROM tbl WHERE i % 100 = 11;

Then we create two table sinks that refer to expressions from the above
source statement:

  Insert sink (i, s, 5)
  Delete sink (INPUT__FILE__NAME, FILE__POSITION)

The tuples in the rowbatch of MultiDataSink contain slots for all the
above expressions (i, s, 5, INPUT__FILE__NAME, FILE__POSITION).
MultiDataSink forwards each row batch to each registered TableSink.
They will pick their relevant expressions from the tuple and write
data/delete files. The tuples are sorted by INPUTE__FILE__NAME and
FILE__POSITION because we need to write the delete records in this
order.

For partitioned tables we need to shuffle and sort the input tuples.
In this case we also add virtual columns "PARTITION__SPEC__ID" and
"ICEBERG__PARTITION__SERIALIZED" to the source statement and shuffle
and sort the rows based on them.

Data files and delete files are now separated in the DmlExecState, so
at the end of the operation we'll have two sets of files. We use these
two sets to create a new Iceberg snapshot.

Why does this patch have the limitations?
 - Because we are shuffling and sorting rows based on the delete
   records and their partitions. This means that the new data files
   might not get written in an efficient way, e.g. there will be
   too many of them, or we will need to keep too many open file
   handles during writing.
   Also, if the table has SORT BY properties, we cannot respect
   it as the input rows are ordered in a way to favor the position
   deletes.
   Part 3 will introduce a buffering writer for position delete
   files. This means we will shuffle and sort records based on
   the data records' partitions and SORT BY properties while
   delete records get buffered and written out at the end (sorted
   by file_path and position). In some edge cases the delete records
   might not get written efficiently, but it is a smaller problem
   then inefficient data files.

Testing:
 * negative tests
 * planner tests
 * update all supported data types
 * partitioned tables
 * Impala/Hive interop tests
 * authz tests
 * concurrent tests

Change-Id: Iff0ef6075a2b6ebe130d15daa389ac1a505a7a08
Reviewed-on: http://gerrit.cloudera.org:8080/20677
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2023-12-09 03:04:05 +00:00
LPL
929b91ac64 IMPALA-11013: Support 'MIGRATE TABLE' for external Hive tables
This patch implements the migration from legacy Hive tables to Iceberg
tables. The target Iceberg tables inherit the location of the original
Hive tables. The Hive table has to be a non-transactional table.

To migrate a Hive format table stored in a distributed system or object
store to an Iceberg table use the command:

ALTER TABLE [dbname.]table_name CONVERT TO ICEBERG [TBLPROPERTIES(...)];

Currently only 'iceberg.catalog' is allowed as a table property.

For example
     - ALTER TABLE hive_table CONVERT TO ICEBERG;
     - ALTER TABLE hive_table CONVERT TO ICEBERG TBLPROPERTIES(
       'iceberg.catalog' = 'hadoop.tables');

The HDFS table to be converted must follow those requirements:
     - table is not a transactional table
     - InputFormat must be either PARQUET, ORC, or AVRO

This is an in-place migration so the original data files of the legacy
Hive table are re-used and not moved, copied or re-created by this
operation. The new Iceberg table will have the 'external.table.purge'
property set to true after the migration.

NUM_THREADS_FOR_TABLE_MIGRATION query option can control the maximum
number of threads to execute the table conversion. The default value is
one, meaning that table conversion runs on one thread. It can be
configured in a range of [0, 1024]. Zero means that the number of CPU
cores will be the degree of parallelism. A value greater than zero will
imply the number of threads used for table conversion, however, there
is a cap of the number of CPU cores as the highest degree of
parallelism.

Process of migration:
 - Step 1: Setting table properties,
           e.g. 'external.table.purge'=false on the HDFS table.
 - Step 2: Rename the HDFS table to a temporary table name using a name
           format of "<original_table_name>_tmp_<random_ID>".
 - Step 3: Refresh the renamed HDFS table.
 - Step 4: Create an external Iceberg table by Iceberg API using the
           data of the Hdfs table.
 - Step 5 (Optional): For an Iceberg table in Hadoop Tables, run a
           CREATE TABLE query to add the Iceberg table to HMS as well.
 - Step 6 (Optional): For an Iceberg table in Hive catalog, run an
           INVALIDATE METADATA to make the new table available for all
           coordinators right after the conversion finished.
 - Step 7 (Optional): For an Iceberg table in Hadoop Tables, set the
           'external.table.purge' property to true in an ALTER TABLE
           query.
 - Step 8: Drop the temporary HDFS table.

Testing:
 - Add e2e tests
 - Add FE UTs
 - Manually tested the runtime performance for a table that is
   unpartitioned and has 10k data files. The runtime is around 10-13s.

Co-authored-by: lipenglin <lipenglin@apache.org>

Change-Id: Iacdad996d680fe545cc9a45e6bc64a348a64cd80
Reviewed-on: http://gerrit.cloudera.org:8080/20077
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Tamas Mate <tmater@apache.org>
2023-07-03 08:17:41 +00:00
Zoltan Borok-Nagy
c579212251 IMPALA-11877: (part 1) Add support for DELETE statements for UNPARTITIONED Iceberg tables
This patch adds support for DELETE statements on unpartitioned Iceberg
tables. Impala uses the 'merge-on-read' mode with position delete files.

The patch reuses the existing IcebergPositionDeleteTable as the target
table of the DELETE statements, because this table already has the same
schema as position delete files, even with correct Iceberg field IDs.

The patch basically rewrites DELETE statements to INSERT statements,
e.g.:

from:
 DELETE FROM ice_t WHERE id = 42;

to:
 INSERT INTO ice_t-POSITION-DELETE
 SELECT INPUT__FILE__NAME, FILE__POSITION
 FROM ice_t
 WHERE id = 42;

Position delete files need to be ordered by (file_path, pos), so
we add an extra SORT node before the table sink operator.

In the backend the patch adds a new table sink operator, the
IcebergDeleteSink. It writes the incoming rows (file_path, position) to
delete files. It reuses a lot of code from HdfsTableSink, so this patch
moves the common code to the new common base class: TableSinkBase.

The coordinator then collects the written delete files and invokes
UpdateCatalog to finalize the DELETE statement.

The Catalog then uses Iceberg APIs to create a new snapshot with the
created delete files. It also validates that there was no conflicting
data files written since the operation started.

Testing:
 * added planer test
 * e2e tests
 * interop test between Impala and Hive

Change-Id: Ic933b2295abe54b46d2a736961219988ff42915b
Reviewed-on: http://gerrit.cloudera.org:8080/19776
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Gabor Kaszab <gaborkaszab@cloudera.com>
2023-06-13 11:33:32 +00:00
wzhou-code
eda2aa5553 IMPALA-11129: Support running KRPC over Unix Domain Socket
This patch make following changes to support running KRPC over UDS.
  - Add FLAGS_rpc_use_unix_domain_socket to enable running KRPC over
    UDS. Add FLAGS_uds_address_unique_id to specify unique Id for UDS
    address. It could be 'ip_address', 'backend_id', or 'none'.
  - Add variable uds_address in NetworkAddressPB and TNetworkAddress.
    Replace TNetworkAddress with NetworkAddressPB for KRPC related
    class variables and APIs.
  - Set UDS address for each daemon as @impala-kprc:<unique_id>
    during initialization with unique_id specified by starting flag
    FLAGS_uds_address_unique_id.
  - When FLAG_rpc_use_unix_domain_socket is true, the socket of KRPC
    server will be binded to the UDS address of the daemon.
    KRPC Client will connect to KRPC server with the UDS address of
    the server when creating proxy service, which in turn call
    kudu::Socket::Connect() function to connect KRPC server.
  - rpcz Web page show TCP related stats as 'N/A' when using UDS.
    Show remote UDS address for KRPC inbound connections on rpcz Web
    page as '*' when using UDS since the remote UDS addresses are
    not available.
  - Add new unit-tests for UDS.
  - BackendId of admissiond is not available. Use admissiond's IP
    address as unique ID for UDS.
    TODO: Advertise BackendId of admissiond in global admission
    control mode.

Testing:
  - Passed core test with FLAG_rpc_use_unix_domain_socket as fault
    value false.
  - Passed core test with FLAG_rpc_use_unix_domain_socket as true.

Change-Id: I439f5a03eb425c17451bcaa96a154bb0bca17ee7
Reviewed-on: http://gerrit.cloudera.org:8080/18369
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2022-05-25 06:27:24 +00:00
skyyws
a850cd3cc6 IMPALA-10361: Use field id to resolve columns for Iceberg tables
We supported resolve column by field id for Iceberg table in this
patch. Currently, we use field id to resolve column for Iceberg
tables, which means 'PARQUET_FALLBACK_SCHEMA_RESOLUTION' is invalid
for Iceberg tables.

Change-Id: I057bdc6ab2859cc4d40de5ed428d0c20028b8435
Reviewed-on: http://gerrit.cloudera.org:8080/16788
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Zoltan Borok-Nagy <boroknagyz@cloudera.com>
2020-12-10 19:01:08 +00:00
Thomas Tauber-Marshall
ae0bd674a8 IMPALA-9425 (part 1): Introduce uuids for impalads
This patch introduces the concept of 'backend ids', which are unique
ids that can be used to identify individual impalads. The ids are
generated by each impalad on startup.

The patch then uses the ids to fix a bug where the statestore may fail
to inform coordinators when an executor has failed and restarted. The
bug was caused by the fact that the statestore cluster membership
topic was keyed on statestore subscriber ids, which are host:port
pairs.

So, if an impalad fails and a new one is started at the same host:port
before a particular coordinator has a cluster membership update
generated for it by the statestore, the statestore has no way of
differentiating the prior impalad from the newly started impalad, and
the topic update will not show the deletion of the original impalad.

With this patch, the cluster membership topic is now keyed by backend
id, so since the restarted impalad will have a different backend id
the next membership update after the prior impalad failed is
guaranteed to reflect that failure.

The patch also logs the backend ids on startup and adds them to the
/backends webui page and to the query locations section of the
/queries page, for use in debugging.

Further patches will apply the backend ids in other places where we
currently key off host:port pairs to identify impalads.

Testing:
- Added an e2e test that uses a new debug action to add delay to
  statestore topic updates. Due to the use of JITTER the test is
  non-deterministic, but it repros the original issue locally for me
  about 50% of the time.
- Passed a full run of existing tests.

Change-Id: Icf8067349ed6b765f6fed830b7140f60738e9061
Reviewed-on: http://gerrit.cloudera.org:8080/15321
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2020-03-04 23:43:43 +00:00
Tim Armstrong
0bb056e525 IMPALA-4224: execute separate join builds fragments
This enables parallel plans with the join build in a
separate fragment and fixes all of the ensuing fallout.
After this change, mt_dop plans with joins have separate
build fragments. There is still a 1:1 relationship between
join nodes and builders, so the builders are only accessed
by the join node's thread after it is handed off. This lets
us defer the work required to make PhjBuilder and NljBuilder
safe to be shared between nodes.

Planner changes:
* Combined the parallel and distributed planning code paths.
* Misc fixes to generate reasonable thrift structures in the
  query exec requests, i.e. containing the right nodes.
* Fixes to resource calculations for the separate build plans.
** Calculate separate join/build resource consumption.
** Simplified the resource estimation by calculating resource
   consumption for each fragment separately, and assuming that
   all fragments hit their peak resource consumption at the
   same time. IMPALA-9255 is the follow-on to make the resource
   estimation more accurate.

Scheduler changes:
* Various fixes to handle multiple TPlanExecInfos correctly,
  which are generated by the planner for the different cohorts.
* Add logic to colocate build fragments with parent fragments.

Runtime filter changes:
* Build sinks now produce runtime filters, which required
  planner and coordinator fixes to handle.

DataSink changes:
* Close the input plan tree before calling FlushFinal() to release
  resources. This depends on Send() not holding onto references
  to input batches, which was true except for NljBuilder. This
  invariant is documented.

Join builder changes:
* Add a common base class for PhjBuilder and NljBuilder with
  functions to handle synchronisation with the join node.
* Close plan tree earlier in FragmentInstanceState::Exec()
  so that peak resource requirements are lower.
* The NLJ always copies input batches, so that it can close
  its input tree.

JoinNode changes:
* Join node blocks waiting for build-side to be ready,
  then eventually signals that it's done, allowing the builder
  to be cleaned up.
* NLJ and PHJ nodes handle both the integrated builder and
  the external builder. There is a 1:1 relationship between
  the node and the builder, so we don't deal with thread safety
  yet.
* Buffer reservations are transferred between the builder and join
  node when running with the separate builder. This is not really
  necessary right now, since it is all single-threaded, but will
  be important for the shared broadcast.
  - The builder transfers memory for probe buffers to the join node
    at the end of each build phase.
  - At end of each probe phase, reservation needs to be handed back
    to builder (or released).

ExecSummary changes:
* The summary logic was modified to handle connecting fragments
  via join builds. The logic is an extension of what was used
  for exchanges.

Testing:
* Enable --unlock_mt_dop for end-to-end tests
* Migrate some tests to run as part of end-to-end tests instead of
  custom cluster.
* Add mt_dop dimension to various end-to-end tests to provide
  coverage of join queries, spill-to-disk and cancellation.
* Ran a single node TPC-H and TPC-DS stress test with mt_dop=0
  and mt_dop=4.

Perf:
* Ran TPC-H scale factor 30 locally with mt_dop=0. No significant
  change.

Change-Id: I4403c8e62d9c13854e7830602ee613f8efc80c58
Reviewed-on: http://gerrit.cloudera.org:8080/14859
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2020-02-20 01:51:54 +00:00
norbert.luksa
288c8c41b5 IMPALA-8755: Frontend support for Z-ordering
Extended the SQL grammar with an optional and a default flag for
SORT BY, namely ZORDER and LEXICAL. If set, the new 'sort.algorithm'
table property will be set to ZORDER and the information will sink
down to the backend. The default order is indicated by LEXICAL
and can be omitted. Examples are:

CREATE TABLE t (a INT, b INT) PARTITIONED BY (c INT)
  SORT BY ZORDER (a, b);
CREATE TABLE t SORT BY ZORDER (int_col,id) LIKE u;
CREATE TABLE t LIKE PARQUET '/foo' SORT BY ZORDER (id,zip);

ALTER TABLE t SORT BY ZORDER (int_col,id);

The following two are the same statements:
CREATE TABLE t (a INT, b INT) SORT BY (a, b);
CREATE TABLE t (a INT, b INT) SORT BY LEXICAL (a, b);

For strings, varchars, floats and doubles Z-ordering is currently
not supported. It's not suitable for strings and varchars, but
support can be added for floats and doubles later. The supported
types are: boolean, int types, decimals, date, timestamp, and char.

Currently ZORDER has the same functionality as a simple SORT BY clause,
therefore hidden behind a feature flag: unlock_zorder. The custom
sorting with Z-ordering will be in a different commit later.

Testing:
 * Added tests for the ZORDER option for every SORT BY test.
 * Modified some tests by adding the LEXICAL option.
 * The .test workloads are temporarily put in separate test files
   in order to set up the feature flag. These tests are run from
   tests/custom_cluster/test_zorder.py which is a duplication of
   the relevant tests, but with CustomClusterTestSuite decorator.

Change-Id: Ie122002ca8f52ca2c1e1ec8ff1d476ae1f4f875d
Reviewed-on: http://gerrit.cloudera.org:8080/13955
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2019-09-26 18:35:06 +00:00
Lars Volker
91d8a8f628 IMPALA-8234: Fix ordering of Thrift enum, fix enum values, add warning
IMPALA-7694 added a field in the middle of the Metrics.TUnit enum, which
broke backwards compatibility with profiles that had been written by
older versions of Impala. This change fixes the ordering by moving the
field to the end of the enum.

Additionally, it adds a warning to the top of all Thrift files that are
part of the binary profile format, and an note of caution to the main
definition in RuntimeProfile.thrift.

This change also fixes the order of all enums in our Thrift files to
make errors like this less likely in the future.

Change-Id: If215f16a636008757ceb439edbd6900a1be88c59
Reviewed-on: http://gerrit.cloudera.org:8080/12543
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2019-02-22 04:01:35 +00:00
Bharath Vissapragada
f7df8adfae IMPALA-5872: Testcase builder for query planner
Implements a new testcase builder for simulating query plans
from one cluster on a different cluster/minicluster with
different number of nodes. The testcase is collected from one
cluster and can be replayed on any other cluster. It includes
all the information that is needed to replay the query plan
exactly as in the source cluster.

Also adds a stand-alone tool (PlannerTestCaseLoader) that can
replay the testcase without having to start an actual cluster
or a dev minicluster. This is done to make testcase debugging
simpler.

Motivation:
----------
- Make query planner issues easily reproducible
- Improve user experience while collecting query diagnostics
- Make it easy to test new planner features by testing it on customer
  usecases collected from much larger clusters.

Commands:
--------
-- Collect testcase for a query stmt (outputs the testcase file path).
impala-shell> COPY TESTCASE TO <hdfs dirpath> <query stmt>

-- Load the testcase metadata in a target cluster (dumps the query stmt)
impala-shell> COPY TESTCASE FROM <hdfs testcase file path>
-- Replay the query plan
impala-shell> SET PLANNER_DEBUG_MODE=true
impala-shell> EXPLAIN <query stmt>

How it works?
------------
- During export on the source cluster, the command dumps all the thrift
  states of referenced objects in the query into a gzipped binary file.
- During replay on a target cluster, it adds these objects to the catalog
  cache by faking them as DDLs.
- The planner also fakes the number of hosts by using the scan range
  information from the target cluster.

Caveats:
------
- Tested to work with HDFS tables. Tables based on other filesystems like
  HBase/Kudu may not work as desired.
- The tool does not collect actual data files for the tables. Only the
  metadata state is dumped.
- Currently only imports databases/tables/views. We can extend it to
  work for UDFS etc.
- It only works for QueryStmts (select/union queries)
- On a sentry enabled cluster, the role running the query requires
  VIEW_METADATA privilege on every db/table/view referenced in the query
  statement.
- Once the metadata dump is loaded on a target cluster, the state is
  volatile. Hence it cannot survive a cluster restart / invalidate
  metadata
- Loading a testcase requires setting the query option (SET
  PLANNER_DEBUG_MODE=true) so that the planner knows to fake the number
  of hosts. Otherwise it takes into account the local cluster topology.
- Cross version compatibility of testcases needs some thought. For
  example, creating a testcase from Impala version 3.2 and trying to
  replay it on Impala version 3.5. This could be problematic if we don't
  keep the underlying thrift structures backward compatible.

Change-Id: Iec83eeb2dc5136768b70ed581fb8d3ed0335cb52
Reviewed-on: http://gerrit.cloudera.org:8080/12221
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2019-02-09 03:59:10 +00:00
Tim Armstrong
d8792c21c5 IMPALA-1048: show sinks in exec summary
The exec summary now includes the total time taken and memory
consumed by the data sink at the root of each fragment. Previously
the exec summary could hide where time and memory went while
executing a query.

The high-level changes are:
* Generalising logic in the exec summary and runtime profile to
  handle data sinks, not just plan nodes, including adding richer
  metadata to runtime profile nodes.
* Threading through metadata about the data sinks, like names and
  estimates, so that it can appear in the exec summary.

The major potential downside is that the new timings reported for
data stream sender can overlap with the receiver's time and
potentially cause confusion.

[localhost:21000] default> select count(distinct l_comment) from tpch_parquet.lineitem; summary;
Query: select count(distinct l_comment) from tpch_parquet.lineitem
Query submitted at: 2018-11-20 16:47:03 (Coordinator: http://tarmstrong-box:25000)
Query progress can be monitored at: http://tarmstrong-box:25000/query_plan?query_id=f5464383a3bb6878:54b5252b00000000
+---------------------------+
| count(distinct l_comment) |
+---------------------------+
| 4580667                   |
+---------------------------+
Fetched 1 row(s) in 4.53s
+---------------------+--------+----------+----------+-------+------------+-----------+---------------+-----------------------+
| Operator            | #Hosts | Avg Time | Max Time | #Rows | Est. #Rows | Peak Mem  | Est. Peak Mem | Detail                |
+---------------------+--------+----------+----------+-------+------------+-----------+---------------+-----------------------+
| F02:ROOT            | 1      | 50.56us  | 50.56us  |       |            | 0 B       | 0 B           |                       |
| 06:AGGREGATE        | 1      | 275.89us | 275.89us | 1     | 1          | 16.00 KB  | 10.00 MB      | FINALIZE              |
| 05:EXCHANGE         | 1      | 49.08us  | 49.08us  | 3     | 1          | 32.00 KB  | 16.00 KB      | UNPARTITIONED         |
| F01:EXCHANGE SENDER | 3      | 100.06us | 113.49us |       |            | 16.00 KB  | 0 B           |                       |
| 02:AGGREGATE        | 3      | 19.32ms  | 19.57ms  | 3     | 1          | 16.00 KB  | 10.00 MB      |                       |
| 04:AGGREGATE        | 3      | 1.29s    | 1.43s    | 4.58M | 4.65M      | 98.02 MB  | 62.63 MB      |                       |
| 03:EXCHANGE         | 3      | 241.64ms | 246.54ms | 5.01M | 4.65M      | 9.05 MB   | 10.12 MB      | HASH(l_comment)       |
| F00:EXCHANGE SENDER | 3      | 2.43s    | 2.58s    |       |            | 337.53 KB | 0 B           |                       |
| 01:AGGREGATE        | 3      | 1.26s    | 1.46s    | 5.01M | 4.65M      | 97.20 MB  | 121.17 MB     | STREAMING             |
| 00:SCAN HDFS        | 3      | 39.87ms  | 41.36ms  | 6.00M | 6.00M      | 27.87 MB  | 80.00 MB      | tpch_parquet.lineitem |
+---------------------+--------+----------+----------+-------+------------+-----------+---------------+-----------------------+

Testing:
Added a basic observability test.

Change-Id: I3fdf7bacae8ff597b255da65af453e174ba53544
Reviewed-on: http://gerrit.cloudera.org:8080/11967
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2018-12-07 07:33:01 +00:00
Tim Armstrong
f46de21140 IMPALA-1760: Implement shutdown command
This is the same patch except with fixes for the test failures
on EC and S3 noted in the JIRA.

This allows graceful shutdown of executors and partially graceful
shutdown of coordinators (new operations fail, old operations can
continue).

Details:
* In order to allow future admin commands, this is implemented with
  function-like syntax and does not add any reserved words.
* ALL privilege is required on the server
* The coordinator impalad that the client is connected to can be shut
  down directly with ":shutdown()".
* Remote shutdown of another impalad is supported, e.g. with
  ":shutdown('hostname')", so that non-coordinators can be shut down
  and for the convenience of the client, which does not have to
  connect to the specific impalad. There is no assumption that the
  other impalad is registered in the statestore; just that the
  coordinator can connect to the other daemon's thrift endpoint.
  This simplifies things and allows shutdown in various important
  cases, e.g. statestore down.
* The shutdown time limit can be overridden to force a quicker or
  slower shutdown by specifying a deadline in seconds after the
  statement is executed.
* If shutting down, a banner is shown on the root debug page.

Workflow:
1. (if a coordinator) clients are prevented from submitting
  queries to this coordinator via some out-of-band mechanism,
  e.g. load balancer
2. the shutdown process is started via ":shutdown()"
3. a bit is set in the statestore and propagated to coordinators,
  which stop scheduling fragment instances on this daemon
  (if an executor).
4. the query startup grace period (which is ideally set to the AC
  queueing delay plus some additional leeway) expires
5. once the daemon is quiesced (i.e. no fragments, no registered
  queries), it shuts itself down.
6. If the daemon does not successfully quiesce (e.g. rogue clients,
  long-running queries), after a longer timeout (counted from the start
  of the shutdown process) it will shut down anyway.

What this does:
* Executors can be shut down without causing a service-wide outage
* Shutting down an executor will not disrupt any short-running queries
  and will wait for long-running queries up to a threshold.
* Coordinators can be shut down without query failures only if
  there is an out-of-band mechanism to prevent submission of more
  queries to the shut down coordinator. If queries are submitted to
  a coordinator after shutdown has started, they will fail.
* Long running queries or other issues (e.g. stuck fragments) will
  slow down but not prevent eventual shutdown.

Limitations:
* The startup grace period needs to be configured to be greater than
  the latency of statestore updates + scheduling + admission +
  coordinator startup. Otherwise a coordinator may send a
  fragment instance to the shutting down impalad. (We could
  automate this configuration as a follow-on)
* The startup grace period means a minimum latency for shutdown,
  even if the cluster is idle.
* We depend on the statestore detecting the process going down
  if queries are still running on that backend when the timeout
  expires. This may still be subject to existing problems,
  e.g. IMPALA-2990.

Tests:
* Added parser, analysis and authorization tests.
* End-to-end test of shutting down impalads.
* End-to-end test of shutting down then restarting an executor while
  queries are running.
* End-to-end test of shutting down a coordinator
  - New queries cannot be started on coord, existing queries continue to run
  - Exercises various Beeswax and HS2 operations.

Change-Id: I8f3679ef442745a60a0ab97c4e9eac437aef9463
Reviewed-on: http://gerrit.cloudera.org:8080/11484
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2018-09-26 01:28:36 +00:00
Tim Armstrong
16a04ce81b Revert "IMPALA-1760: Implement shutdown command"
This reverts commit fda44aed9d.

A couple of the tests broken on S3 and erasure coding. Reverting
to unblock testing until we can come up with a proper fix.

Change-Id: Icef47b3aa67bc056c40592d47e93c4ebc57be98c
Reviewed-on: http://gerrit.cloudera.org:8080/11435
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
2018-09-14 01:12:22 +00:00
Tim Armstrong
fda44aed9d IMPALA-1760: Implement shutdown command
This allows graceful shutdown of executors and partially graceful
shutdown of coordinators (new operations fail, old operations can
continue).

Details:
* In order to allow future admin commands, this is implemented with
  function-like syntax and does not add any reserved words.
* ALL privilege is required on the server
* The coordinator impalad that the client is connected to can be shut
  down directly with ":shutdown()".
* Remote shutdown of another impalad is supported, e.g. with
  ":shutdown('hostname')", so that non-coordinators can be shut down
  and for the convenience of the client, which does not have to
  connect to the specific impalad. There is no assumption that the
  other impalad is registered in the statestore; just that the
  coordinator can connect to the other daemon's thrift endpoint.
  This simplifies things and allows shutdown in various important
  cases, e.g. statestore down.
* The shutdown time limit can be overridden to force a quicker or
  slower shutdown by specifying a deadline in seconds after the
  statement is executed.
* If shutting down, a banner is shown on the root debug page.

Workflow:
1. (if a coordinator) clients are prevented from submitting
  queries to this coordinator via some out-of-band mechanism,
  e.g. load balancer
2. the shutdown process is started via ":shutdown()"
3. a bit is set in the statestore and propagated to coordinators,
  which stop scheduling fragment instances on this daemon
  (if an executor).
4. the query startup grace period (which is ideally set to the AC
  queueing delay plus some additional leeway) expires
5. once the daemon is quiesced (i.e. no fragments, no registered
  queries), it shuts itself down.
6. If the daemon does not successfully quiesce (e.g. rogue clients,
  long-running queries), after a longer timeout (counted from the start
  of the shutdown process) it will shut down anyway.

What this does:
* Executors can be shut down without causing a service-wide outage
* Shutting down an executor will not disrupt any short-running queries
  and will wait for long-running queries up to a threshold.
* Coordinators can be shut down without query failures only if
  there is an out-of-band mechanism to prevent submission of more
  queries to the shut down coordinator. If queries are submitted to
  a coordinator after shutdown has started, they will fail.
* Long running queries or other issues (e.g. stuck fragments) will
  slow down but not prevent eventual shutdown.

Limitations:
* The startup grace period needs to be configured to be greater than
  the latency of statestore updates + scheduling + admission +
  coordinator startup. Otherwise a coordinator may send a
  fragment instance to the shutting down impalad. (We could
  automate this configuration as a follow-on)
* The startup grace period means a minimum latency for shutdown,
  even if the cluster is idle.
* We depend on the statestore detecting the process going down
  if queries are still running on that backend when the timeout
  expires. This may still be subject to existing problems,
  e.g. IMPALA-2990.

Tests:
* Added parser, analysis and authorization tests.
* End-to-end test of shutting down impalads.
* End-to-end test of shutting down then restarting an executor while
  queries are running.
* End-to-end test of shutting down a coordinator
  - New queries cannot be started on coord, existing queries continue to run
  - Exercises various Beeswax and HS2 operations.

Change-Id: I4d5606ccfec84db4482c1e7f0f198103aad141a0
Reviewed-on: http://gerrit.cloudera.org:8080/10744
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
2018-09-11 23:57:20 +00:00
Todd Lipcon
1c6058fa16 IMPALA-7203. Support UDFs in CatalogdMetaProvider
This adds support to fetch the list of function names within a database,
and then to fetch the list of overloads for a given function name. These
items are cached on the coordinator and invalidated when minimal
function objects are seen on the minimal catalog topic stream.

Aside from the straight-forward plumbing of this new RPC, it's worth
noting that this patch changes the MetaProvider interface to provide
Impala Functions directly instead of HMS Function objects. This means
that we will now fully support all the types of functions supported in
legacy catalogs. As such, test_udfs now passes.

Making this change simplified the code in LocalDb but also means that
DirectMetaProvider no longer supports fetching functions. When we move
to trying to eliminate catalogd altogether at some point, we can revive
this code. For now, I just throw an exception for the function-related
code in DirectMetaProvider, as it's unused.

The one other notable thing here is that TFunction now has optional
fields where it used to have required ones. This was necessary in order
to send invalidations as TCatalogObjects. Given that TFunction is an
internal-only construct, this shouldn't raise compatibility issues.

Change-Id: Ifef8ece9f214dca9441833b00f65c7c152d0ab53
Reviewed-on: http://gerrit.cloudera.org:8080/11359
Reviewed-by: Todd Lipcon <todd@apache.org>
Tested-by: Todd Lipcon <todd@apache.org>
2018-09-06 17:27:07 +00:00
Vuk Ercegovac
2894884deb IMPALA-6670: refresh lib-cache entries from plan
When an impalad is in executor-only mode, it receives no
catalog updates. As a result, lib-cache entries are never
refreshed. A consequence is that udf queries can return
incorrect results or may not run due to resolution issues.
Both cases are caused by the executor using a stale copy
of the lib file. For incorrect results, an old version of
the method may be used. Resolution issues can come up if
a method is added to a lib file.

The solution in this change is to capture the coordinator's
view of the lib file's last modified time when planning.
This last modified time is then shipped with the plan to
executors. Executors must then use both the lib file path
and the last modified time as a key for the lib-cache.
If the coordinator's last modified time is more recent than
the executor's lib-cache entry, then the entry is refreshed.

Brief discussion of alternatives:

- lib-cache always checks last modified time
  + easy/local change to lib-cache
  - adds an fs lookup always. rejected for this reason

- keep the last modified time in the catalog
  - bound on staleness is too loose. consider the case where
    fn's f1, f2, f3 are created with last modified times of
    t1, t2, t3. treat the fn's last modified time as a low-watermark;
    if the cache entry has a more recent time, use it. Such a scheme
    would allow the version at t2 to persist. An old fn may keep the
    state from converging to the latest. This could end up with strange
    cases where different versions of the lib are used across executors
    for a single query.

    In contrast, the change in this path relies on the statestore to
    push versions forward at all coordinators, so will push all
    versions at all caches forward as well.

Testing:
- added an e2e custom cluster test

Change-Id: Icf740ea8c6a47e671427d30b4d139cb8507b7ff6
Reviewed-on: http://gerrit.cloudera.org:8080/9697
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Impala Public Jenkins
2018-03-24 04:38:53 +00:00
Tim Armstrong
852e1bb728 IMPALA-3931: arbitrary fixed-size uda intermediate types
Make many builtin aggregate functions use fixed-length intermediate
types:
* avg()
* ndv()
* stddev(), variance(), etc
* distinctpc(), distinctpcsa()

sample(), appx_median(), histogram() and group_concat() actually
allocate var-len data so aren't changed.

This has some major benefits:
* Spill-to-disk works properly with these aggregations.
* Aggregations are more efficient because there is one less pointer
  indirection.
* Aggregations use less memory, because we don't need an extra 12-byte
  StringValue for the indirection.

Adds a special-purpose internal type FIXED_UDA_INTERMEDIATE. The type
is represented in the same way as CHAR - a fixed-size array of bytes,
stored inline in tuples. However, it is not user-visible and does
not support CHAR semantics, i.e. users can't declare tables, functions,
etc with the type. The pointer and length is passed into aggregate functions
wrapped in a StringVal.

Updates some internal codegen functions to work better with the new
type. E.g. store values directly into the result tuple instead of
via an intermediate stack allocation.

Testing:
This change only affects builtin aggregate functions, for which we
have test coverage already. If we were to allow wider use of this type,
it would need further testing.

Added an analyzer test to ensure we can't use the type for UDAs.

Added a regression test for spilling avg().

Added a regression test for UDA with CHAR intermediate hitting DCHECK.

Perf:
Ran TPC-H locally. TPC-H Q17, which has a high-cardinality AVG(),
improved dramatically.

+----------+-----------------------+---------+------------+------------+----------------+
| Workload | File Format           | Avg (s) | Delta(Avg) | GeoMean(s) | Delta(GeoMean) |
+----------+-----------------------+---------+------------+------------+----------------+
| TPCH(60) | parquet / none / none | 18.44   | -17.54%    | 11.92      | -5.34%         |
+----------+-----------------------+---------+------------+------------+----------------+

+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+
| Workload | Query    | File Format           | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%) | Base StdDev(%) | Num Clients | Iters |
+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+
| TPCH(60) | TPCH-Q12 | parquet / none / none | 18.40  | 17.64       | +4.32%     |   0.77%   |   1.09%        | 1           | 5     |
| TPCH(60) | TPCH-Q22 | parquet / none / none | 7.07   | 6.90        | +2.36%     |   0.28%   |   0.30%        | 1           | 5     |
| TPCH(60) | TPCH-Q3  | parquet / none / none | 12.37  | 12.11       | +2.10%     |   0.18%   |   0.15%        | 1           | 5     |
| TPCH(60) | TPCH-Q7  | parquet / none / none | 42.48  | 42.09       | +0.93%     |   2.45%   |   0.80%        | 1           | 5     |
| TPCH(60) | TPCH-Q6  | parquet / none / none | 3.18   | 3.15        | +0.89%     |   0.67%   |   0.76%        | 1           | 5     |
| TPCH(60) | TPCH-Q19 | parquet / none / none | 7.24   | 7.20        | +0.50%     |   0.95%   |   0.67%        | 1           | 5     |
| TPCH(60) | TPCH-Q10 | parquet / none / none | 13.37  | 13.30       | +0.50%     |   0.48%   |   1.39%        | 1           | 5     |
| TPCH(60) | TPCH-Q5  | parquet / none / none | 7.47   | 7.44        | +0.36%     |   0.58%   |   0.54%        | 1           | 5     |
| TPCH(60) | TPCH-Q11 | parquet / none / none | 2.03   | 2.02        | +0.06%     |   0.26%   |   1.95%        | 1           | 5     |
| TPCH(60) | TPCH-Q4  | parquet / none / none | 5.48   | 5.50        | -0.27%     |   0.62%   |   1.12%        | 1           | 5     |
| TPCH(60) | TPCH-Q13 | parquet / none / none | 22.11  | 22.18       | -0.31%     |   0.18%   |   0.55%        | 1           | 5     |
| TPCH(60) | TPCH-Q15 | parquet / none / none | 8.45   | 8.48        | -0.32%     |   0.40%   |   0.47%        | 1           | 5     |
| TPCH(60) | TPCH-Q9  | parquet / none / none | 33.39  | 33.66       | -0.81%     |   0.75%   |   0.59%        | 1           | 5     |
| TPCH(60) | TPCH-Q21 | parquet / none / none | 71.34  | 72.07       | -1.01%     |   1.84%   |   1.79%        | 1           | 5     |
| TPCH(60) | TPCH-Q14 | parquet / none / none | 5.93   | 6.00        | -1.07%     |   0.15%   |   0.69%        | 1           | 5     |
| TPCH(60) | TPCH-Q20 | parquet / none / none | 5.72   | 5.79        | -1.09%     |   0.59%   |   0.51%        | 1           | 5     |
| TPCH(60) | TPCH-Q18 | parquet / none / none | 45.42  | 45.93       | -1.10%     |   1.42%   |   0.50%        | 1           | 5     |
| TPCH(60) | TPCH-Q2  | parquet / none / none | 4.81   | 4.89        | -1.52%     |   1.68%   |   1.01%        | 1           | 5     |
| TPCH(60) | TPCH-Q16 | parquet / none / none | 5.41   | 5.52        | -1.98%     |   0.66%   |   0.73%        | 1           | 5     |
| TPCH(60) | TPCH-Q1  | parquet / none / none | 27.58  | 29.13       | -5.34%     |   0.24%   |   1.51%        | 1           | 5     |
| TPCH(60) | TPCH-Q8  | parquet / none / none | 12.61  | 14.30       | -11.78%    |   6.20%   | * 15.28% *     | 1           | 5     |
| TPCH(60) | TPCH-Q17 | parquet / none / none | 43.74  | 126.58      | I -65.44%  |   1.34%   |   9.60%        | 1           | 5     |
+----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------------+-------+

Change-Id: Ife90cf27989f98ffb5ef5c39f1e09ce92e8cb87c
Reviewed-on: http://gerrit.cloudera.org:8080/7526
Tested-by: Impala Public Jenkins
Reviewed-by: Tim Armstrong <tarmstrong@cloudera.com>
2017-08-17 03:12:48 +00:00
Michael Ho
b38d9826d7 IMPALA-4192: Disentangle Expr and ExprContext
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
2017-06-18 11:08:25 +00:00
Marcel Kornacker
a9b9933b5f IMPALA-3902: Scheduler improvements for running multiple fragment
instances on a single backend

This is an extension of the scheduler and coordinator for multi-threaded
execution. It mainly removes the assumption of having one instance per
fragment per host. The approach taken here is to create parallel data
structures and control flow functions, where necessary, and otherwise to
leave the existing single-instance logic in place. The parallel
structures' and functions' names are prefixed with "Mt" to facilitate
the enventual clean-up.
Not much of an attempt was made to factor out common functionality
between the Mt- and the single-threaded version, because the
single-threaded version will disappear in a follow-on patch and
refactoring the existing code to fit into two parallel functions from
which it's being called might end up obscuring the code more than helping
it. Also, this code is relatively stable and having two parallel paths won't
cause much extra work (in terms of having to apply the same changes/fixes
twice) in the medium term.

Changes to data structures:
- QuerySchedule: per-instance and per-fragment structs with complete
  execution parameters (instead of partially relying on TQueryExecRequest);
  the per-instance execution parameter struct is a child of the per-fragment
  parameter struct
- explicit fragment id, with range 0..#fragments-1 (instead of relying on an
  index into an array in TQueryExecRequest)

Excluded:
- runtime filter handling
- anything related to RM

Change-Id: I240445199e22f009f4e72fdb8754eb8d77e3d680
Reviewed-on: http://gerrit.cloudera.org:8080/4054
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
Tested-by: Internal Jenkins
2016-10-06 00:20:36 +00:00
Thomas Tauber-Marshall
b2c2fe7813 IMPALA-3786: Replace "cloudera" with "apache" (part 2)
As part of the ASF transition, we need to replace references to
Cloudera in Impala with references to Apache. This primarily means
changing Java package names from com.cloudera.impala.* to
org.apache.impala.*

A prior patch renamed all the files as necessary, and this patch
performs the actual code changes. Most of the changes in this patch
were generated with some commands of the form:

find . | grep "\.java\|\.py\|\.h\|\.cc" | \
  xargs sed -i s/'com\(.\)cloudera\(\.\)impala/org\1apache\2impala/g

along with some manual fixes.

After this patch, the remaining references to Cloudera in the repo
mostly fall into the categories:
- External components that have cloudera in their own package names,
  eg. com.cloudera.kudu/llama
- URLs, eg. https://repository.cloudera.com/

Change-Id: I0d35fa6602a7fc0c212b2ef5e2b3322b77dde7e2
Reviewed-on: http://gerrit.cloudera.org:8080/3937
Reviewed-by: Thomas Tauber-Marshall <tmarshall@cloudera.com>
Reviewed-by: Jim Apple <jbapple@cloudera.com>
Tested-by: Internal Jenkins
2016-09-29 21:14:13 +00:00
Dan Hecht
ffa7829b70 IMPALA-3918: Remove Cloudera copyrights and add ASF license header
For files that have a Cloudera copyright (and no other copyright
notice), make changes to follow the ASF source file header policy here:

http://www.apache.org/legal/src-headers.html#headers

Specifically:
1) Remove the Cloudera copyright.
2) Modify NOTICE.txt according to
   http://www.apache.org/legal/src-headers.html#notice
   to follow that format and add a line for Cloudera.
3) Replace or add the existing ASF license text with the one given
   on the website.

Much of this change was automatically generated via:

git grep -li 'Copyright.*Cloudera' > modified_files.txt
cat modified_files.txt | xargs perl -n -i -e 'print unless m#Copyright.*Cloudera#i;'
cat modified_files_txt | xargs fix_apache_license.py [1]

Some manual fixups were performed following those steps, especially when
license text was completely missing from the file.

[1] https://gist.github.com/anonymous/ff71292094362fc5c594 with minor
    modification to ORIG_LICENSE to match Impala's license text.

Change-Id: I2e0bd8420945b953e1b806041bea4d72a3943d86
Reviewed-on: http://gerrit.cloudera.org:8080/3779
Reviewed-by: Dan Hecht <dhecht@cloudera.com>
Tested-by: Internal Jenkins
2016-08-09 08:19:41 +00:00
Michael Ho
a59408b575 IMPALA-3286: Prefetching for PHJ probing.
This change pipelines the code which probes the hash tables.
This is based on the idea which Mostafa presented earlier.
Essentially, all rows in a row batch will be evaluated and
hashed first before being probed against the hash tables.
Hash table buckets are prefetched as hash values of rows are
computed.

To avoid re-evaluating the rows again during probing (as the rows
have been evaluated once to compute the hash values), hash table
context has been updated to cache the evaluated expression values,
null bits and hash values of some number of rows. Hash table context
provies a new iterator like interface to iterate through the cached
values.

A PREFETCH_MODE query option has also been added to disable prefetching
if necessary. The default mode is 1 which means hash table buckets will
be prefetched. In the future, this mode may be extended to support hash
table buckets' data prefetching too.

Combined with the build side prefetching, a self join of table lineitem
improves by 40% on a single node run on average:

select count(*)
from lineitem o1, lineitem o2
where o1.l_orderkey = o2.l_orderkey and
      o1.l_linenumber = o2.l_linenumber;

Change-Id: Ib42b93d99d09c833571e39d20d58c11ef73f3cc0
Reviewed-on: http://gerrit.cloudera.org:8080/2959
Reviewed-by: Michael Ho <kwho@cloudera.com>
Tested-by: Internal Jenkins
2016-05-17 01:30:12 -07:00
Marcel Kornacker
3b7d5b7c17 MT: Planner for multi-threaded execution
New classes:
- ParallelPlanner: creates build plans, assigns plans to cohorts
- JoinBuildSink: DataSink for plan fragments that materialize build sides
- ids for plans, hash tables, plan fragments

Tests: this adds a new test file section PARALLELPLANS and augments the tpc-h/-ds tests with
those sections.

In the interest of keeping this patch small I didn't augment other test files with that
section yet (which will happen at a later date, to cover more corner cases).

Change-Id: Ic3c34dd3f9190a131e6f03d901b4bfcd164a5174
Reviewed-on: http://gerrit.cloudera.org:8080/2846
Tested-by: Internal Jenkins
Reviewed-by: Marcel Kornacker <marcel@cloudera.com>
2016-05-12 14:17:56 -07:00
Alex Behm
049ede9f62 OPSAPS-32457: Fix CatalogService Thrift changes to be backwards compatible.
While adding support for permanent UDFs we made incompatible changes to the
CatalogService Thrift definitions. Some services like BDR rely on a stable
catalog API.

This patch fixes the incompatibility.

Change-Id: Iec04d07c48d7159d2837667d7039046de126a3ad
Reviewed-on: http://gerrit.cloudera.org:8080/2455
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: Internal Jenkins
2016-03-06 15:23:39 -08:00
Bharath Vissapragada
ef0dac661c IMPALA-2843: Persist hive udfs across catalog restarts
This commit adds a new feature to persist hive/java udfs across
catalog restarts. IMPALA-1748 already added this for non-java
udfs by storing them in parameters map of the Db object and
reading them back at catalog startup. However we follow a
different approach for hive udfs by converting them to Hive's
function format and adding them as hive functions to the metastore.
This makes it possible to share udfs between hive and Impala as the
udfs added from one service are accessible to other. This commit
takes care of format conversions between hive and impala and user
can just add function once in either of the services.

Background: Hive and impala treat udfs differently. Hive resolves the
evaluate function in the udf class at runtime depending on the data
types of the input arguments. So user can add one function by name and
can pass any arguments to it as long as there is a compatible evaluate
function in the udf class. However Impala takes the input types of the
udf as a part of function definition (that maps to only one evaluate
function) and loads the function only for those set of input argument
types. If we have multiple 'evaluate' methods, we need to add multiple
functions one for each of them.

This commit adds new variants of CREATE | DROP FUNCTIONS  to Impala which
lets the user to create and drop hive/java udfs without input argument
types or return types. Catalog takes care of loading/dropping the udf
signatures corresponding to each "evaluate" method in the udf symbol
class. The syntax is as follows,

CREATE FUNCTION [IF NOT EXISTS] <function name> <function_opts>
DROP FUNCTION [IF EXISTS] <function name>

Examples:

CREATE FUNCTION IF NOT EXISTS foo location '/path/to/jar' SYMBOL='TestUdf';
CREATE FUNCTION bar location '/path/to/jar' SYMBOL='TestUdf2';
DROP FUNCTION foo;
DROP FUNCTION IF EXISTS bar;

The older way of creating hive/java udfs with specific signature is still supported,
however they are *not* persisted across restarts. So a restart of catalog can
wipe them out. Additionally this commit also loads all the compatible java udfs
added outside of Impala and they needn't be separately loaded. One thing
to note here is that the functions added using the new CREATE FUNCTION
can only be dropped using the new DROP FUNCTION syntax (without
signature). The same rule applies for the java udfs added using the old
CREATE FUNCTION syntax (with signature).

Change-Id: If31ed3d5ac4192e3bc2d57610a9a0bbe1f62b42d
Reviewed-on: http://gerrit.cloudera.org:8080/2250
Reviewed-by: Bharath Vissapragada <bharathv@cloudera.com>
Tested-by: Internal Jenkins
2016-02-19 23:04:03 -08:00
Henry Robinson
2cc586372c IMPALA-3001 / 3008: Runtime filter polish
This commit polishes a few parts of the runtime filter feature:

1. ENABLE_RUNTIME_FILTER_PROPAGATION has been replaced with
RUNTIME_FILTER_MODE which takes as values LOCAL, GLOBAL and OFF.

2. The filter routing table is only printed in mode GLOBAL.

3. The filter routing table is now printed with TablePrinter, and
includes whether a filter is broadcast, or if it's a partition-only
filter.

4. Parquet per-row filtering can be disabled using
DISABLE_PARQUET_ROW_FILTERING.

5. De-serialisation of the Thrift Bloom filter is moved out of the
spinlock scope in Coordinator::UpdateFilter().

Change-Id: I9257aa079c3793c1c4b3e2be51e25fc207298c32
Reviewed-on: http://gerrit.cloudera.org:8080/2194
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Henry Robinson <henry@cloudera.com>
2016-02-19 23:04:02 -08:00
Alex Behm
de75278125 Add SHOW ANALYTIC FUNCTIONS and additional analysis checks.
Change-Id: Ic1aac60fb9b094349b9cfbec68608ac50fc5660c
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/4298
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: jenkins
2014-09-13 00:19:21 -07:00
Matthew Jacobs
5bf1c1f223 Analytic Functions: Add rank() and dense_rank()
Adds the rank() and dense_rank() analytic functions and makes internal
changes to the AggFnEvaluator that are necessary to support calling
Finalize() repeatedly (as the AnalyticEvalNode does) on UDAs that destroy
state in Finalize().

Rank requires both the current rank and the count at that rank in order to
determine the next rank, so the intermediate state is a StringVal containing
a struct with these two fields.

Aggregate functions (internally only, for now) can expose a GetValue() method
which takes an intermediate value and returns a final value without destroying
the intermediate state. Finalize() is then used to clean up intermediate
state, if necessary.

This also adds a second optional, internal-only function for UDAs to allow
removing values from intermediate state: Remove(). This will be required for
implementing sliding windows later but is added here because the change is
nearly identical to that for adding GetValue().

Some cleanup in the AnalyticEvalNode, most notably we avoid allocating tuples
to DeepCopy prev_input_row_ between input batches. Instead, we keep the last
two child row batches because the prev child row batch owns the resources for
prev_input_row_.

Change-Id: I5a30eb517a38d369fe63f7af91904a4b9786fadc
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/3962
Reviewed-by: Matthew Jacobs <mj@cloudera.com>
Tested-by: jenkins
(cherry picked from commit 137bb45d81ea57655aefbf5cde0cbeab0121b8f0)
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/4183
2014-09-05 02:15:42 -07:00
Victor Bittorf
2dce31f6c2 Adding VARCHAR front & backend.
VARCHAR is treated as StringVal in the backend. All UDAs and UDFs which accept STRING
will also accept VARCHAR(N).

TODO: Reverted Avro codegen to fix Jenkins; needs separate patch.

Change-Id: Ifc120b6f0fe1f996b11a48b134d339ad3719331e
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/2527
Reviewed-by: Victor Bittorf <victor.bittorf@cloudera.com>
Tested-by: jenkins
(cherry picked from commit 3fcbf4f677b8e26c37eded4d8bb628e6fc53c1e9)
Reviewed-on: http://gerrit.sjc.cloudera.com:8080/4058
2014-08-27 13:52:58 -07:00
Dan Hecht
1fee56cb26 IMPALA-1080: Implement "SET <query_option>" as SQL statement.
Also add support for "SET", which returns a table of query options and
their respective values.

The front-end parses the option into a (key, value) pair and then the
existing backend logic is used to set the option, or return the result
sets.

Change-Id: I40dbd98537e2a73bdd5b27d8b2575a2fe6f8295b
Reviewed-on: http://gerrit.ent.cloudera.com:8080/3582
Reviewed-by: Daniel Hecht <dhecht@cloudera.com>
Tested-by: jenkins
(cherry picked from commit aa0f6a2fc1d3fe21f22cc7bc56887e1fdb02250b)
Reviewed-on: http://gerrit.ent.cloudera.com:8080/3614
2014-07-25 10:25:09 -07:00
Alex Behm
e9864d5f78 Introduce type hierarchy and add complex types.
This patch replaces ColumnType with a hierarchy of types that models
the existing scalar types as well as the new complex types ARRAY, MAP,
and STRUCT.

Change-Id: Ia895f41153e99febb0c35412acac12689c3c2064
Reviewed-on: http://gerrit.ent.cloudera.com:8080/3491
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: jenkins
Reviewed-on: http://gerrit.ent.cloudera.com:8080/3538
2014-07-21 20:00:46 -07:00
Skye Wanderman-Milne
6ceed1e632 UDF API additions
This patch introduces the ability to specify a prepare and close
function for a UDF, as well as FunctionContext methods for maintaining
state across UDF invocations within a query. Many of the changes are
related to adding an Expr::Open() function which calls the UDF's
prepare function, if specified (it has to be called in Open() since
the LLVM module must be compiled first).

Change-Id: I581d90d03dff71f7ff5d4a6bef839ba6bc46b443
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1693
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: jenkins
(cherry picked from commit 8e2ed7fb9051d98f89327715fdebd6f5ed22d6ee)
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1757
2014-03-05 07:32:34 -08:00
Nong Li
f0a67153d3 Decimal analysis changes.
Change-Id: Ib7d6a6a7650cc9058ff1486fc7546ab66c698d46
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1734
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: jenkins
2014-03-03 21:15:00 -08:00
Nong Li
0d2919fe7f Refactor scalar and aggregate function analysis and execution.
This patch cleans up analysis and execution of scalar and aggregate functions
so that there is no difference between how builtins and user functions are
handled. The only difference is that the catalog is populated with the builtins
all the time.

The BE always gets a TFunction object and just executes it (builtins will have
an empty hdfs file location).

This removes the opcode registry and all of the functionality is subsumed by
the catalog, most of which was already duplicated there anyway.

This also introduces the concept of a system database; databases that the
user cannot modify and is populated automatically on startup.

Change-Id: Iaa3f84dad0a1a57691f5c7d8df7305faf01d70ed
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1386
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: jenkins
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1577
2014-02-18 18:40:08 -08:00
Nong Li
69fe1c6c10 Change FE to use ColumnType instead of PrimitiveType.
PrimitiveType is an enum and cannot be used for more complex types. The change
touches a lot of files but very mechanically.

A similar change needs to be done in the BE which will be a subsequent patch.

The version as I have it breaks rolling upgrade due to the thrift changes. If
this is not okay, we can work around that but it will be annoying.

Change-Id: If3838bb27377bfc436afd6d90a327de2ead0af54
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1287
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: jenkins
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1304
Reviewed-by: Henry Robinson <henry@cloudera.com>
Tested-by: Henry Robinson <henry@cloudera.com>
2014-01-17 14:32:55 -08:00
Alex Behm
6799c93922 Simplified/enhanced explain plans with a total of four explain levels.
There are now 4 explain levels summarized as follows:
- Level 0: MINIMAL
  Non-fragmented parallel plan only showing plan nodes with minimal attributes
- Level 1: STANDARD
  Non-fragmented parallel plan with some details in plan nodes
- Level 2: EXTENDED
  Non-fragmented parallel plan with full details in plan nodes including
  the table/column stats, row size, #hosts, cardinality,
  and estimated per-host memory requirement
- Level 3: VERBOSE
  Fragmented parallel plan with full details (like level 2)

This patch also includes several bugfixes related to plan costing and/or
testing of explain plans.

Change-Id: I622310f01d1b3d53ea1031adaf3b3ffdd94eba30
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1211
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
Tested-by: jenkins
2014-01-10 19:17:59 -08:00
Skye Wanderman-Milne
acdc792355 IMPALA-695: Use the local path of Hive UDF jars in the FE.
The FE was creating class loaders with the HDFS locations of Hive UDF
libs, rather than the local locations created by the BE. Our tests
still passed since we only used UDFs already on the classpath
(e.g. Hive builtins).

Change-Id: Idbe9c98ad6adb84b70cb44efbf9ad0afc53366ca
Reviewed-on: http://gerrit.ent.cloudera.com:8080/1081
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
Tested-by: jenkins
2014-01-08 10:54:25 -08:00
Nong Li
601f24a198 UDA execution loose ends.
Unfortunately, the BE does not have the codegen path to execute UDAs.
This puts some restrictions on the UDAs we can run.

- No IR UDAs
- No varargs
- Must have 8 arguments or less.

The code to do this is almost all there for UDFs but I'm not sure I'll get to it.

Change-Id: I8a06e635a9138397c8474a5704c3e588bb92347b
Reviewed-on: http://gerrit.ent.cloudera.com:8080/703
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: Nong Li <nong@cloudera.com>
2014-01-08 10:53:38 -08:00
Nong Li
e959e49b7c Update opcode registry to support UDF-interface builtins.
There's a bigger change to migrate the rest of them but I think this is how
the builtins, when not running as cross compiled, should be run. This mode
is still useful when developing the builtin.

When run as cross compiled IR, we wouldn't do anything to distinguish between
a builtin and an external UDF.

Change-Id: I6aa336b22aa19b00507bad33c9df3978baa576cc
Reviewed-on: http://gerrit.ent.cloudera.com:8080/542
Tested-by: jenkins
Reviewed-by: Skye Wanderman-Milne <skye@cloudera.com>
2014-01-08 10:53:06 -08:00
Nong Li
e39de94316 Add parser/analysis to support UDAs.
I looked around some and I think having create/drop/show [aggregate] function
seems reasonable and extends nicely for UDTs.

The create aggregate function can accept a lot of arguments. The non-essential one, I
went with resolving them by name rather than position (i.e. argName="value"). I think
this is better for the user than specifying it by position.

The grammar is:
CREATE AGGREGATE <name>(<arg_types>) RETURNS <type> [INTERMEDIATE <type>]
LOCATION '/path' UpdateFn='Fn' [comment='comment']
[SerializeFn='symbol'] [MergeFn='symbol'] [InitFn='symbol'] [FinalizeFn='symbol']

The optional args at the end can be in any order. If the other symbols are not
specified, we derive them from the UpdateFn symbol that's required. The analyzer
would try to figure it out and fail if we can't find the derived symbol in the binary.

The simplest example would be:
CREATE AGGREGATE FUNCTION count(float) RETURNS BIGINT LOCATION '/path'
UpdateFn='CountUpdateFn';

In which case we assume the intermediate type is the return type and the other functions
are called 'CountInitFn', 'CountSerializeFn', 'CountMergeFn' 'CountFinalizeFn'.

Change-Id: Iefc5741293050f5b295df28e9d1a7d039ead8675
Reviewed-on: http://gerrit.ent.cloudera.com:8080/513
Reviewed-by: Nong Li <nong@cloudera.com>
Tested-by: Nong Li <nong@cloudera.com>
2014-01-08 10:52:59 -08:00
Nong Li
a0bf45a0b4 Add udf type.
Change-Id: Ic5f52c127750cc9c847a3e34d3fdcfc78bee5a8a
Reviewed-on: http://gerrit.ent.cloudera.com:8080/454
Tested-by: jenkins
Reviewed-by: Alex Behm <alex.behm@cloudera.com>
2014-01-08 10:52:48 -08:00
Lenni Kuff
c2cfc7e2a3 IMPALA-373: Add support for 'LOAD DATA' statements
This change adds Impala support for LOAD DATA statements. This allows the user
to load one or more files into a table or partition from a given HDFS location. The
load operation only moves files, it does not convert data to match the target
table/partition's file format.
2014-01-08 10:51:02 -08:00
Alan Choi
b1de018298 IMPALA-31 Support EXPLAIN <query>
Hue is moving to HiveServer2 but HiveServer2 does not have an "explain" RPC
call. To support "explain", I added it to the language.

An "explain" statement will return a result set: one row per explain line.
2014-01-08 10:50:32 -08:00
Alex Behm
1b2e8280d4 Fix NULL issues. 2014-01-08 10:49:32 -08:00