You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@impala.apache.org by "Attila Jeges (Code Review)" <ge...@cloudera.org> on 2019/02/14 16:07:47 UTC

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Attila Jeges has uploaded this change for review. ( http://gerrit.cloudera.org:8080/12481


Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. DATE values do not have a time of day component. The range
of values supported for the DATE type is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from BIGINT, INT, SMALLINT, TINYINT to DATE. The source value is
      interpreted as a number of days since the epoch.
    - from DATE to BIGINT, INT, SMALLINT, TINYINT. The resulting
      integer is the number of days since epoch.
    - from DOUBLE, DECIMAL to DATE. The source value's fractional part
      is ignored, the integer part is interpreted as a number of days
      since epoch.
    - from DATE to DOUBLE, DECIMAL. The resulting value is the number
      of days since epoch.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions.
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/decimal-operators-ir.cc
M be/src/exprs/decimal-operators.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
141 files changed, 2,891 insertions(+), 205 deletions(-)



  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/1
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 1
Gerrit-Owner: Attila Jeges <at...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   public boolean compare(Function other, CompareMode mode) {
> Thanks for putting this together!
I checked this and Attila's points make sense to me.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Sat, 20 Apr 2019 00:26:00 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(6 comments)

I had some high-level questions/comments.

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@11
PS3, Line 11: of values supported for the DATE type is 0000-01-01 to 9999-12-31.
Can you mention the literal syntax supported for date values?


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@35
PS3, Line 35: - Implicit casting between DATE and other types:
Do you plan to support the DATE 'yyyy-mm-dd' literal syntax? It looks like Hive does. I wonder if that would make it easier to write queries that depend less on implicit casting.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@40
PS3, Line 40: - Since both STRING -> DATE and STRING -> TIMESTAMP implicit
Do you have an example of where this is required?


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@63
PS3, Line 63:       tests/query_test/test_date_queries.py.
Can we add a simple test that runs a query returning date through impala-shell, just to make sure that the client-side handling of DATE there works.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@64
PS3, Line 64: 
I think we should plan on adding DATE support to the random query generator (tests/comparison), otherwise we'd be leaving an arbitrary gap in the support there. I believe the semantics of DATE should be the same for all databases so hopefully that is relatively straightforward.

The coverage you have looks quite complete, and we should make efforts to understand if there are any gaps, but the extra line of defense is useful.

It's also helpful if you add more standard date functions later on.


http://gerrit.cloudera.org:8080/#/c/12481/3/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/3/tests/query_test/test_date_queries.py@18
PS3, Line 18: decimal
date



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 Feb 2019 19:45:24 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@70
PS11, Line 70: complete DATE type implementation
> should we consider turning it off by default and only enabling it once we'r
It seems easier to leave it enabled so long as the current support is correct. I think we should convince ourselves that the testing is adequate before merging this though.


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@72
PS11, Line 72: - Add date support to the random query generator.
> should we transition TPC-DS test tables over to 'date' instead of 'string' 
Seems reasonable to do once the support is complete, I don't think it's a requirement to call DATE support "complete" though.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc
File be/src/exprs/aggregate-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc@381
PS11, Line 381: val_struct->sum / val_struct->count
> Is this correct behavior when the dates are negative? Dividing as signed in
I agree avg(date) is likely not important. Kind-of nice to have for consistency but I guess if there's any concerns about the semantics we can just avoid the issues by not having it.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Mon, 18 Mar 2019 16:13:36 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

> (1 comment)
 > 
 > Also, is the HBase data representation just a string? Is it
 > compatible with Hive's representation.

I've done some simple ad-hoc testing with HBASE DATE tables, seems to be working fine. The HBASE date representation is string-based and compatible with Hive.

On second thought (since we will have an upstream release soon), maybe it makes sense to spend couple more days on adding HBASE tests to this change and then commit everything in one patch. Splitting it into two parts would further complicate things.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 17:29:25 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#12). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
159 files changed, 3,945 insertions(+), 354 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/12
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 12
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 16: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc
File be/src/exprs/conditional-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc@37
PS11, Line 37: IS_NULL_COMPUTE_
> Done
Sorry for the churn here induced by my original comment. I do think there's something worth revisiting later just as a maintainability tihng - there would be some value in infrastructure to "stamp out" these more mechanical aspects of a data type, but it requires some thought. E.g. for something like a numeric type, where there's a set of common operations and functions, it seems like we should be able to define a new numeric type in one place and have the required definitions added automatically elsewhere, rather than having to chase down all the locations in the codebase.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 16
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 02 Apr 2019 17:49:56 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#17). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncted silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,229 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/17
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 17
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#10). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
A be/src/util/macro-util.h
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
162 files changed, 3,975 insertions(+), 397 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/10
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 10
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 6:

> (2 comments)
 > 
 > Do you mean that, with these code changes, you can scan HBase date
 > tables? Or that we should have negative tests for HBase?
 > 
 > I think we should have test coverage but I'm open to splitting that
 > out into a separate patch if that helps this get in faster.

Yes, with these code changes Impala can scan/write HBase DATE tables.

At the minimum, DATE-specific tests should be added to
hbase-inserts.test,
hbase-scan-node.test,
hbase-compute-stats-incremental.test and
hbase-compute-stats.test.

This would also involve adding extra HBASE tables to functional_hbase.

I'd rather do this in a separate change.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 6
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 16:01:56 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#15). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
160 files changed, 3,985 insertions(+), 371 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/15
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 15
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 17:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2651/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 17
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 04 Apr 2019 20:14:01 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 23: Code-Review+2

Todd is out right now, but I took a look at the changes and believe you addressed his last round of feedback. Let's get this merged and we can do any fixups later.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Sat, 20 Apr 2019 00:24:01 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#16). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
159 files changed, 3,982 insertions(+), 368 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/16
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 16
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 14:

(5 comments)

Looking pretty close to good. I just have some questions digging into the function resolution stuff. I don't have a really specific example of something I think is wrong, but I want to make sure we're pretty clear on what our function resolution policy is and that we think it won't have any unexpected user behavior. I'll spend some more time on that tomorrow

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@52
PS14, Line 52:   public List<PartitionKeyValue> getPartitionSpecKeyValues() { return partitionSpec_; }
Do we ever expect this to get called prior to analysis?

Maybe it makes sense to add a bool analyzed_ and Preconditions.checkState(analyzed_); here to ensure that we don't ever return the pre-analysis KVs? This makes sure that we don't expose out the partitionSpec_ before it becomes immutable.


http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@156
PS14, Line 156:   public List<TPartitionKeyValue> toThrift() {
same (Precondition on analysis if possible)


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   public boolean compare(Function other, CompareMode mode) {
> I took a quick look at the SQL ANSI standard, but couldn't find anything ab
hm, I spent some time researching this across multiple databases:

DB2's logic is very clearly described: https://www.ibm.com/support/knowledgecenter/en/SSEPEK_11.0.0/sqlref/src/tpc/db2z_determinbestfitfunction.html
(it seems to go left-to-right)

Postgres's is also pretty well described and seems relatively similar to what's done here:

https://www.postgresql.org/docs/10/typeconv-func.html

Hive's also seems to bail earlier with an "ambiguous" error when doing method resolution for UDFs: "Closest match is defined as the one that requires the least number of arguments to be converted. In case more than one matches are found, the method throws an ambiguous method exception"


Comparing Postgres to what's implemented here, it seems like our two gaps are:
1) they have a well-defined concept of "preferred type". I checked the pg catalog and for the 'date/time' category, timestamptz is the 'preferred type': 

=> select oid,typname, typispreferred from pg_type where typcategory = 'D';
  oid  |   typname   | typispreferred 
-------+-------------+----------------
   702 | abstime     | f
  1082 | date        | f
  1083 | time        | f
  1114 | timestamp   | f
  1184 | timestamptz | t
  1266 | timetz      | f
 12401 | time_stamp  | f
(7 rows)

It seems that, because it only allows implicit conversion to "preferred" types, it's somewhat more strict than what we do today, though.

2) they have a concept of 'unknown' type to represent string literals in queries as being distinct from an expression with a defined type (eg any other expression than a string literal). I wonder if we should be treating StringLiteral casts differently than other exprs?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@236
PS11, Line 236:     // Check trailing varargs.
I think we could simplify this code a lot by adding some utility function like:

Type[] tryExtendArgsToLength(int numArgs) {
  if (!hasVarArgs_ || argTypes_.length <= numArgs) return argTypes_;
  Type[] ret = Arrays.copyOf(argTypes_, numArgs);
  for (int i = argTypes_.length; i < numArgs; i++) {
    ret[i] = getVarArgsType();
  }
  return ret;
}

then in these comparison functions, we can extend the varargs to match the user-provided call types and do the rest of the logic on fixed-length argument lists, without having to duplicate code between L227-234 and L239-247.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java
File fe/src/main/java/org/apache/impala/util/FunctionUtils.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java@243
PS11, Line 243:     return max_func;
it's interesting that our function resolution returns the first fit, from the order returned by the catalog. Best I can tell, we never will fail with an "ambiguous resolution" error. Now that we have both DATE and TIMESTAMP (and assumedly we'll add DATETIME at some point?) is there really no case that we want to force an explicit match by failing the user query when ambiguous?



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 14
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 26 Mar 2019 00:21:02 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#7). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
A be/src/util/macro-util.h
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
A testdata/workloads/functional-query/queries/QueryTest/date_text_only_support.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
153 files changed, 3,534 insertions(+), 395 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/7
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 7
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 1:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2124/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 1
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 Feb 2019 16:50:30 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 24:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/4052/ DRY_RUN=false


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 24
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 23 Apr 2019 08:16:41 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 17:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/17/be/src/runtime/date-test.cc
File be/src/runtime/date-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/17/be/src/runtime/date-test.cc@61
PS17, Line 61:   const DateValue v4 = ParseValidateDate("1990-10-20 23:59:59.999999999", true, 1990, 10, 20);
line too long (94 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 17
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 04 Apr 2019 19:49:22 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 19: Code-Review+1


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 19
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Mon, 08 Apr 2019 22:00:20 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#9). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
A be/src/util/macro-util.h
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
A testdata/workloads/functional-query/queries/QueryTest/date_text_only_support.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
153 files changed, 3,533 insertions(+), 395 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/9
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 8:

I've also realized that besides TEXT, DATE should work for HBASE as well. Do you think, I should add HBASE tests to this patch-set, or it should be a separate change?


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 8
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 Mar 2019 16:42:18 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 17:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@20
PS11, Line 20: - Explicit casting between DATE and other types:
             :     - from STRING
> This seems to be stricter than CAST() in postgres, mysql, and hive (see htt
Fixed it to allow and silently truncate the time component.


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@38
PS11, Line 38: > DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
             :   implicit conversions are now all possible, the existing function
             :   overload resolution logic is not adequate anymore.
             :   For example, it resolves the
> Should we be emitting a WARNING for any out-of-range values? it seems surpr
I've added warnings to indicate that DATE->TIMESTAMP, STRING->DATE, TIMESTAMP->DATE conversions failed.

Adding warning to STRING->TIMESTAMP conversions needs more work. Since these conversions are not related to the DATE type I'd prefer to fix them separately.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   // Compares this to 'other' for 'mode'.
> I'm afraid about introducing new semantics here which we'll need to break a
Thanks for putting this together!

- Impala already works as Hive 3.1 in rows 8-11, 13-14, 17, 20 and 22.

- I've changed STRING->DATE conversions to accept (and silently truncate) the time component. This fixes rows 1-7, 12 and 21.

- Row 15, 16 and 19 happens because Impala and PostgreSQL convert STRING, DATE, TIMESTAMP parameters to TIMESTAMP, while Hive converts them to STRING. Fixing these might not be possible w/o breaking backward compatibility.

- Row 18 returns an error in Hive 2.1, and probably fails in Hive 3.1 as well (haven't tried it though). Impala and PostgreSQL convert DATE and TIMESTAMP parameters to TIMESTAMP.

- Impala's behavior in row 23 is expected as valid timestamps in Impala start with year 1400. Adding YEAR(DATE) won't fix this problem either as we will still have to resolve YEAR('0009-02-15') to YEAR(TIMESTAMP) instead of YEAR(DATE) not to break backward compatibility. Users will have to be more implicit to avoid this issue and call YEAR(DATE '0009-02-15'). The best we can do is to issue a warning (which I haven't done here as it requires quite bit of work and technically it is a TIMESTAMP issue not a DATE issue. I'd prefer to do it in a separate patch.)



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 17
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 04 Apr 2019 19:48:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 2:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2125/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 2
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 Feb 2019 17:16:30 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 19:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2658/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 19
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Fri, 05 Apr 2019 14:26:16 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#2). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. DATE values do not have a time of day component. The range
of values supported for the DATE type is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from BIGINT, INT, SMALLINT, TINYINT to DATE. The source value is
      interpreted as a number of days since the epoch.
    - from DATE to BIGINT, INT, SMALLINT, TINYINT. The resulting
      integer is the number of days since epoch.
    - from DOUBLE, DECIMAL to DATE. The source value's fractional part
      is ignored, the integer part is interpreted as a number of days
      since epoch.
    - from DATE to DOUBLE, DECIMAL. The resulting value is the number
      of days since epoch.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions.
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/decimal-operators-ir.cc
M be/src/exprs/decimal-operators.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
141 files changed, 2,910 insertions(+), 215 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/2
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 2
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 4:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12481/4/be/src/exprs/expr-test.cc
File be/src/exprs/expr-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/4/be/src/exprs/expr-test.cc@1141
PS4, Line 1141: void ExprTest::TestCast(const string& stmt, const char* val, bool timestamp_out_of_range) {
line too long (91 > 90)


http://gerrit.cloudera.org:8080/#/c/12481/4/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/4/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@76
PS4, Line 76:     // 1. Validate each partition key/value specified, ensuring a matching partition column
line too long (91 > 90)


http://gerrit.cloudera.org:8080/#/c/12481/4/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
File fe/src/test/java/org/apache/impala/analysis/ParserTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/4/fe/src/test/java/org/apache/impala/analysis/ParserTest.java@3310
PS4, Line 3310:         "Expected: ALL, CASE, CAST, DATE, DEFAULT, DISTINCT, EXISTS, FALSE, IF, INTERVAL, " +
line too long (93 > 90)



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 4
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 18:25:37 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 20:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2736/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 20
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 11 Apr 2019 15:31:02 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#11). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
A be/src/util/macro-util.h
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
161 files changed, 3,975 insertions(+), 396 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/11
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

> Uploaded patch set 11.

Rebased it.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 13 Mar 2019 13:07:39 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 1:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/12481/1/be/src/exprs/utility-functions-ir.cc
File be/src/exprs/utility-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/1/be/src/exprs/utility-functions-ir.cc@218
PS1, Line 218: template StringVal UtilityFunctions::TypeOf(FunctionContext* ctx, const DateVal& input_val);
line too long (92 > 90)


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/hs2/test_fetch.py
File tests/hs2/test_fetch.py:

http://gerrit.cloudera.org:8080/#/c/12481/1/tests/hs2/test_fetch.py@180
PS1, Line 180: ;
flake8: E703 statement ends with a semicolon


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@20
PS1, Line 20: from copy import copy
flake8: F401 'copy.copy' imported but unused


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@25
PS1, Line 25: from tests.common.test_vector import ImpalaTestDimension
flake8: F401 'tests.common.test_vector.ImpalaTestDimension' imported but unused


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@27
PS1, Line 27: class TestDateQueries(ImpalaTestSuite):
flake8: E302 expected 2 blank lines, found 1


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@37
PS1, Line 37:  
flake8: E203 whitespace before ':'


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@38
PS1, Line 38:  
flake8: E203 whitespace before ':'


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@39
PS1, Line 39:  
flake8: E203 whitespace before ':'


http://gerrit.cloudera.org:8080/#/c/12481/1/tests/query_test/test_date_queries.py@41
PS1, Line 41: \
flake8: E502 the backslash is redundant between brackets



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 1
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 Feb 2019 16:09:04 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#6). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
146 files changed, 3,234 insertions(+), 249 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/6
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 6
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 23: Code-Review+1

Carry +1


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 17 Apr 2019 18:09:07 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#20). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types (note that invalid
  casts will fail with an error, just like invalid DECIMAL_V2 casts):
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncated silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,260 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/20
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 20
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Csaba Ringhofer (Code Review)" <ge...@cloudera.org>.
Csaba Ringhofer has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(4 comments)

I ran through the non-test cpp code.

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc
File be/src/exec/hdfs-scan-node-base.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc@644
PS3, Line 644:   // TODO: Remove this block once DATE type is supported accross all file formats.
             :   if (has_materialized_date_slot_ && partition->file_format() != THdfsFileFormat::TEXT) {
             :     context->ClearStreams();
             :     return Status("DATE type is only supported with TEXT file format.");
             :   }
I think that this logic should be implemented in frontend, and the backend should only contains DCHECKs.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc@310
PS3, Line 310:   int year, month, day;
             :   if (UNLIKELY(!dv.ToYearMonthDay(&year, &month, &day))) return TimestampVal::null();
             :   if (year < 1400 || year > 9999) return TimestampVal::null();
             : 
             :   const boost::gregorian::date d(year, month, day);
             :   const boost::posix_time::time_duration t(0, 0, 0, 0);
             :   TimestampValue tv(d, t);
performance: this could be done much faster, as both DateVal and TimestampValue's date_ are "number of days since some epoch", so the conversion should be simply checking a range + add the difference between the to epoch.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc@330
PS3, Line 330:   if (val.is_null) return DateVal::null();
             :   TimestampValue tv = TimestampValue::FromTimestampVal(val);
             :   if (UNLIKELY(!tv.HasDate())) return DateVal::null();
             : 
             :   const boost::gregorian::date d = tv.date();
             :   const DateValue dv(d.year(), d.month(), d.day());
             :   return dv.ToDateVal();
same as in line 310: this could be done with a simple addition. There is a function that does exactly this conversion in https://gerrit.cloudera.org/#/c/12247/8/be/src/runtime/timestamp-value.inline.h ( int64_t TimestampValue::DaysSinceUnixEpoch() )


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/slot-ref.cc
File be/src/exprs/slot-ref.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/slot-ref.cc@485
PS3, Line 485:   const DateValue dv(*reinterpret_cast<int32_t*>(t->GetSlot(slot_offset_))
Why don't we reinterpret_cast to DateValue* directly? This would avoid the unnecessary range check in DateValue's constructor.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 Feb 2019 13:29:31 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 10:

(1 comment)

Added DATE error and HBASE tests.

http://gerrit.cloudera.org:8080/#/c/12481/9//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/9//COMMIT_MSG@74
PS9, Line 74: - Add Parquet support.
> Need to update this.
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 10
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 Mar 2019 19:07:32 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 13:

> Uploaded patch set 13.

This patch-set contains the lowercase method name changes in Java files.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 13
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 19:54:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 13:

> I don't think we should bundle a wholesale reformatting change into
 > this patch. Just suggested fixing the name of the method for the
 > ones you added. If we want to reformat, let's do that in a separate
 > gerrit. otherwise this is going to be a pain to review/rebase/etc.
 > 
 > I'll take a look at your changes today or tomorrow.

Okay, I'll do that. Please ignore patch-set 13.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 13
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 20:18:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Csaba Ringhofer (Code Review)" <ge...@cloudera.org>.
Csaba Ringhofer has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 19:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/12481/19//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/19//COMMIT_MSG@20
PS19, Line 20: Explicit casting between DATE and other types:
Please mention that invalid conversion result in a warning, which is different than the other casts work.


http://gerrit.cloudera.org:8080/#/c/12481/19//COMMIT_MSG@24
PS19, Line 24: truncted
typo: truncated


http://gerrit.cloudera.org:8080/#/c/12481/19/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/19/be/src/exprs/cast-functions-ir.cc@312
PS19, Line 312:   if (UNLIKELY(!dv.IsValid())) {
              :     ctx->AddWarning("String to Date parse failed.");
              :     return DateVal::null();
              :   }
I totally prefer returning an error if we are casting a constant, but I am not sure about values fetched during the query. What will happen is that warnings will be added to the list of general errors, and when the number of errors reach MAX_ERRORS (query option, 100 by default in my minicluster), the query will fail.

If we want to fail more consistently, SetError() could be called instead of AddWarning(). Postgres seems to do this, it fails at the first row it cannot cast.

On the other side, the "return NULL without warning" seems like a useful thing when the values are from a table, as it makes it easy for the user to handle the case when not every string is a valid date.

So I would prefer to return error only for constant casts. FunctionContext has a IsArgConstant() functions, but I don't know whether it works during constant folding. 

If we want to postpone this decision, an error could be returned for now, and we could change Impala to be more forgiving in the future and have some special logic for constant casts.


http://gerrit.cloudera.org:8080/#/c/12481/19/be/src/exprs/cast-functions-ir.cc@323
PS19, Line 323:     ctx->AddWarning("Timestamp to Date conversion failed. "
              :         "Timestamp has no date component.");
Note that "dateless timestamps" are nearly useless, see IMPALA-5942. I regret not dropping them among the breaking changes in Impala 3.0. A comment could be added to related functions/tests to avoid giving the impression that these can be actually used for things.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 19
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 10 Apr 2019 14:35:49 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#18). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncted silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,230 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/18
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 18
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc
File be/src/exec/hdfs-scan-node-base.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc@644
PS3, Line 644:   // TODO: Remove this block once DATE type is supported accross all file formats.
             :   if (has_materialized_date_slot_ && partition->file_format() != THdfsFileFormat::TEXT) {
             :     context->ClearStreams();
             :     return Status("DATE type is only supported with TEXT file format.");
             :   }
> I think it would be much more difficult (or impossible) to do this in the F
I think it might be better to reject it in the frontend even if it means rejecting some potentially runnable queries.

Queries like the one you mentioned will be flaky, because they will succeed only if the filters arrive early enough to filter out non-text partitions.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 21:13:37 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

I'm ok with deferring that testing so long as you do a couple of sanity checks to make sure it's not broken. We should file a blocker JIRA to add the testing before the release that this will land in to make sure that it gets done. Otherwise we would need to revert this.

In principle we shouldn't add functionality without comprehensive tests but it's worth making an exception here because of the heroic nature of this code change.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 16:43:15 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc
File be/src/exec/hdfs-scan-node-base.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc@644
PS3, Line 644:   // TODO: Remove this block once DATE type is supported accross all file formats.
             :   if (has_materialized_date_slot_ && partition->file_format() != THdfsFileFormat::TEXT) {
             :     context->ClearStreams();
             :     return Status("DATE type is only supported with TEXT file format.");
             :   }
> I think it might be better to reject it in the frontend even if it means re
Actually I think if you did it in the frontend it depends whether you check before/after static partition pruning. I was thinking that you could check for DATE columns after partition pruning.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 21:15:11 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 24: Verified+1


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 24
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 23 Apr 2019 13:33:56 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#21). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types (note that invalid
  casts will fail with an error, just like invalid DECIMAL_V2 casts):
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncated silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,260 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/21
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 21
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#19). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncted silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,255 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/19
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 19
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/7/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/7/tests/query_test/test_date_queries.py@20
PS7, Line 20: import pytest
flake8: F401 'pytest' imported but unused



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 7
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 Mar 2019 16:38:05 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types (note that invalid
  casts will fail with an error just like invalid DECIMAL_V2 casts,
  while failed casts to other types do no lead to warning or error):
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncated silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Reviewed-on: http://gerrit.cloudera.org:8080/12481
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,254 insertions(+), 476 deletions(-)

Approvals:
  Impala Public Jenkins: Looks good to me, approved; Verified

-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 25
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Csaba Ringhofer (Code Review)" <ge...@cloudera.org>.
Csaba Ringhofer has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 22: Code-Review+1


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 22
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 17 Apr 2019 09:58:25 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 21:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2740/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 21
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 11 Apr 2019 21:09:45 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 18:

(2 comments)

Thanks for the changes.

http://gerrit.cloudera.org:8080/#/c/12481/18/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/18/be/src/exprs/cast-functions-ir.cc@301
PS18, Line 301:         "The valid date range in Timestamps is 1400-01-01..9999-12-31");
Maybe reword a bit:

 "The valid date range for the Timestamp type is..."


http://gerrit.cloudera.org:8080/#/c/12481/18/testdata/workloads/functional-query/queries/QueryTest/date.test
File testdata/workloads/functional-query/queries/QueryTest/date.test:

http://gerrit.cloudera.org:8080/#/c/12481/18/testdata/workloads/functional-query/queries/QueryTest/date.test@564
PS18, Line 564: select coalesce('2012-01-01', cast('2012-02-02' as timestamp), cast('2012-02-02' as timestamp)),
I don't know if it matters but it looks like hive 3 also returns the same for this line and the next except for an extra digit of padding: 2012-01-01 0:00:00

I.e. there are cases where it matches both that seem to have arbitrarily ended up here versus the above query.

Again, I don't know if it matters.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 18
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Fri, 05 Apr 2019 00:33:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

(1 comment)

Also, is the HBase data representation just a string? Is it compatible with Hive's representation.

http://gerrit.cloudera.org:8080/#/c/12481/9//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/9//COMMIT_MSG@74
PS9, Line 74: - Add HBase and Kudu support.
Need to update this.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 16:44:38 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 22:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2815/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 22
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 17 Apr 2019 09:07:57 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 18:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2652/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 18
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 04 Apr 2019 20:29:21 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@20
PS11, Line 20:     - from STRING to DATE. The string value must be formatted as
             :       yyyy-MM-dd.
This seems to be stricter than CAST() in postgres, mysql, and hive (see https://docs.google.com/spreadsheets/d/1IkceWy8lBSkF5NaL-Jls2y86zosyD9pwxtO7qKpSxZw/edit#gid=0 row 31)


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@38
PS11, Line 38: E.g: year('2019-02-15') must resolve to
             :        year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
             :        not implemented yet, so this is not an issue at the moment but
             :        it will be in the future.
> Not necessarily. The valid range for TIMESTAMP is from 1400-01-01 to 9999-1
Should we be emitting a WARNING for any out-of-range values? it seems surprising to silently emit NULL for out-of-range casts (I had a customer complain about this last week wrt timestamp)


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   public boolean compare(Function other, CompareMode mode) {
> I agree that the function overload resolution algorithm needs to be redesig
I'm afraid about introducing new semantics here which we'll need to break again later. I did some testing on your patch relative to Hive 2, Hive 3, postgres, and MariaDB, and the results were pretty interesting:

https://docs.google.com/spreadsheets/d/1IkceWy8lBSkF5NaL-Jls2y86zosyD9pwxtO7qKpSxZw/edit#gid=0

It seems like at least in the cases where Hive3 and Postgres agree, we should probably be giving the same results as well? It certainly seems like Impala is giving NULL in some places I wouldn't have expected.

Maybe we can add a few more queries here, and a column for Impala (prior to the patch) to make sure we aren't introducing any backward-incompatible behavior?

In other words, I think the priorities should be:

1) don't break existing queries
2) for queries where postgres and Hive3 agree, we should give the same result
3) for queries where postgres and Hive differ, we should probably go with Hive3 or collaborate with the Hive team to see if they think this is a bug

In the case that this would require a big restructuring, I'd prefer if in the meantime we result these queries in an error like "ambiguous types detected, insert casts to resolve". Later, if we can figure out correct implicit casting rules, we can loosen it. But I'm against introducing new behavior which we already know that we'll have to break.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 02 Apr 2019 19:35:01 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#22). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types (note that invalid
  casts will fail with an error just like invalid DECIMAL_V2 casts,
  while failed casts to other types do no lead to warning or error):
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncated silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,260 insertions(+), 474 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/22
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 22
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#5). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
146 files changed, 3,234 insertions(+), 249 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/5
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 5
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 7:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/2373/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 7
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 Mar 2019 17:20:20 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(32 comments)

nice work, pretty impressive test coverage. Mostly small items here.

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@29
PS11, Line 29:     - from STRING to DATE if the source string value is used in a
             :       context where a DATE value is expected.
it's interesting that this differs from the behavior of TIMESTAMP today, at least for timestamp arithmetic expressions: SELECT '2019-01-01 12;12:12' + interval 1 day does not do an implicit conversion. (interestingly it does in MySQL).

Does the ANSI standard give rules for this?


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@38
PS11, Line 38: E.g: year('2019-02-15') must resolve to
             :        year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
             :        not implemented yet, so this is not an issue at the moment but
             :        it will be in the future.
wouldn't the result be the same either way for this example?


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@42
PS11, Line 42: better fit
how is this defined, specifically?


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@70
PS11, Line 70: complete DATE type implementation
should we consider turning it off by default and only enabling it once we're sure it's stable and correct?


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@72
PS11, Line 72: - Add date support to the random query generator.
should we transition TPC-DS test tables over to 'date' instead of 'string' for the appropriate columns?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/codegen/codegen-anyval.cc
File be/src/codegen/codegen-anyval.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/codegen/codegen-anyval.cc@68
PS11, Line 68:       return cg->i64_type();
why do we lower to i64 instead of i32 given the slot size is only 4 bytes? seems like we're using int32 below


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exec/hdfs-table-sink.cc
File be/src/exec/hdfs-table-sink.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exec/hdfs-table-sink.cc@500
PS11, Line 500: stringstream error_msg;
              :             error_msg << "Cannot write DATE column to a PARQUET table.";
              :             return Status(error_msg.str());
nit: why not just pass that string directly?

Could we make this better by including the column name from the table desc?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc
File be/src/exprs/aggregate-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc@381
PS11, Line 381: val_struct->sum / val_struct->count
Is this correct behavior when the dates are negative? Dividing as signed integers means we'll "truncate towards zero" which means that average of (1969-01-01, 1969-01-02) would be 1969-01-02 whereas average of 1971-01-01 and 1971-01-02 would be 1971-01-01. I think that's surprising because it exposes the epoch reference point in the semantics of the operation.

Does the ANSI standard have anything to say here?

Checking briefly on sqlfiddle:
- postgres 9.6 doesn't support AVG(date) at all. Neither does Oracle 11g R2 nor SQL Server 2017.
- mysql 5.6 implicitly casts the dates to an DECIMIAL and return something weird like 19690101.5. Not sure what's going on under the covers.
- hive 3.1 doesn't support AVG(date) either

Maybe we should remove this functionality for now and consider adding it back based on user demand?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc
File be/src/exprs/conditional-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc@37
PS11, Line 37: ITERATE_OVER_SEQ
per comment elsewhere, I found it clearer in the old version since I didn't need to understand boost magic


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc
File be/src/exprs/expr-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc@3386
PS11, Line 3386: 
               : TEST_F(ExprTest, CastDateExprs) {
add some tests for invalid dates like feb 30


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc@3475
PS11, Line 3475:   TestDateValue("cast(cast('1400-01-01 00:00:00' as timestamp) as date)",
can you test a before-the-epoch timestamp that includes a time portion? eg '1960-01-01 23:59' and make sure it truncates "down" and not truncate "towards the epoch"?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/date-value.h
File be/src/runtime/date-value.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/date-value.h@40
PS11, Line 40: /// - Proleptic Gregorian calendar is used to calculate the number of days since epoch,
             : ///   which can lead to different representation of historical dates compared to Hive.
             : ///   (https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar)
I know this was from the original change a few months back, but should we be concerned about this? Should we have a compatibility mode even if there is a performance hit?

Doing some research online it seems like hive might have had different behavior previously but then switched to proleptic gregorian as of 3.1?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/timestamp-value.inline.h
File be/src/runtime/timestamp-value.inline.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/timestamp-value.inline.h@36
PS11, Line 36:   return TimestampValue(EPOCH + boost::gregorian::date_duration(days), t);
is there any chance of exceptions in this boost API? I seem to recall there were some unexpected exceptions thrown by boost::date_time stuff


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/service/fe-support.cc
File be/src/service/fe-support.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/service/fe-support.cc@638
PS11, Line 638:     jclass caller_class, jstring date) {
I see 'caller_class' was copied from above, but that's a somewhat inaccurate name. Really the class here is the FeSupport class itself, given this is a static method, right?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/udf/udf.h
File be/src/udf/udf.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/udf/udf.h@599
PS11, Line 599:   DateVal(underlying_type_t val = 0) : val(val) { }
should be marked explicit?


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc
File be/src/util/string-parser-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc@534
PS11, Line 534:   // Test bad formats.
worth a check where there is a valid leading date followed by junk ("2010-01-01 foo") as well as one where it's a full valid timestamp ("2010-01-01 12:22:22") to make sure we get the expected results


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc@544
PS11, Line 544:   // Test invalid month/day values.
what about special days like February 29th? Worth testing that in a year when it doesn't exist and a year when it exists, including perhaps some "exception" leap years? ie 2000-02-29 does exist but 2100-02-29 doesn't


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/symbols-util.cc
File be/src/util/symbols-util.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/symbols-util.cc@129
PS11, Line 129:     ITERATE_OVER_SEQ2(CASE_TYPE_APPEND_MANGLED_TOKEN,
I dunno that using the ITERATE_OVER is really any clearer than just repeating the macro N times and not even using BOOST_PP_STRINGIZE:

CASE_TYPE_APPEND_MANGLED_TOKEN(TYPE_BOOLEAN, "BooleanVal");
CASE_TYPE_APPEND_MANGLED_TOKEN(TYPE_TINYINT, "TinyIntVal");
...

it's only slightly more typing and I think less cognitive overhead for someone to go figure out what this weird boost macro magic is doing


http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/Exprs.thrift
File common/thrift/Exprs.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/Exprs.thrift@58
PS11, Line 58:   // String representation
             :   2: required string date_string
why is it necessary to include the string representation here too? I don't see a similar field in other literal expr types that have to be "parsed" and may not round-trip to exactly the same string the user input (timestamp, float, etc)


http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/ImpalaInternalService.thrift
File common/thrift/ImpalaInternalService.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/ImpalaInternalService.thrift@745
PS11, Line 745: sinxce
typo


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@61
PS11, Line 61:       throw new AnalysisException("Invalid date literal: " + e.getMessage(), e);
would this actually be due to an invalid date literal? or would this be truly a bug? maybe makes sense to have a different error message here due to an internal error?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@65
PS11, Line 65:       throw new AnalysisException("Invalid date literal: " + strDate);
I think it's worth quoting the date literal they passed in so if they did something like DATE ' ' then they will get a readable error message


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@66
PS11, Line 66: else
nit: no need for else here since you threw above


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@103
PS11, Line 103: return
nit: missing space


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@83
PS11, Line 83: This is necessary as a DATE literal can have different string
             :     //       representations, but all these representations refer to the same partition
hrm... this part is spooking me a little bit. What happens if you have multiple partitions with different representations for the same date? Also, are you sure we aren't relying on immutability of the partitionSpec_ list anywhere? Is there any other way we can accomplish this?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
File fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java@1148
PS11, Line 1148:     // Avg(Date)
is this capability part of the SQL standard?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
File fe/src/main/java/org/apache/impala/catalog/ColumnStats.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java@293
PS11, Line 293: LongMath.pow(2, Integer.SIZE)
should we instead cap at the number of unique dates in 0000-00-00 to 9999-12-31? I think this is smaller than 2^32 right?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   public boolean compare(Function other, CompareMode mode) {
Is there an ANSI standard for function resolution with implicit casts? Have you checked what Hive does here? It would be no good if we were inconsistent


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
File fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java@1248
PS11, Line 1248:               "Scanning DATE values is not supported for non-text fileformats ");
Can we include the table name here?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java@732
PS11, Line 732: TestResolveFunction
nit: here and elsewhere, the function names should start with a lower case letter in java


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
File fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java@67
PS11, Line 67:     testLiteralExprNegative("ABC", Type.DATE);
worth a negative test here for an invalid date like february 31?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
File fe/src/test/java/org/apache/impala/analysis/ParserTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/ParserTest.java@1215
PS11, Line 1215:     ParserError("select date '2011--01'");
same, check for feb 31



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 14 Mar 2019 04:22:12 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@29
PS11, Line 29:     - from STRING to DATE if the source string value is used in a
             :       context where a DATE value is expected.
> it's interesting that this differs from the behavior of TIMESTAMP today, at
The interval implementation in Impala is kind-of weird because it's implemented as a special operator instead of as an INTERVAL data type. I don't think this is related to the (lack of) implicit casting but it does relate to the standard support.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Mon, 18 Mar 2019 16:02:14 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#23). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types (note that invalid
  casts will fail with an error just like invalid DECIMAL_V2 casts,
  while failed casts to other types do no lead to warning or error):
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd HH:mm:ss.SSSSSSSSS. The date component is mandatory,
      the time component is optional. If the time component is
      present, it will be truncated silently.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/datetime-parse-util.cc
M be/src/runtime/datetime-parse-util.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-parse-util.cc
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
163 files changed, 4,254 insertions(+), 476 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/23
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 23:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2821/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 17 Apr 2019 14:23:40 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 8:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/2374/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 8
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 06 Mar 2019 17:23:36 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

(2 comments)

Do you mean that, with these code changes, you can scan HBase date tables? Or that we should have negative tests for HBase?

I think we should have test coverage but I'm open to splitting that out into a separate patch if that helps this get in faster.

http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@127
PS6, Line 127:     return daysSinceEpoch_ - other.daysSinceEpoch_;
> Done. At the very beginning 'daysSinceEpoch_' was not a member and later I 
Makes sense!


http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/date.test
File testdata/workloads/functional-query/queries/QueryTest/date.test:

http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/date.test@36
PS6, Line 36: where '2017-11-28' in (date_col)
> To be honest, I didn't give this too much thought. I just copied the timest
Ah ok, that makes sense. I don't think there are likely to be bugs here but it's good to be sure.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 00:55:30 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 12:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2504/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 12
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 19:32:39 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 5:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/2216/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 5
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 19:13:53 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#13). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/AnalyticExpr.java
M fe/src/main/java/org/apache/impala/analysis/AnalyticWindow.java
M fe/src/main/java/org/apache/impala/analysis/Analyzer.java
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/Expr.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladTableUsageTracker.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/catalog/local/CatalogdMetaProvider.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/planner/ValueRange.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeModifyStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeUpsertStmtTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprSelectivityTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/FrontendTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/TestFileParser.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
178 files changed, 9,641 insertions(+), 6,045 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/13
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 13
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc
File be/src/exprs/aggregate-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc@381
PS11, Line 381: val_struct->sum / val_struct->count
> I agree avg(date) is likely not important. Kind-of nice to have for consist
Yea, my feeling was just that one fewer feature to test and bug-fix and support is good if it's not an important use case.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Mon, 18 Mar 2019 16:36:31 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#8). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
A be/src/util/macro-util.h
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
A testdata/workloads/functional-query/queries/QueryTest/date_text_only_support.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
153 files changed, 3,533 insertions(+), 395 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/8
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 8
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 12:

(32 comments)

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@29
PS11, Line 29: 
             : - Implicit casting between DATE and other typ
> it's interesting that this differs from the behavior of TIMESTAMP today, at
Implicit casting from STRING to TIMESTAMP works in Impala in most cases (parameters to function calls/operators, values in insert/select statements, etc).
- Hive does these kind of implicit conversions too,
- PostgreSQL on the other hand doesn't.
- Couldn't find anything about STRING->TIMESTAMP implicit conversions in the SQL ANSI standard.

Timestamp arithmetic expressions in Impala are an exception: in TimestampArithmeticExpr.java implicit conversions are explicitly forbidden.
- Neither Hive nor PostgreSQL allow implicit conversions in arithmetic expressions.


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@38
PS11, Line 38: ution logic is not adequate anymore.
             :   For example, it resolves the
             :   if(false, '2011-01-01', DATE '1499-02-02') function call to the
             :   if(BOOLEAN, TIMESTAMP, TIMESTA
> wouldn't the result be the same either way for this example?
Not necessarily. The valid range for TIMESTAMP is from 1400-01-01 to 9999-12-31. The valid range for DATE is from 0000-01-01 to 9999-12-31. Therefore:
year(cast('1399-12-31' as TIMESTAMP)) returns NULL, whereas
year(cast('1399-12-31' as DATE)) will return 1399.

In Hive the valid range for TIMESTAMP and DATE is the same, so this is not an issue.


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@42
PS11, Line 42: DATE, DATE
> how is this defined, specifically?
I've changed the wording to describe the function resolution algorithm and related issues in more detail.


http://gerrit.cloudera.org:8080/#/c/12481/11//COMMIT_MSG@70
PS11, Line 70: iders overloaded function
> It seems easier to leave it enabled so long as the current support is corre
I agree with Tim, introducing a feature flag for DATE support would take a considerable effort since this patch touches lots of files.

Testing whether turning off the feature flag works as expected (i.e. we get the same behavior and error messages as before) can also be complicated.

Also, the patch includes a sql-parser change. Moving the parser change behind a feature flag is probably not that straightforward either.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/codegen/codegen-anyval.cc
File be/src/codegen/codegen-anyval.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/codegen/codegen-anyval.cc@68
PS11, Line 68:       return cg->i64_type();
> why do we lower to i64 instead of i32 given the slot size is only 4 bytes? 
The highest 4 bytes store the actual value, the lowest byte is used for the "is_null" flag.

Similarly, TYPE_INT represents an i32 value but its lowered type is i64.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exec/hdfs-table-sink.cc
File be/src/exec/hdfs-table-sink.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exec/hdfs-table-sink.cc@500
PS11, Line 500:  (type.type == TYPE_DATE) {
              :             ColumnDescriptor col_desc = table_desc_->col_descs()[num_clustering_cols + i];
              :             stringstream error_msg;
> nit: why not just pass that string directly?
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc
File be/src/exprs/aggregate-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/aggregate-functions-ir.cc@381
PS11, Line 381: s should not have a negative perfor
> Yea, my feeling was just that one fewer feature to test and bug-fix and sup
Removed avg(DATE) to avoid semantic problems.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc
File be/src/exprs/conditional-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/conditional-functions-ir.cc@37
PS11, Line 37: IS_NULL_COMPUTE_
> per comment elsewhere, I found it clearer in the old version since I didn't
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc
File be/src/exprs/expr-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc@3386
PS11, Line 3386: 
               : TEST_F(ExprTest, CastDateExprs) {
> add some tests for invalid dates like feb 30
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/exprs/expr-test.cc@3475
PS11, Line 3475:   TestError("cast(cast(null as date) as double)");
> can you test a before-the-epoch timestamp that includes a time portion? eg 
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/date-value.h
File be/src/runtime/date-value.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/date-value.h@40
PS11, Line 40: /// - Proleptic Gregorian calendar is used to calculate the number of days since epoch,
             : ///   which can lead to different representation of historical dates compared to Hive.
             : ///   (https://en.wikipedia.org/wiki/Proleptic_Gregorian_calendar)
> I know this was from the original change a few months back, but should we b
I didn't get the chance to test Hive 3.1, but yes it seems that Hive 3.1 switched to using Proleptic Gregorian calendar too.

We could introduce a feature flag in Impala to switch between Julian/Gregorian and Proleptic Gregorian calendars. Please note though that this is not just a DATE issue: TIMESTAMP type already has the same calendar-incompatibility problem.

If we want to fix this, I suggest introducing a feature flag that affects both TIMESTAMP & DATE types in a separate patch.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/timestamp-value.inline.h
File be/src/runtime/timestamp-value.inline.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/runtime/timestamp-value.inline.h@36
PS11, Line 36:   return TimestampValue(EPOCH + boost::gregorian::date_duration(days), t);
> is there any chance of exceptions in this boost API? I seem to recall there
I've tested EPOCH + boost::gregorian::date_duration(days) with days outside of the valid [1400-01-01..9999.12.31] range and it doesn't throw an exception.

If 'days' is outside of the supported range, L36 returns a TimestampValue instance where TimestampValue.HasDate() is false.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/service/fe-support.cc
File be/src/service/fe-support.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/service/fe-support.cc@638
PS11, Line 638:     jclass fe_support_class, jstring date) {
> I see 'caller_class' was copied from above, but that's a somewhat inaccurat
Renamed it to 'fe_support_class' here and elsewhere.


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/udf/udf.h
File be/src/udf/udf.h:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/udf/udf.h@599
PS11, Line 599:   explicit DateVal(underlying_type_t val = 0) : val(val) { }
> should be marked explicit?
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc
File be/src/util/string-parser-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc@534
PS11, Line 534:   // Test bad formats.
> worth a check where there is a valid leading date followed by junk ("2010-0
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/string-parser-test.cc@544
PS11, Line 544:   TestDateValue("2010-01-01 12:22:22", invalid_date, StringParser::PARSE_FAILURE);
> what about special days like February 29th? Worth testing that in a year wh
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/symbols-util.cc
File be/src/util/symbols-util.cc:

http://gerrit.cloudera.org:8080/#/c/12481/11/be/src/util/symbols-util.cc@129
PS11, Line 129:     CASE_TYPE_APPEND_MANGLED_TOKEN(TYPE_SMALLINT, SmallIntVal)
> I dunno that using the ITERATE_OVER is really any clearer than just repeati
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/Exprs.thrift
File common/thrift/Exprs.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/Exprs.thrift@58
PS11, Line 58:   // String representation
             :   2: required string date_string
> why is it necessary to include the string representation here too? I don't 
Having the string representation here is not absolutely necessary, it is included for convenience.

When we create a DateLiteral object from a TDateLiteral object, it is beneficial to pass both the string and the i32 representation to the DateLiteral constructor. Otherwise, we would have to convert the i32 representation to the string representation on the Java side. This would require calling a native BE function from Java, which is somewhat inconvenient and I'd like to avoid it if possible.

FLOAT and other similar types don't have this problem as they can always be converted to string in Java easily.

TIMESTAMPs are handled differently as there is no way to directly specify a literal timestamp from SQL.


http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/ImpalaInternalService.thrift
File common/thrift/ImpalaInternalService.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/11/common/thrift/ImpalaInternalService.thrift@745
PS11, Line 745: since 
> typo
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@61
PS11, Line 61:       throw new AnalysisException("Error parsing date literal: " + e.getMessage(), e);
> would this actually be due to an invalid date literal? or would this be tru
True, this is the wrong error message. Replaced with "Error parsing.." which is what we use for similar errors in other places..


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@65
PS11, Line 65:       throw new AnalysisException("Invalid date literal: '" + strDate + "'");
> I think it's worth quoting the date literal they passed in so if they did s
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@66
PS11, Line 66: 
> nit: no need for else here since you threw above
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@103
PS11, Line 103:  retur
> nit: missing space
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@83
PS11, Line 83:  possible, replace STRING_LITERAL value with the corresponding
             :     //       DateLiteral object.
> hrm... this part is spooking me a little bit. What happens if you have mult
I've changed the wording, I hope it makes more sense now.

L125-131 was added to solve the problem they you described: different string representations of the same date will be disambiguated and replaced with the corresponding DateLiteral object.

'partitionSpec_' is made immutable in L135 in PartitionSpec.analyze().

I've checked the FE code: PartitionSpec.getPartitionSpecKeyValues() is called only after PartitionSpec.analyze() is called, so the returned list should already be immutable.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
File fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java@1148
PS11, Line 1148: 
> is this capability part of the SQL standard?
It isn't, and Hive doesn't support it either. Removed it for now.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
File fe/src/main/java/org/apache/impala/catalog/ColumnStats.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/ColumnStats.java@293
PS11, Line 293: t dates in the 0000-01-01..99
> should we instead cap at the number of unique dates in 0000-00-00 to 9999-1
True, fixed it.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   public boolean compare(Function other, CompareMode mode) {
> Is there an ANSI standard for function resolution with implicit casts? Have
I took a quick look at the SQL ANSI standard, but couldn't find anything about function name resolution.

Hive takes a very different approach, it doesn't have overloaded functions in the same sense as Impala. Built-in functions take generic parameters and decide on their own what implicit casts to apply on their arguments and in which order. For instance:
- add_months() used to convert its first argument to date (even if it was a timestamp string) until recently, but since HIVE-19370 it tries to do a timestamp-conversion first and falls back on date-conversion only if that fails.
- date_add() on the other hand still does date-conversion only. This is probably a bug, but this is how it works right now.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
File fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java@1248
PS11, Line 1248:               "Scanning DATE values in table '" + table.getFullName() +
> Can we include the table name here?
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java@732
PS11, Line 732: TestResolveFunction
> nit: here and elsewhere, the function names should start with a lower case 
Changed the 1st letter of function names to lower case in all the java files that I've touched in this patch. I ended up changing another 5K+ lines in 37 files. I will upload these changes in a separate patch-set.


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
File fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java@67
PS11, Line 67:     testLiteralExprNegative("ABC", Type.DATE);
> worth a negative test here for an invalid date like february 31?
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
File fe/src/test/java/org/apache/impala/analysis/ParserTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/test/java/org/apache/impala/analysis/ParserTest.java@1215
PS11, Line 1215:     ParserError("select date '2011--01'");
> same, check for feb 31
Done


http://gerrit.cloudera.org:8080/#/c/12481/10/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/10/tests/query_test/test_date_queries.py@107
PS10, Line 107:   def test_impala_shell(self):
> We don't need to test this for multiple file formats, right?
Correct. Note, that this test will be executed only once, since 'test_impala_shell' doesn't take a 'vector' parameter.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 12
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 18:47:06 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 19:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/12481/18/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/18/be/src/exprs/cast-functions-ir.cc@301
PS18, Line 301:         "The valid date range for the Timestamp type is 1400-01-01..9999-12-31.");
> Maybe reword a bit:
Done


http://gerrit.cloudera.org:8080/#/c/12481/18/testdata/workloads/functional-query/queries/QueryTest/date.test
File testdata/workloads/functional-query/queries/QueryTest/date.test:

http://gerrit.cloudera.org:8080/#/c/12481/18/testdata/workloads/functional-query/queries/QueryTest/date.test@564
PS18, Line 564: # Impala returns the same results as PostgreSQL.
> I don't know if it matters but it looks like hive 3 also returns the same f
The main issue with the first and second expression in this select is that they return TIMESTAMPs in Impala and STRINGs in Hive3.

This is one of the subtle differences between how implicit casts work in Impala and Hive3. Impala implicitly converts all three parameters to TIMESTAMP, whereas Hive3 converts all three to STRING. Consequently the return types are different too. Unfortunately changing this behavior in Impala would break backward compatibility, as that would change TIMESTAMP behavior too, not just DATE behavior.

You are correct about the third expression: it returns a TIMESTAMP both in Impala and Hive3. Hive3's string representation of the TIMESTAMP contains an extra ".0" padding, but I don't think that matters. I've moved this expression to the previous section.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 19
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Fri, 05 Apr 2019 13:41:25 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 15:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@52
PS14, Line 52:   public PartitionSpec(List<PartitionKeyValue> partitionSpec) {
> Do we ever expect this to get called prior to analysis?
Done


http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@156
PS14, Line 156:       }
> same (Precondition on analysis if possible)
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java
File fe/src/main/java/org/apache/impala/catalog/Function.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@191
PS11, Line 191:   // Compares this to 'other' for 'mode'.
> hm, I spent some time researching this across multiple databases:
I agree that the function overload resolution algorithm needs to be redesigned but I don't think we should do it now as it would break backward compatibility.

We can reconsider implicit casting rules and re-implement the resolution logic (preferably modelled after what Hive does) in a separate change when we are ready for a breaking change.

What do you think?


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/catalog/Function.java@236
PS11, Line 236:   private int calcSuperTypeOfMatchScore(Function other, boolean strict) {
> I think we could simplify this code a lot by adding some utility function l
Done


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java
File fe/src/main/java/org/apache/impala/util/FunctionUtils.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java@243
PS11, Line 243:     return max_func;
> it's interesting that our function resolution returns the first fit, from t
That's correct, currently we never raise "ambiguous resolution" error.

Adding an error to that effect sounds reasonable but I'm worried that it will introduce a breaking change.
E.g. year('2011-01-01') resolves to year(TIMESTAMP) right now, but after introducing year(DATE) users will get an error.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 15
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 27 Mar 2019 17:42:27 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#4). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions:
     - If both implicit conversions are applicable equally well,
       STRING -> TIMESTAMP is preferred for backward compatibility
       reasons. E.g: year('2019-02-15') must resolve to
       year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is
       not implemented yet, so this is not an issue at the moment but
       it will be in the future.
     - If one implicit conversion is a better fit over the other, we
       must choose the better one. E.g:
       if(false, '2011-01-01', DATE '1499-02-02') must resolve to
       if(BOOLEAN, DATE, DATE) instead of
       if(BOOLEAN, TIMESTAMP, TIMESTAMP).
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add HBase and Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
146 files changed, 3,232 insertions(+), 249 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/4
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 4
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(19 comments)

Thanks for the reviews!

While I was going through the comments, I realized that partitioning by DATE wasn't properly implemented. I fixed those issues as well and added some extra tests to cover partitioning functionality.

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@11
PS3, Line 11: of values supported for the DATE type is 0000-01-01 to 9999-12-31.
> Can you mention the literal syntax supported for date values?
Done


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@22
PS3, Line 22:     - from BIGINT, INT, SMALLINT, TINYINT to DATE. The source value is
            :       interpreted as a number of days since the epoch.
            :     - from DATE to BIGINT, INT, SMALLINT, TINYINT. The resulting
            :       integer is the number of days since epoch.
            :     - from DOUBLE, FLOAT, DECIMAL to DATE. The source value's
            :       fractional part is ignored, the integer part is interpreted as a
            :       number of days since epoch.
            :     - from DATE to DOUBLE, FLOAT, DECIMAL. The resulting value is the
            :       number of days since epoch.
> Hive does not support these casts (I checked with Hive 2.1.1), are you sure
I introduced them as all these conversions are supported for timestamps. On the other hand if Hive doesn't support them, probably we shouldn't either.
Removed them.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@35
PS3, Line 35: - Implicit casting between DATE and other types:
> Do you plan to support the DATE 'yyyy-mm-dd' literal syntax? It looks like 
Good idea, added support for that.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@40
PS3, Line 40: - Since both STRING -> DATE and STRING -> TIMESTAMP implicit
> Do you have an example of where this is required?
Added more information with some examples.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@63
PS3, Line 63:       tests/query_test/test_date_queries.py.
> Can we add a simple test that runs a query returning date through impala-sh
Added an extra test to test_date_queries.py.


http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@64
PS3, Line 64: 
> I think we should plan on adding DATE support to the random query generator
Added an extra section to the commit msg to describe future plans.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc
File be/src/exec/hdfs-scan-node-base.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exec/hdfs-scan-node-base.cc@644
PS3, Line 644:   // TODO: Remove this block once DATE type is supported accross all file formats.
             :   if (has_materialized_date_slot_ && partition->file_format() != THdfsFileFormat::TEXT) {
             :     context->ClearStreams();
             :     return Status("DATE type is only supported with TEXT file format.");
             :   }
> I think that this logic should be implemented in frontend, and the backend 
I think it would be much more difficult (or impossible) to do this in the FE. 

Imagine joining a table to a partitioned table on a partition column, where each partition has a different file format. I don't think it is possible to figure out in the analyze/plan phase whether non-text partitions will be scanned or not.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc@310
PS3, Line 310:   int year, month, day;
             :   if (UNLIKELY(!dv.ToYearMonthDay(&year, &month, &day))) return TimestampVal::null();
             :   if (year < 1400 || year > 9999) return TimestampVal::null();
             : 
             :   const boost::gregorian::date d(year, month, day);
             :   const boost::posix_time::time_duration t(0, 0, 0, 0);
             :   TimestampValue tv(d, t);
> performance: this could be done much faster, as both DateVal and TimestampV
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/cast-functions-ir.cc@330
PS3, Line 330:   if (val.is_null) return DateVal::null();
             :   TimestampValue tv = TimestampValue::FromTimestampVal(val);
             :   if (UNLIKELY(!tv.HasDate())) return DateVal::null();
             : 
             :   const boost::gregorian::date d = tv.date();
             :   const DateValue dv(d.year(), d.month(), d.day());
             :   return dv.ToDateVal();
> same as in line 310: this could be done with a simple addition. There is a 
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/slot-ref.cc
File be/src/exprs/slot-ref.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/exprs/slot-ref.cc@485
PS3, Line 485:   const DateValue dv(*reinterpret_cast<int32_t*>(t->GetSlot(slot_offset_))
> Why don't we reinterpret_cast to DateValue* directly? This would avoid the 
This was intentional: DateValue uses a specific int32_t value (DateValue::INVALID_DAYS_SINCE_EPOCH) to represent invalid dates.

*reinterpret_cast<int32_t*>(t->GetSlot(slot_offset_) may return an invalid value that differs from INVALID_DAYS_SINCE_EPOCH


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/udf/udf.h
File be/src/udf/udf.h:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/udf/udf.h@586
PS3, Line 586: /// Date value represented as days since epoch 1970-01-01.
> The valid range and the fact that the value is interpreted as proleptic Gre
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc
File be/src/util/string-parser-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc@526
PS3, Line 526:   TestDateValue("0000-01-01", DateValue(0, 1, 1), StringParser::PARSE_SUCCESS);
             :   TestDateValue("9999-12-31", DateValue(9999, 12, 31), StringParser::PARSE_SUCCESS);
> Please add the "wrong side" of the edge values:
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc@531
PS3, Line 531:   TestDateValue("2-11-10", invalid_date, StringParser::PARSE_FAILURE);
> Please add tests where month/day have the correct number of characters but 
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/common/thrift/Exprs.thrift
File common/thrift/Exprs.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/3/common/thrift/Exprs.thrift@56
PS3, Line 56:   // string representation of date formatted as yyyy-MM-dd.
            :   1: required string value;
> Why do you use string instead of i32 to represent the date? TTimestampLiter
I've kept the string member and added the i32 value to this struct.

TDateLiteral is the thrift representation od DateLiteral Java class. DateLiteral is used to represent DATE 'yyyy-MM-dd' date literals.

When we create a DateLiteral object from a TDateLiteral object, it is beneficial to pass both the string and the i32 representation to the DateLiteral constructor. Otherwise, we would have to convert the i32 representation to the string representation on the Java side. Such a conversion could potentially be inconsistent with what the BE would produce.

TIMESTAMPs are handled differently as there is no way to directly specify a literal timestamp from SQL.


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@32
PS3, Line 32: yyy
> Isn't it yyyy?
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
File fe/src/main/java/org/apache/impala/analysis/StringLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java@155
PS3, Line 155: isDateType
> By implementing the Date type the name of this function became very mislead
Done


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java@194
PS3, Line 194: AnalyzesOk("select cast (cast ('1970-10-10 10:00:00.123' as timestamp) as date)");
> What will happen if some writes
Yes, it returns NULL w/o a warning. This is how explicit casts from string work in impala in general. E.g.:

select cast('123.0' as int); 

also returns NULL w/o a warning.


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
File fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java@368
PS3, Line 368:     assertToSqlWithImplicitCasts(ctx, "select * from functional.alltypes, "
             :         + "functional.date_tbl where timestamp_col = date_col",
             :         "SELECT * FROM functional.alltypes, functional.date_tbl "
             :         + "WHERE timestamp_col = CAST(date_col AS TIMESTAMP)");
> I saw some weirdness around this in Hive, it seems to convert everything to
Interesting, in postgresql and mysql:

select cast('1970-01-01' as timestamp) > cast('1970-01-01' as date);

returns false. This could be a Hive bug.


http://gerrit.cloudera.org:8080/#/c/12481/3/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/3/tests/query_test/test_date_queries.py@18
PS3, Line 18: decimal
> date
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 18:09:30 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 15:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2563/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 15
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 27 Mar 2019 17:32:24 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 6:

(10 comments)

I did a pass over the backend code. I think it's looking pretty good. Most comments I had were actually about existing issues in the codebase.

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/aggregate-functions-ir.cc
File be/src/exprs/aggregate-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/aggregate-functions-ir.cc@386
PS6, Line 386:   if (UNLIKELY(src.is_null)) return DateVal::null();
It's weird to handle is_null here and not in GetValue(). I think it's unnecessary. Seems to be carried over from the other functions. I'm ok if you don't want to handle it right now, but maybe file a JIRA if you agree with me that it's wrong?


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/cast-functions-ir.cc@179
PS6, Line 179: lexical_cast
It would be better to use ToString() directly rather than involving the indirection of lexical_cast and operator<<.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/conditional-functions-ir.cc
File be/src/exprs/conditional-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/conditional-functions-ir.cc@35
PS6, Line 35: IS_NULL_COMPUTE_FUNCTION(BooleanVal);
I know we'd talked about how to make it easier to add more data types in future. This is one case where we could have some macros to stamp out functions like this for each *Val type. Not saying we need to do it here, just thinking.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/expr-test.cc
File be/src/exprs/expr-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/expr-test.cc@3466
PS6, Line 3466: a
are


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/slot-ref.cc
File be/src/exprs/slot-ref.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/exprs/slot-ref.cc@451
PS6, Line 451: TimestampVal SlotRef::GetTimestampVal(
These functions look regular enough that we could maybe stamp them out automatically from all types with a macro, except for the parameterised types like DecimalVal. Don't need to fix now, just thinking.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.h
File be/src/runtime/date-value.h:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.h@96
PS6, Line 96: underlaying
underlying


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.cc
File be/src/runtime/date-value.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.cc@68
PS6, Line 68:   (void)DateParser::Parse(str, len, &dv);
I think you'll need to use discard_result() here. I can't remember exactly but either GCC7 or Clang doesn't consider this to be actually discarding the return value.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/fe-support.cc
File be/src/service/fe-support.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/fe-support.cc@639
PS6, Line 639:  std::s
nit: std:: prefix not needed since it's imported into the global namespace by common/names.h


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/hs2-util.cc
File be/src/service/hs2-util.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/hs2-util.cc@539
PS6, Line 539:         RawValue::PrintValue(value, TYPE_DATE, -1, &(hs2_col_val->stringVal.value));
Can we just use StringValue::ToString() instead? Seems like a lot of indirection to use this. Oh I guess TIMESTAMP does this below, oh well.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/util/symbols-util.cc
File be/src/util/symbols-util.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/util/symbols-util.cc@135
PS6, Line 135:     case TYPE_DATE:
Another example where it's just stamped out for each *Val type.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 6
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Tue, 26 Feb 2019 01:58:29 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2126/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Comment-Date: Thu, 14 Feb 2019 17:20:17 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 10:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/10/tests/query_test/test_date_queries.py
File tests/query_test/test_date_queries.py:

http://gerrit.cloudera.org:8080/#/c/12481/10/tests/query_test/test_date_queries.py@107
PS10, Line 107:   def test_impala_shell(self):
We don't need to test this for multiple file formats, right?



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 10
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Mon, 18 Mar 2019 16:08:13 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 23: Code-Review+1

(1 comment)

Looked at the delta, good catch by csaba

http://gerrit.cloudera.org:8080/#/c/12481/19/be/src/exprs/cast-functions-ir.cc
File be/src/exprs/cast-functions-ir.cc:

http://gerrit.cloudera.org:8080/#/c/12481/19/be/src/exprs/cast-functions-ir.cc@312
PS19, Line 312:   if (UNLIKELY(!dv.IsValid())) {
              :     ctx->SetError("String to Date parse failed.");
              :     return DateVal::null();
              :   }
> Casts from STRING to most types don't return an error and don't emit a warn
Yeah this was a good point that I missed. We've tried to be stricter with new functionality, e.g. DECIMAL_V2 errors in more cases than before.

I think in principle we should avoid different behaviour between constants and variables, i.e. f(1) and f(x) where x=1 should have exactly the same behaviour. This is an invariant that most programmers would use to reason about their programs. 

So the current patch's behaviour is good.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 18 Apr 2019 00:31:34 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Csaba Ringhofer (Code Review)" <ge...@cloudera.org>.
Csaba Ringhofer has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(9 comments)

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/3//COMMIT_MSG@22
PS3, Line 22:     - from BIGINT, INT, SMALLINT, TINYINT to DATE. The source value is
            :       interpreted as a number of days since the epoch.
            :     - from DATE to BIGINT, INT, SMALLINT, TINYINT. The resulting
            :       integer is the number of days since epoch.
            :     - from DOUBLE, FLOAT, DECIMAL to DATE. The source value's
            :       fractional part is ignored, the integer part is interpreted as a
            :       number of days since epoch.
            :     - from DATE to DOUBLE, FLOAT, DECIMAL. The resulting value is the
            :       number of days since epoch.
Hive does not support these casts (I checked with Hive 2.1.1), are you sure that it is useful to support them in Impala? My first impression is that these casts are more confusing than useful.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/udf/udf.h
File be/src/udf/udf.h:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/udf/udf.h@586
PS3, Line 586: /// Date value represented as days since epoch 1970-01-01.
The valid range and the fact that the value is interpreted as proleptic Gregorian could be also mentioned here.


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc
File be/src/util/string-parser-test.cc:

http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc@526
PS3, Line 526:   TestDateValue("0000-01-01", DateValue(0, 1, 1), StringParser::PARSE_SUCCESS);
             :   TestDateValue("9999-12-31", DateValue(9999, 12, 31), StringParser::PARSE_SUCCESS);
Please add the "wrong side" of the edge values:

-0001-12-31 (Hive had a bug with these kinds of values)
10000-01-01


http://gerrit.cloudera.org:8080/#/c/12481/3/be/src/util/string-parser-test.cc@531
PS3, Line 531:   TestDateValue("2-11-10", invalid_date, StringParser::PARSE_FAILURE);
Please add tests where month/day have the correct number of characters but the value is invalid, e.g 2018-00-10, 2018-13-10, 2018-01-0, 2018-01-32, 2019-02-39


http://gerrit.cloudera.org:8080/#/c/12481/3/common/thrift/Exprs.thrift
File common/thrift/Exprs.thrift:

http://gerrit.cloudera.org:8080/#/c/12481/3/common/thrift/Exprs.thrift@56
PS3, Line 56:   // string representation of date formatted as yyyy-MM-dd.
            :   1: required string value;
Why do you use string instead of i32 to represent the date? TTimestampLiteral uses binary representation instead of string.


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@32
PS3, Line 32: yyy
Isn't it yyyy?


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
File fe/src/main/java/org/apache/impala/analysis/StringLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java@155
PS3, Line 155: isDateType
By implementing the Date type the name of this function became very misleading. Can you rename it to something like isDateOrTimeType()?


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java@194
PS3, Line 194: AnalyzesOk("select cast (cast ('1970-10-10 10:00:00.123' as timestamp) as date)");
What will happen if some writes
select cast('1970-10-10 10:00:00.123' as date)
? Will it return NULL without warning?


http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
File fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java:

http://gerrit.cloudera.org:8080/#/c/12481/3/fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java@368
PS3, Line 368:     assertToSqlWithImplicitCasts(ctx, "select * from functional.alltypes, "
             :         + "functional.date_tbl where timestamp_col = date_col",
             :         "SELECT * FROM functional.alltypes, functional.date_tbl "
             :         + "WHERE timestamp_col = CAST(date_col AS TIMESTAMP)");
I saw some weirdness around this in Hive, it seems to convert everything to string:

select cast("1970-01-01" as timestamp) > cast("1970-01-01" as date);
return True
What happened is that the appended 00:00:00 part made the timestamp version larger.

I prefer to compare them as timestamps, but this difference could be documented somewhere.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 15 Feb 2019 17:03:12 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 23:

> Uploaded patch set 23.

Change has been rebased


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 23
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 17 Apr 2019 13:42:42 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2423/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 13 Mar 2019 13:53:15 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Todd Lipcon (Code Review)" <ge...@cloudera.org>.
Todd Lipcon has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 13:

I don't think we should bundle a wholesale reformatting change into this patch. Just suggested fixing the name of the method for the ones you added. If we want to reformat, let's do that in a separate gerrit. otherwise this is going to be a pain to review/rebase/etc.

I'll take a look at your changes today or tomorrow.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 13
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 20:03:57 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2378/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 02:14:41 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 14:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2521/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 14
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Fri, 22 Mar 2019 16:25:38 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 15:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
File fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java:

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java@351
PS14, Line 351:         return "AVG requires a numeric, timestamp or date parameter: " + toSql();
> Needs update
Done



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 15
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 28 Mar 2019 13:00:23 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#14). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. For example: DATE '2019-02-15'. DATE values do not have a
time of day component. The range of values supported for the DATE type
is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT and HBASE fileformats only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Support for DATE literal syntax.

- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.

- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.

- Since STRING -> DATE, STRING -> TIMESTAMP and DATE -> TIMESTAMP
  implicit conversions are now all possible, the existing function
  overload resolution logic is not adequate anymore.
  For example, it resolves the
  if(false, '2011-01-01', DATE '1499-02-02') function call to the
  if(BOOLEAN, TIMESTAMP, TIMESTAMP) version of the overloaded
  function, instead of the if(BOOLEAN, DATE, DATE) version.

  This is clearly wrong, so the function overload resolution logic had
  to be changed to resolve function calls to the best-fit overloaded
  function definition if there are multiple applicable candidates.

  An overloaded function definition is an applicable candidate for a
  function call if each actual parameter in the function call either
  matches the corresponding formal parameter's type (without casting)
  or is implicitly castable to that type.

  When looking for the best-fit applicable candidate, a parameter
  match score (i.e. the number of actual parameters in the function
  call that match their corresponding formal parameter's type without
  casting) is calculated and the applicable candidate with the highest
  parameter match score is chosen.

  There's one more issue that the new resolution logic has to address:
  if two applicable candidates have the same parameter match score and
  the only difference between the two is that the first one requires a
  STRING -> TIMESTAMP implicit cast for some of its parameters while
  the second one requires a STRING -> DATE implicit cast for the same
  parameters then the first candidate has to be chosen not to break
  backward compatibility.
  E.g: year('2019-02-15') function call must resolve to
  year(TIMESTAMP) instead of year(DATE). Note, that year(DATE) is not
  implemented yet, so this is not an issue at the moment but it will
  be in the future.
  When the resolution algorithm considers overloaded function
  definitions, first it orders them lexicographically by the types in
  their parameter lists. To ensure the backward compatible behavior
  Primitivetype.DATE enum value has to come after
  PrimitiveType.TIMESTAMP.

- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' (and the
  corresponding HBASE table 'functional_hbase.date_tbl') was
  introduced for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT and HBASE fileformats
      only, most DATE tests were implemented separately in
      tests/query_test/test_date_queries.py.

Note, that this change-set is not a complete DATE type implementation,
but it lays the foundation for future work:
- Add date support to the random query generator.
- Implement a complete set of built-in functions.
- Add Parquet support.
- Add Kudu support.
- Optionally support Avro and ORC.
For further details, see IMPALA-6169.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-parse-util.h
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/timestamp-value.h
M be/src/runtime/timestamp-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/ImpalaInternalService.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/cup/sql-parser.cup
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
M fe/src/main/java/org/apache/impala/service/FeSupport.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaConverter.java
M fe/src/main/java/org/apache/impala/util/AvroSchemaParser.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
M testdata/data/README
A testdata/data/date_tbl.avro
A testdata/data/date_tbl.parquet
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
A testdata/data/date_tbl_error/0000.txt
A testdata/data/date_tbl_error/0001.txt
A testdata/data/date_tbl_error/0002.txt
A testdata/data/date_tbl_error/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/DataErrorsTest/hdfs-scan-node-errors.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date-partitioning.test
A testdata/workloads/functional-query/queries/QueryTest/date-text-only-support.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-inserts.test
M testdata/workloads/functional-query/queries/QueryTest/hbase-scan-node.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
159 files changed, 3,945 insertions(+), 354 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/14
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 14
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 10:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2419/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 10
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Tue, 12 Mar 2019 19:47:22 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 24: Code-Review+2


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 24
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 23 Apr 2019 08:16:40 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 4:

Build Failed 

https://jenkins.impala.io/job/gerrit-code-review-checks/2215/ : Initial code review checks failed. See linked job for details on the failure.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 4
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 19:05:56 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 16:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2571/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 16
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 28 Mar 2019 13:42:04 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 9:

I'll leave that up to you. I think it depends whether we're trying to get this into the upstream release or not - that's probably something to figure out between you and Gabor.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 9
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 07 Mar 2019 18:06:46 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 3:

(8 comments)

I did a pass over the remaining bits of the patch. Overall it looks really good.

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.h
File be/src/runtime/date-value.h:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/runtime/date-value.h@59
PS6, Line 59:   DateValue(int32_t days_since_epoch) : days_since_epoch_(INVALID_DAYS_SINCE_EPOCH) {
Can you make these constructors explicit? Just to avoid any potential unexpected conversions.


http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/hs2-util.cc
File be/src/service/hs2-util.cc:

http://gerrit.cloudera.org:8080/#/c/12481/6/be/src/service/hs2-util.cc@539
PS6, Line 539:         RawValue::PrintValue(value, TYPE_DATE, -1, &(hs2_col_val->stringVal.value));
> Can we just use StringValue::ToString() instead? Seems like a lot of indire
I still think it would be good to fix this, just so that the new code is efficient.


http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
File fe/src/main/java/org/apache/impala/analysis/DateLiteral.java:

http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@40
PS6, Line 40:     strValue_ = strValue;
Mention that it's the same representation as DateValue in backend?


http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/DateLiteral.java@127
PS6, Line 127: 
Why not compare daysSinceEpoch_?


http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java
File fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java:

http://gerrit.cloudera.org:8080/#/c/12481/6/fe/src/main/java/org/apache/impala/analysis/PartitionSpec.java@81
PS6, Line 81: alysis
whether


http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
File testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test:

http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test@4
PS6, Line 4: create table date_tbl like functional.date_tbl;
Test might be slightly clearer if this had a different name from the functional version, e.g. tmp_date_tbl or date_tbl_copy.


http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/date.test
File testdata/workloads/functional-query/queries/QueryTest/date.test:

http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/date.test@35
PS6, Line 35: select count(*) from date_tbl
We should test the IN predicate with both long and short lists to exercise both the set lookup and iterate code. I don't expect there to be bugs here but it's just interesting enough that it would be good to have the coverage.


 // Threshold based on InPredicateBenchmark results
      int setLookupThreshold = children_.get(0).getType().isStringType() ? 2 : 6;
      if (children_.size() - 1 < setLookupThreshold) useSetLookup = false;


http://gerrit.cloudera.org:8080/#/c/12481/6/testdata/workloads/functional-query/queries/QueryTest/date.test@36
PS6, Line 36: where '2017-11-28' in (date_col)
Is it deliberate that all this IN/NOT IN/BETWEEN predicates can be rewritten to simple comparisons by the planner? I don't understand focusing on this rather than variants that can't be simplified (testing edge cases is good, we just need to test the regular cases too).



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Tue, 05 Mar 2019 02:43:28 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Csaba Ringhofer (Code Review)" <ge...@cloudera.org>.
Csaba Ringhofer has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 21: Code-Review+1

(1 comment)

http://gerrit.cloudera.org:8080/#/c/12481/21//COMMIT_MSG
Commit Message:

http://gerrit.cloudera.org:8080/#/c/12481/21//COMMIT_MSG@21
PS21, Line 21:   casts will fail with an error, just like invalid DECIMAL_V2 casts):
It could be added that "while failed casts to other types do no lead to warning or error".



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 21
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Tue, 16 Apr 2019 14:04:49 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 11:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
File fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java:

http://gerrit.cloudera.org:8080/#/c/12481/14/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java@351
PS14, Line 351:         return "AVG requires a numeric, timestamp or date parameter: " + toSql();
Needs update


http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java
File fe/src/main/java/org/apache/impala/util/FunctionUtils.java:

http://gerrit.cloudera.org:8080/#/c/12481/11/fe/src/main/java/org/apache/impala/util/FunctionUtils.java@243
PS11, Line 243:     return max_func;
> That's correct, currently we never raise "ambiguous resolution" error.
I don't think this behaviour was a well thought-out decision. The earliest versions of this that I saw was very hacky and actually returned an arbitrary function when there were multiple valid overloads - whichever was listed first in the catalog. I changed that to be a bit more intelligent about choosing functions that don't result in loss of information and also and checking functions in a deterministic order. It's not the way I would have designed it but it was the least disruptive way to evolve it.

Introducing an error where there wasn't one before is pretty risky because it will probably break existing queries. Your example seems highly likely to be used in practice. 

There are some cases where it might not be a breaking change to raise an error, because they didn't occur before, e.g. if one of the inputs is a DATE. But not sure how interesting those cases are.

I tend to think that something like this is probably the least of the evils, but the complexity of the behaviour is not ideal.



-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 11
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Wed, 27 Mar 2019 17:59:06 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 13:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2506/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 13
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 21 Mar 2019 20:34:59 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 6:

Build Successful 

https://jenkins.impala.io/job/gerrit-code-review-checks/2218/ : Initial code review checks passed. Use gerrit-verify-dryrun-external or gerrit-verify-dryrun to run full precommit tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 6
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Fri, 22 Feb 2019 21:39:17 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has uploaded a new patch set (#3). ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................

IMPALA-7368: Add initial support for DATE type

DATE values describe a particular year/month/day in the form
yyyy-MM-dd. DATE values do not have a time of day component. The range
of values supported for the DATE type is 0000-01-01 to 9999-12-31.

This initial DATE type support covers TEXT fileformat only.
'DateValue' is used as the internal type to represent DATE values.

The changes are as follows:
- Explicit casting between DATE and other types:
    - from STRING to DATE. The string value must be formatted as
      yyyy-MM-dd.
    - from DATE to STRING. The resulting string value is formatted as
      yyyy-MM-dd.
    - from BIGINT, INT, SMALLINT, TINYINT to DATE. The source value is
      interpreted as a number of days since the epoch.
    - from DATE to BIGINT, INT, SMALLINT, TINYINT. The resulting
      integer is the number of days since epoch.
    - from DOUBLE, FLOAT, DECIMAL to DATE. The source value's
      fractional part is ignored, the integer part is interpreted as a
      number of days since epoch.
    - from DATE to DOUBLE, FLOAT, DECIMAL. The resulting value is the
      number of days since epoch.
    - from TIMESTAMP to DATE. The source timestamp's time of day
      component is ignored.
    - from DATE to TIMESTAMP. The target timestamp's time of day
      component is set to 00:00:00.
- Implicit casting between DATE and other types:
    - from STRING to DATE if the source string value is used in a
      context where a DATE value is expected.
    - from DATE to TIMESTAMP if the source date value is used in a
      context where a TIMESTAMP value is expected.
- Since both STRING -> DATE and STRING -> TIMESTAMP implicit
  conversions are possible, the function resolution logic was changed
  to select the right version of overloaded functions.
- Codegen infrastructure changes for expression evaluation.
- 'IS [NOT] NULL' and '[NOT] IN' predicates.
- Common comparison operators (including the 'BETWEEN' operator).
- Infrastructure changes for built-in functions.
- Some built-in functions: conditional, aggregate, analytical and
  math functions.
- C++ UDF/UDA support.
- Support partitioning and grouping by DATE.
- Beeswax, HiveServer2 support.

These items are tightly coupled and it makes sense to implement them
in one change-set.

Testing:
- A new partitioned TEXT table 'functional.date_tbl' was introduced
  for DATE-related tests.
- BE and FE tests were extended to cover DATE type.
- E2E tests:
    - since DATE type is supported for TEXT fileformat only, most DATE
      tests were implemented separately in
      tests/query_test/test_date_queries.py.

Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
---
M be/src/codegen/codegen-anyval.cc
M be/src/codegen/codegen-anyval.h
M be/src/codegen/gen_ir_descriptions.py
M be/src/codegen/llvm-codegen.cc
M be/src/exec/aggregator.cc
M be/src/exec/data-source-scan-node.cc
M be/src/exec/hash-table.cc
M be/src/exec/hdfs-scan-node-base.cc
M be/src/exec/hdfs-scan-node-base.h
M be/src/exec/hdfs-scanner-ir.cc
M be/src/exec/hdfs-table-sink.cc
M be/src/exec/text-converter.cc
M be/src/exec/text-converter.inline.h
M be/src/exprs/agg-fn-evaluator.cc
M be/src/exprs/aggregate-functions-ir.cc
M be/src/exprs/aggregate-functions.h
M be/src/exprs/anyval-util.cc
M be/src/exprs/anyval-util.h
M be/src/exprs/case-expr.cc
M be/src/exprs/case-expr.h
M be/src/exprs/cast-functions-ir.cc
M be/src/exprs/cast-functions.h
M be/src/exprs/conditional-functions-ir.cc
M be/src/exprs/conditional-functions.h
M be/src/exprs/decimal-operators-ir.cc
M be/src/exprs/decimal-operators.h
M be/src/exprs/expr-test.cc
M be/src/exprs/expr-value.h
M be/src/exprs/hive-udf-call.cc
M be/src/exprs/hive-udf-call.h
M be/src/exprs/in-predicate-ir.cc
M be/src/exprs/in-predicate.h
M be/src/exprs/is-null-predicate-ir.cc
M be/src/exprs/literal.cc
M be/src/exprs/literal.h
M be/src/exprs/math-functions-ir.cc
M be/src/exprs/math-functions.h
M be/src/exprs/null-literal.cc
M be/src/exprs/null-literal.h
M be/src/exprs/operators-ir.cc
M be/src/exprs/operators.h
M be/src/exprs/scalar-expr-evaluator.cc
M be/src/exprs/scalar-expr-evaluator.h
M be/src/exprs/scalar-expr-ir.cc
M be/src/exprs/scalar-expr.cc
M be/src/exprs/scalar-expr.h
M be/src/exprs/scalar-fn-call.cc
M be/src/exprs/scalar-fn-call.h
M be/src/exprs/slot-ref.cc
M be/src/exprs/slot-ref.h
M be/src/exprs/timestamp-functions.h
M be/src/exprs/utility-functions-ir.cc
M be/src/exprs/utility-functions.h
M be/src/rpc/thrift-util.cc
M be/src/runtime/date-test.cc
M be/src/runtime/date-value.cc
M be/src/runtime/date-value.h
M be/src/runtime/raw-value-ir.cc
M be/src/runtime/raw-value-test.cc
M be/src/runtime/raw-value.cc
M be/src/runtime/raw-value.inline.h
M be/src/runtime/types.cc
M be/src/runtime/types.h
M be/src/service/fe-support.cc
M be/src/service/hs2-util.cc
M be/src/service/query-result-set.cc
M be/src/testutil/test-udas.cc
M be/src/testutil/test-udfs.cc
M be/src/udf/udf-test.cc
M be/src/udf/udf.h
M be/src/util/CMakeLists.txt
M be/src/util/static-asserts.cc
M be/src/util/string-parser-test.cc
M be/src/util/string-parser.h
M be/src/util/symbols-util-test.cc
M be/src/util/symbols-util.cc
M bin/rat_exclude_files.txt
M common/function-registry/impala_functions.py
M common/thrift/Exprs.thrift
M common/thrift/Types.thrift
M ext-data-source/test/src/main/java/org/apache/impala/extdatasource/AllTypesDataSource.java
M fe/src/main/java/org/apache/impala/analysis/CastExpr.java
A fe/src/main/java/org/apache/impala/analysis/DateLiteral.java
M fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
M fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
M fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
M fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
M fe/src/main/java/org/apache/impala/catalog/ColumnStats.java
M fe/src/main/java/org/apache/impala/catalog/DataSourceTable.java
M fe/src/main/java/org/apache/impala/catalog/Function.java
M fe/src/main/java/org/apache/impala/catalog/PrimitiveType.java
M fe/src/main/java/org/apache/impala/catalog/ScalarFunction.java
M fe/src/main/java/org/apache/impala/catalog/ScalarType.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
M fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
M fe/src/main/java/org/apache/impala/util/FunctionUtils.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzeSubqueriesTest.java
M fe/src/test/java/org/apache/impala/analysis/AnalyzerTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
M fe/src/test/java/org/apache/impala/analysis/ExprRewriterTest.java
M fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
M fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
M fe/src/test/java/org/apache/impala/catalog/PartialCatalogInfoTest.java
M fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
M fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
M fe/src/test/java/org/apache/impala/hive/executor/TestUdf.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M testdata/bin/compute-table-stats.sh
M testdata/bin/create-data-source-table.sql
A testdata/data/date_tbl/0000.txt
A testdata/data/date_tbl/0001.txt
A testdata/data/date_tbl/0002.txt
A testdata/data/date_tbl/0003.txt
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/functional/schema_constraints.csv
M testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
M testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
M testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
M testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
A testdata/workloads/functional-query/queries/QueryTest/compute-stats-date.test
M testdata/workloads/functional-query/queries/QueryTest/data-source-tables.test
A testdata/workloads/functional-query/queries/QueryTest/date.test
M testdata/workloads/functional-query/queries/QueryTest/decimal-exprs.test
M testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
M testdata/workloads/functional-query/queries/QueryTest/joins.test
M testdata/workloads/functional-query/queries/QueryTest/misc.test
M testdata/workloads/functional-query/queries/QueryTest/subquery.test
M testdata/workloads/functional-query/queries/QueryTest/uda.test
M testdata/workloads/functional-query/queries/QueryTest/udf.test
M testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
M tests/custom_cluster/test_permanent_udfs.py
M tests/hs2/test_fetch.py
M tests/metadata/test_compute_stats.py
A tests/query_test/test_date_queries.py
M tests/query_test/test_udfs.py
M tests/test-hive-udfs/src/main/java/org/apache/impala/TestUdf.java
141 files changed, 2,910 insertions(+), 215 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/81/12481/3
-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 3
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>

[Impala-ASF-CR] IMPALA-7368: Add initial support for DATE type

Posted by "Attila Jeges (Code Review)" <ge...@cloudera.org>.
Attila Jeges has posted comments on this change. ( http://gerrit.cloudera.org:8080/12481 )

Change subject: IMPALA-7368: Add initial support for DATE type
......................................................................


Patch Set 21:

Fixed a couple failing e2e tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/12481
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iea8155ef09557e0afa2f8b2d0b2dc9d0896dc30f
Gerrit-Change-Number: 12481
Gerrit-PatchSet: 21
Gerrit-Owner: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Attila Jeges <at...@cloudera.com>
Gerrit-Reviewer: Csaba Ringhofer <cs...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins <im...@cloudera.com>
Gerrit-Reviewer: Lars Volker <lv...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>
Gerrit-Comment-Date: Thu, 11 Apr 2019 20:26:48 +0000
Gerrit-HasComments: No