(hive) branch master updated: HIVE-28129 Execute statement doesnot report the correct query string information (#5155) (Ramesh Kumar reviewed by Stamatis Zampetakis)

2024-03-26 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 106c52e999c HIVE-28129 Execute statement doesnot report the correct 
query string information (#5155) (Ramesh Kumar reviewed by Stamatis Zampetakis)
106c52e999c is described below

commit 106c52e999cbd656909061b473bbd71f9a464aa5
Author: Ramesh Kumar 
AuthorDate: Tue Mar 26 11:08:31 2024 -0700

HIVE-28129 Execute statement doesnot report the correct query string 
information (#5155) (Ramesh Kumar reviewed by Stamatis Zampetakis)
---
 .../hive/ql/parse/ExecuteStatementAnalyzer.java|   2 +
 .../results/clientpositive/llap/prepare_plan.q.out | 178 -
 .../llap/prepare_plan_partition_pruning.q.out  |   8 +-
 3 files changed, 74 insertions(+), 114 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
index 535d8d855ed..1428bea874b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
@@ -206,8 +206,10 @@ public class ExecuteStatementAnalyzer extends 
SemanticAnalyzer{
 
   // reset config
   String queryId = this.conf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID);
+  String queryString = 
this.conf.getVar(HiveConf.ConfVars.HIVE_QUERY_STRING);
   this.conf.syncFromConf(cachedPlan.getQueryState().getConf());
   this.conf.setVar(HiveConf.ConfVars.HIVE_QUERY_ID, queryId);
+  this.conf.setVar(HiveConf.ConfVars.HIVE_QUERY_STRING, queryString);
 
   // set rest of the params
   this.inputs = cachedPlan.getInputs();
diff --git a/ql/src/test/results/clientpositive/llap/prepare_plan.q.out 
b/ql/src/test/results/clientpositive/llap/prepare_plan.q.out
index fa892cd7345..d20ade93f20 100644
--- a/ql/src/test/results/clientpositive/llap/prepare_plan.q.out
+++ b/ql/src/test/results/clientpositive/llap/prepare_plan.q.out
@@ -128,11 +128,11 @@ STAGE PLANS:
   Processor Tree:
 ListSink
 
-PREHOOK: query: prepare pcount from select count(*) from src where key > ?
+PREHOOK: query: explain execute pcount using '200'
 PREHOOK: type: EXECUTE QUERY
 PREHOOK: Input: default@src
  A masked pattern was here 
-POSTHOOK: query: prepare pcount from select count(*) from src where key > ?
+POSTHOOK: query: explain execute pcount using '200'
 POSTHOOK: type: EXECUTE QUERY
 POSTHOOK: Input: default@src
  A masked pattern was here 
@@ -194,20 +194,20 @@ STAGE PLANS:
   Processor Tree:
 ListSink
 
-PREHOOK: query: prepare pcount from select count(*) from src where key > ?
+PREHOOK: query: execute pcount using '200'
 PREHOOK: type: EXECUTE QUERY
 PREHOOK: Input: default@src
  A masked pattern was here 
-POSTHOOK: query: prepare pcount from select count(*) from src where key > ?
+POSTHOOK: query: execute pcount using '200'
 POSTHOOK: type: EXECUTE QUERY
 POSTHOOK: Input: default@src
  A masked pattern was here 
 378
-PREHOOK: query: prepare pcount from select count(*) from src where key > ?
+PREHOOK: query: execute pcount using '0'
 PREHOOK: type: EXECUTE QUERY
 PREHOOK: Input: default@src
  A masked pattern was here 
-POSTHOOK: query: prepare pcount from select count(*) from src where key > ?
+POSTHOOK: query: execute pcount using '0'
 POSTHOOK: type: EXECUTE QUERY
 POSTHOOK: Input: default@src
  A masked pattern was here 
@@ -283,11 +283,11 @@ STAGE PLANS:
   Processor Tree:
 ListSink
 
-PREHOOK: query: prepare p1 from select * from src where key > ? order by key 
limit 10
+PREHOOK: query: explain execute p1 using '100'
 PREHOOK: type: EXECUTE QUERY
 PREHOOK: Input: default@src
  A masked pattern was here 
-POSTHOOK: query: prepare p1 from select * from src where key > ? order by key 
limit 10
+POSTHOOK: query: explain execute p1 using '100'
 POSTHOOK: type: EXECUTE QUERY
 POSTHOOK: Input: default@src
  A masked pattern was here 
@@ -354,11 +354,11 @@ STAGE PLANS:
   Processor Tree:
 ListSink
 
-PREHOOK: query: prepare p1 from select * from src where key > ? order by key 
limit 10
+PREHOOK: query: execute p1 using '100'
 PREHOOK: type: EXECUTE QUERY
 PREHOOK: Input: default@src
  A masked pattern was here 
-POSTHOOK: query: prepare p1 from select * from src where key > ? order by key 
limit 10
+POSTHOOK: query: execute p1 using '100'
 POSTHOOK: type: EXECUTE QUERY
 POSTHOOK: Input: default@src
  A masked pattern was here 
@@ -459,13 +459,13 @@ STAGE PLANS:
   Processor Tree:
 ListSink
 
-PREHOOK: query: prepare pint
-from select avg(ctinyint) as ag from alltypesorc where cint <= ?  and 
cbigint <= ? and cfloat != ? group by ct

(hive) branch master updated: HIVE-27751 Log Query Compilation summary in an accumulated way (#4755) (Ramesh Kumar reviewed by Stamatis Zampetakis)

2024-03-22 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new b0245de9c55 HIVE-27751 Log Query Compilation summary in an accumulated 
way (#4755) (Ramesh Kumar reviewed by Stamatis Zampetakis)
b0245de9c55 is described below

commit b0245de9c550b78e6aa7ab4b435c3cc07034d339
Author: Ramesh Kumar 
AuthorDate: Fri Mar 22 22:14:54 2024 -0700

HIVE-27751 Log Query Compilation summary in an accumulated way (#4755) 
(Ramesh Kumar reviewed by Stamatis Zampetakis)
---
 .../org/apache/hadoop/hive/ql/log/PerfLogger.java  |  22 
 .../java/org/apache/hadoop/hive/ql/Compiler.java   |   2 +-
 .../hive/ql/QueryCompilationSummaryHook.java   | 130 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java   |  46 
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java |  18 +++
 .../clientpositive/querycompilesummaryhook.q   |   8 ++
 .../llap/querycompilesummaryhook.q.out |  86 ++
 7 files changed, 291 insertions(+), 21 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java 
b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
index cc57e9b42b0..530a114b950 100644
--- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
@@ -40,10 +40,32 @@ import java.util.concurrent.ConcurrentHashMap;
 public class PerfLogger {
   public static final String ACQUIRE_READ_WRITE_LOCKS = 
"acquireReadWriteLocks";
   public static final String COMPILE = "compile";
+  public static final String COMPILE_STEP = "Compile Step";
   public static final String WAIT_COMPILE = "waitCompile";
   public static final String PARSE = "parse";
   public static final String ANALYZE = "semanticAnalyze";
   public static final String OPTIMIZER = "optimizer";
+  public static final String GENERATE_RESOLVED_PARSETREE = "Generate Resolved 
ParseTree";
+  public static final String LOGICALPLAN_AND_HIVE_OPERATOR_TREE = "Logical 
Plan and hive Operator Tree";
+  public static final String DEDUCE_RESULTSET_SCHEMA = "Deduce 
ResultsetSchema";
+  public static final String PARSE_CONTEXT_GENERATION = "Parse Context 
generation";
+  public static final String SAVE_AND_VALIDATE_VIEW = "Save and Validate View 
Creation";
+  public static final String LOGICAL_OPTIMIZATION = "Logical Optimization";
+  public static final String PHYSICAL_OPTIMIZATION = "Physical Optimization";
+  public static final String POST_PROCESSING = "Post Processing";
+  public static final String GENERATE_LOGICAL_PLAN = "Generate Logical Plan";
+  public static final String GENERATE_OPERATOR_TREE = "Generate Operator Tree";
+  public static final String VIEW_REWRITING = "Calcite: View-based rewriting";
+  public static final String PLAN_GENERATION = "Calcite: Plan generation";
+  public static final String MV_REWRITE_FIELD_TRIMMER = "MV Rewrite and Field 
Trimmer";
+  public static final String REMOVING_SUBQUERY = "Removing SubQuery";
+  public static final String DECORRELATION = "Decorrelation";
+  public static final String VALIDATE_QUERY_MATERIALIZATION = "Validate Query 
Materialization";
+  public static final String PREJOIN_ORDERING = "Calcite: Prejoin ordering 
transformation";
+  public static final String MV_REWRITING = "MV Rewriting";
+  public static final String JOIN_REORDERING = "Calcite: Join Reordering";
+  public static final String POSTJOIN_ORDERING = "Calcite: Postjoin ordering 
transformation";
+  public static final String HIVE_SORT_PREDICATES = "Hive Sort Predicates";
   public static final String MATERIALIZED_VIEWS_REGISTRY_REFRESH = 
"MaterializedViewsRegistryRefresh";
   public static final String DO_AUTHORIZATION = "doAuthorization";
   public static final String DRIVER_EXECUTE = "Driver.execute";
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Compiler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/Compiler.java
index 9d677cd6d24..930c14139fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Compiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Compiler.java
@@ -482,6 +482,7 @@ public class Compiler {
   }
 
   private void cleanUp(Throwable compileException, boolean parsed, boolean 
deferClose) {
+double duration = perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.COMPILE) / 
1000.00;
 // Trigger post compilation hook. Note that if the compilation fails here 
then
 // before/after execution hook will never be executed.
 if (parsed) {
@@ -492,7 +493,6 @@ public class Compiler {
   }

(hive) branch master updated: HIVE_27843 Add QueryOperation to Hive proto logger for post execution hook information (Ramesh Kumar, reviewed by Attila Turoczy, Ayush Saxena)

2023-11-27 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 9b4ea7affa4 HIVE_27843 Add QueryOperation to Hive proto logger for 
post execution hook information (Ramesh Kumar, reviewed by Attila Turoczy, 
Ayush Saxena)
9b4ea7affa4 is described below

commit 9b4ea7affa4902fc2849f1a88b68103940fc9866
Author: Ramesh Kumar 
AuthorDate: Mon Nov 27 07:41:28 2023 -0800

HIVE_27843 Add QueryOperation to Hive proto logger for post execution hook 
information (Ramesh Kumar, reviewed by Attila Turoczy, Ayush Saxena)
---
 ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java  | 3 ++-
 .../test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java | 2 ++
 2 files changed, 4 insertions(+), 1 deletion(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
index 618c7108f67..74d6ac4ce01 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java
@@ -168,7 +168,7 @@ public class HiveProtoLoggingHook implements 
ExecuteWithHookContext {
 
   public enum OtherInfoType {
 QUERY, STATUS, TEZ, MAPRED, INVOKER_INFO, SESSION_ID, THREAD_NAME, 
VERSION, CLIENT_IP_ADDRESS,
-HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID, ERROR_MESSAGE
+HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID, ERROR_MESSAGE, 
QUERY_TYPE
   }
 
   public enum ExecutionMode {
@@ -445,6 +445,7 @@ public class HiveProtoLoggingHook implements 
ExecuteWithHookContext {
   }
   addMapEntry(builder, OtherInfoType.STATUS, Boolean.toString(success));
   addMapEntry(builder, OtherInfoType.ERROR_MESSAGE, 
hookContext.getErrorMessage());
+  addMapEntry(builder, OtherInfoType.QUERY_TYPE, 
hookContext.getQueryState().getCommandType());
   JSONObject perfObj = new JSONObject();
   for (String key : hookContext.getPerfLogger().getEndTimes().keySet()) {
 perfObj.put(key, hookContext.getPerfLogger().getDuration(key));
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java 
b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
index 6b65f0c7873..bff605967cf 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java
@@ -86,6 +86,7 @@ public class TestHiveProtoLoggingHook {
 tmpFolder = folder.newFolder().getAbsolutePath();
 conf.setVar(HiveConf.ConfVars.HIVE_PROTO_EVENTS_BASE_PATH, tmpFolder);
 QueryState state = new QueryState.Builder().withHiveConf(conf).build();
+state.setCommandType(HiveOperation.QUERY);
 @SuppressWarnings("serial")
 QueryPlan queryPlan = new QueryPlan(HiveOperation.QUERY) {};
 queryPlan.setQueryId("test_queryId");
@@ -246,6 +247,7 @@ public class TestHiveProtoLoggingHook {
 Assert.assertEquals("test_op_id", event.getOperationId());
 
 assertOtherInfo(event, OtherInfoType.STATUS, Boolean.TRUE.toString());
+assertOtherInfo(event, OtherInfoType.QUERY_TYPE, 
HiveOperation.QUERY.toString());
 String val = findOtherInfo(event, OtherInfoType.PERF);
 Map map = new ObjectMapper().readValue(val,
 new TypeReference>() {});



[hive] branch master updated: HIVE-27687 Logger variable should be static final as its creation takes more time in query compilation (#4692). (Ramesh Kumar reviewed by Stamatis Zampetakis)

2023-09-29 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 85f6162becb HIVE-27687 Logger variable should be static final as its 
creation takes more time in query compilation (#4692). (Ramesh Kumar reviewed 
by Stamatis Zampetakis)
85f6162becb is described below

commit 85f6162becb8723ff6c9f85875048ced6ca7ae89
Author: Ramesh Kumar 
AuthorDate: Fri Sep 29 08:56:51 2023 -0700

HIVE-27687 Logger variable should be static final as its creation takes 
more time in query compilation (#4692). (Ramesh Kumar reviewed by Stamatis 
Zampetakis)
---
 .../java/org/apache/hadoop/hive/contrib/serde2/RegexSerDe.java |  6 --
 .../src/main/java/org/apache/hive/hcatalog/data/JsonSerDe.java |  8 ++--
 .../src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java  |  6 +-
 .../apache/hadoop/hive/serde2/AbstractEncodingAwareSerDe.java  |  2 +-
 .../src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java  |  4 ++--
 serde/src/java/org/apache/hadoop/hive/serde2/RegexSerDe.java   | 10 +++---
 6 files changed, 25 insertions(+), 11 deletions(-)

diff --git 
a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/RegexSerDe.java 
b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/RegexSerDe.java
index ff8fb8f339d..1bb77ba6976 100644
--- a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/RegexSerDe.java
+++ b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/RegexSerDe.java
@@ -76,6 +76,8 @@ import org.apache.hadoop.io.Writable;
 RegexSerDe.INPUT_REGEX_CASE_SENSITIVE })
 public class RegexSerDe extends AbstractEncodingAwareSerDe {
 
+  private static final Logger LOG = LoggerFactory.getLogger(RegexSerDe.class);
+
   public static final String INPUT_REGEX = "input.regex";
   public static final String OUTPUT_FORMAT_STRING = "output.format.string";
   public static final String INPUT_REGEX_CASE_SENSITIVE = 
"input.regex.case.insensitive";
@@ -173,7 +175,7 @@ public class RegexSerDe extends AbstractEncodingAwareSerDe {
   if (unmatchedRows >= nextUnmatchedRows) {
 nextUnmatchedRows = getNextNumberToDisplay(nextUnmatchedRows);
 // Report the row
-log.warn("{} unmatched rows are found: {}", unmatchedRows, rowText);
+LOG.warn("{} unmatched rows are found: {}", unmatchedRows, rowText);
   }
   return null;
 }
@@ -187,7 +189,7 @@ public class RegexSerDe extends AbstractEncodingAwareSerDe {
 if (partialMatchedRows >= nextPartialMatchedRows) {
   nextPartialMatchedRows = 
getNextNumberToDisplay(nextPartialMatchedRows);
   // Report the row
-  log.warn("" + partialMatchedRows
+  LOG.warn("" + partialMatchedRows
   + " partially unmatched rows are found, " + " cannot find group "
   + c + ": " + rowText);
 }
diff --git 
a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/JsonSerDe.java 
b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/JsonSerDe.java
index 7defd21c081..900b15e0ebc 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/JsonSerDe.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/data/JsonSerDe.java
@@ -42,6 +42,8 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hive.hcatalog.common.HCatException;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @SerDeSpec(schemaProps = {serdeConstants.LIST_COLUMNS,
   serdeConstants.LIST_COLUMN_TYPES,
@@ -49,6 +51,8 @@ import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
   serdeConstants.SERIALIZATION_ENCODING})
 public class JsonSerDe extends AbstractEncodingAwareSerDe {
 
+  private static final Logger LOG = LoggerFactory.getLogger(JsonSerDe.class);
+
   private HCatSchema schema;
 
   private HCatRecordObjectInspector cachedObjectInspector;
@@ -65,8 +69,8 @@ public class JsonSerDe extends AbstractEncodingAwareSerDe {
 cachedObjectInspector = 
HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector(rowTypeInfo);
 try {
   schema = 
HCatSchemaUtils.getHCatSchema(rowTypeInfo).get(0).getStructSubSchema();
-  log.debug("schema : {}", schema);
-  log.debug("fields : {}", schema.getFieldNames());
+  LOG.debug("schema : {}", schema);
+  LOG.debug("fields : {}", schema.getFieldNames());
 } catch (HCatException e) {
   throw new SerDeException(e);
 }
diff --git 
a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java 
b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java
index bf9f03beb8a

[hive] branch master updated: HIVE-27293 Vectorization: Incorrect results with nvl for ORC table (#4379)

2023-06-09 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 7c89b306231 HIVE-27293 Vectorization: Incorrect results with nvl for 
ORC table (#4379)
7c89b306231 is described below

commit 7c89b306231773759172ee67f05ff81f7d08f20c
Author: Ramesh Kumar 
AuthorDate: Fri Jun 9 07:59:44 2023 -0700

HIVE-27293 Vectorization: Incorrect results with nvl for ORC table (#4379)

@ramesh0201 Reviewed by @ayushtkn @rkirtir @difin
---
 data/files/esource.txt |   3 +
 .../queries/clientpositive/vectorization_nvl.q |  50 +++
 .../clientpositive/llap/vectorization_nvl.q.out| 165 +
 .../hive/ql/exec/vector/Decimal64ColumnVector.java |   4 +-
 4 files changed, 220 insertions(+), 2 deletions(-)

diff --git a/data/files/esource.txt b/data/files/esource.txt
new file mode 100644
index 000..e73c3927096
--- /dev/null
+++ b/data/files/esource.txt
@@ -0,0 +1,3 @@
+12345|30001231|9|19613|IND|22|-1.00|abc|20230414|0
+67891|20410308|2|19313|IND|555|-1.00|def|20230414|0
+15678|20370904|1|NULL|IND|444|NULL|xyz|20230414|0
diff --git a/ql/src/test/queries/clientpositive/vectorization_nvl.q 
b/ql/src/test/queries/clientpositive/vectorization_nvl.q
new file mode 100644
index 000..8f2472f9c51
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vectorization_nvl.q
@@ -0,0 +1,50 @@
+drop table if exists esource;
+drop table if exists etarget;
+
+create table if not exists esource_txt (
+ client_id string,
+   id_enddate decimal(10,0), 
+   client_gender string, 
+   birthday decimal(10,0),   
+   nationality string,
+   address_zipcode string,   
+   income decimal(15,2), 
+   address string,
+   part_date int
+) row format delimited fields terminated by '|'
+ lines terminated by '\n' stored as textfile;
+
+create table esource like esource_txt;
+alter table esource set fileformat orc;
+
+load data local inpath '../../data/files/esource.txt' overwrite into table 
esource_txt;
+
+insert overwrite table esource select * from esource_txt;
+
+Select * from esource where part_date = 20230414;
+
+CREATE EXTERNAL TABLE etarget(   
+   client_id string,
+   id_enddate decimal(10,0), 
+   client_gender string, 
+   birthday decimal(10,0),   
+   nationality string,
+   address_zipcode string,   
+   income decimal(15,2), 
+   address string,
+   part_date int,
+   bdata_no int)
+ CLUSTERED BY (
+   bdata_no)   
+ INTO 1 BUCKETS
+ ROW FORMAT SERDE  
+   'org.apache.hadoop.hive.ql.io.orc.OrcSerde' 
+ STORED AS INPUTFORMAT 
+   'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' 
+ OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat';
+
+
+ insert overwrite table etarget
+select mt.*, floor(rand() * 1) as bdata_no from (select nvl(np.client_id,' 
'),nvl(np.id_enddate,cast(0 as decimal(10,0))),nvl(np.client_gender,' 
'),nvl(np.birthday,cast(0 as decimal(10,0))),nvl(np.nationality,' 
'),nvl(np.address_zipcode,' '),nvl(np.income,cast(0 as 
decimal(15,2))),nvl(np.address,' '),nvl(np.part_date,cast(0 as int)) from 
(select * from esource where part_date = 20230414) np) mt;
+
+select client_id,birthday,income from etarget;
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_nvl.q.out 
b/ql/src/test/results/clientpositive/llap/vectorization_nvl.q.out
new file mode 100644
index 000..c0ecef16eff
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vectorization_nvl.q.out
@@ -0,0 +1,165 @@
+PREHOOK: query: drop table if exists esource
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists esource
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists etarget
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists etarget
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table if not exists esource_txt (
+ client_id string,
+   id_enddate decimal(10,0), 
+   client_gender string, 
+   birthday decimal(10,0),   
+   nationality string,
+   address_zipcode string,   
+   income decimal(15,2), 
+   address string,
+   part_date int
+) row format delimited fields terminated by '|'
+ lines terminated by '\n' stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@esource_txt
+POSTHOOK: query: create table if not exists esource_txt (
+ client_id string

[hive] branch master updated: HIVE-26989 Fix predicate pushdown for Timestamp with TZ (#3985)

2023-01-26 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 03fe61f0a76 HIVE-26989 Fix predicate pushdown for Timestamp with TZ 
(#3985)
03fe61f0a76 is described below

commit 03fe61f0a76dd77a085aa3a08bdc2b810d1382a4
Author: Ramesh Kumar 
AuthorDate: Thu Jan 26 10:58:49 2023 -0800

HIVE-26989 Fix predicate pushdown for Timestamp with TZ (#3985)

Ramesh Kumar reviewed by Ayush Saxena
---
 .../TestHiveIcebergStorageHandlerTimezone.java | 24 ++
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java |  4 
 2 files changed, 28 insertions(+)

diff --git 
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java
 
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java
index 122973fb362..865f0856411 100644
--- 
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java
+++ 
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.text.DateFormat;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
 import java.util.Collection;
 import java.util.List;
 import java.util.Optional;
@@ -168,4 +170,26 @@ public class TestHiveIcebergStorageHandlerTimezone {
 result = shell.executeStatement("SELECT * FROM ts_test WHERE 
d_ts='2017-01-01 22:30:57.3'");
 Assert.assertEquals(0, result.size());
   }
+
+  @Test
+  public void testTimestampQueryWithTimeZone() throws IOException {
+Schema timestampSchema = new Schema(optional(1, "d_ts", 
Types.TimestampType.withZone()));
+
+List records = 
TestHelper.RecordsBuilder.newInstance(timestampSchema)
+.add(OffsetDateTime.of(LocalDateTime.of(2019, 1, 22, 9, 44, 54, 
1), ZoneOffset.of("+00")))
+.add(OffsetDateTime.of(LocalDateTime.of(2019, 2, 22, 9, 44, 54, 
2), ZoneOffset.of("+00")))
+.build();
+
+testTables.createTable(shell, "ts_test_tz", timestampSchema, 
FileFormat.PARQUET, records);
+
+List result = shell.executeStatement("SELECT d_ts FROM 
ts_test_tz where d_ts='2019-02-22 09:44:54.200Z'");
+Assert.assertEquals(1, result.size());
+if (timezoneString.equals("America/New_York")) {
+  Assert.assertEquals("2019-02-22 04:44:54.2 " + timezoneString, 
result.get(0)[0]);
+} else if (timezoneString.equals("Asia/Kolkata")) {
+  Assert.assertEquals("2019-02-22 15:14:54.2 " + timezoneString, 
result.get(0)[0]);
+} else if (timezoneString.equals("GMT")) {
+  Assert.assertEquals("2019-02-22 09:44:54.2 " + timezoneString, 
result.get(0)[0]);
+}
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
index 71bf6d7b5e6..d9d3ef653d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -29,6 +29,7 @@ import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -138,6 +139,7 @@ public class ConvertAstToSearchArg {
 case DATE:
   return BoxType.DATE;
 case TIMESTAMP:
+case TIMESTAMPLOCALTZ:
   return BoxType.TIMESTAMP;
 case DECIMAL:
   return BoxType.DECIMAL;
@@ -218,6 +220,8 @@ public class ConvertAstToSearchArg {
 } else if (lit instanceof 
org.apache.hadoop.hive.common.type.Timestamp) {
   ts = ((org.apache.hadoop.hive.common.type.Timestamp) lit)
   .toSqlTimestamp();
+} else if (lit instanceof 
org.apache.hadoop.hive.common.type.TimestampTZ) {
+  ts =  
Timestamp.valueOf(((TimestampTZ)lit).getZonedDateTime().toLocalDateTime());
 } else {
   ts = 
org.apache.hadoop.hive.common.type.Timestamp.valueOf(lit.toString())
   .toSqlTimestamp();



[hive] branch master updated: HIVE-26837 CTLT with hive.create.as.external.legacy as true creates managed table instead of external table (#3854)

2023-01-05 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new c7db232125e HIVE-26837 CTLT with hive.create.as.external.legacy as 
true creates managed table instead of external table (#3854)
c7db232125e is described below

commit c7db232125e753c76861ca816cdf8d5535dbaa7a
Author: Ramesh Kumar 
AuthorDate: Thu Jan 5 10:33:52 2023 -0800

HIVE-26837 CTLT with hive.create.as.external.legacy as true creates managed 
table instead of external table (#3854)
---
 .../apache/hadoop/hive/ql/TestCtltExternalDir.java |  76 +++
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java |   4 +
 .../clientpositive/ctlt_translate_external.q   |  20 
 .../llap/ctlt_translate_external.q.out | 108 +
 4 files changed, 208 insertions(+)

diff --git 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java
 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java
new file mode 100644
index 000..e46b2323a40
--- /dev/null
+++ 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
+import org.apache.hadoop.hive.ql.QTestMiniClusters.MiniClusterType;
+
+import org.junit.Assert;
+import org.junit.Test;
+import static org.junit.Assert.fail;
+
+public class TestCtltExternalDir extends BaseTestQueries {
+  public TestCtltExternalDir() {
+File logDirFile = new File(logDir);
+if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+  fail("Could not create " + logDir);
+}
+  }
+
+  @Test public void testCtltLocation() throws Exception {
+String[] testNames = new String[] { "ctlt_translate_external.q" };
+
+File[] qfiles = setupQFiles(testNames);
+
+String whRootExternal = "/tmp/wh_ext";
+
+QTestUtil qt = new QTestUtil(
+QTestArguments.QTestArgumentsBuilder.instance().withOutDir(resDir + 
"/llap").withLogDir(logDir).withClusterType(MiniClusterType.LLAP_LOCAL).withConfDir(null).withInitScript("").withCleanupScript("")
+.withLlapIo(false).build());
+
+HiveConf hiveConf = qt.getConf();
+hiveConf.setVar(ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL, 
whRootExternal);
+TestTxnDbUtil.setConfValues(hiveConf);
+TestTxnDbUtil.cleanDb(hiveConf);
+TestTxnDbUtil.prepDb(hiveConf);
+qt.postInit();
+qt.newSession();
+qt.setInputFile(qfiles[0]);
+qt.clearTestSideEffects();
+
+boolean success = QTestRunnerUtils.queryListRunnerSingleThreaded(qfiles, 
new QTestUtil[] { qt });
+if (success) {
+  IMetaStoreClient hmsClient = new HiveMetaStoreClient(hiveConf);
+  Table table = hmsClient.getTable("default", "test_ext1");
+  FileSystem fs = FileSystem.get(hiveConf);
+  String location = table.getSd().getLocation();
+  Assert.assertEquals("Not an external table", "file:" + whRootExternal + 
"/test_ext1", location);
+} else {
+  fail("One or more queries failed");
+}
+  }
+}
\ No newline at end of file
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 57702036cdb..41a947dc985 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -13979,6 +13979,10 @@ public class SemanticAnalyzer extends 
BaseSemanticA

[hive] branch master updated: HIVE-23559: Optimise Hive::moveAcidFiles for cloud storage. (#3795)

2023-01-03 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 67906e855da HIVE-23559: Optimise Hive::moveAcidFiles for cloud 
storage. (#3795)
67906e855da is described below

commit 67906e855da46835fe3c4bce22378670efc7b411
Author: Dmitriy Fingerman 
AuthorDate: Tue Jan 3 13:18:56 2023 -0500

HIVE-23559: Optimise Hive::moveAcidFiles for cloud storage. (#3795)

Co-authored-by: Dmitriy Fingerman 
---
 .../org/apache/hadoop/hive/ql/exec/MoveTask.java   |   2 +-
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   | 156 ++---
 2 files changed, 105 insertions(+), 53 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index a5a5ea793d6..555bd7fb5b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -388,7 +388,7 @@ public class MoveTask extends Task implements 
Serializable {
 FileSystem srcFs = sourcePath.getFileSystem(conf);
 FileStatus[] srcs = srcFs.globStatus(sourcePath);
 if(srcs != null) {
-  Hive.moveAcidFiles(srcFs, srcs, targetPath, null);
+  Hive.moveAcidFiles(srcFs, srcs, targetPath, null, conf);
 } else {
   LOG.debug("No files found to move from " + sourcePath + " to " + 
targetPath);
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 238b24d2641..d9301c3ea4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -5102,7 +5102,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
 // If we're moving files around for an ACID write then the rules and paths 
are all different.
 // You can blame this on Owen.
 if (isAcidIUD) {
-  moveAcidFiles(srcFs, srcs, destf, newFiles);
+  moveAcidFiles(srcFs, srcs, destf, newFiles, conf);
 } else {
   // For ACID non-bucketed case, the filenames have to be in the format 
consistent with INSERT/UPDATE/DELETE Ops,
   // i.e, like 00_0, 01_0_copy_1, 02_0.gz etc.
@@ -5124,7 +5124,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   public static void moveAcidFiles(FileSystem fs, FileStatus[] stats, Path dst,
-List newFiles) throws HiveException {
+List newFiles, HiveConf conf) throws 
HiveException {
 // The layout for ACID files is 
table|partname/base|delta|delete_delta/bucket
 // We will always only be writing delta files ( except IOW which writes 
base_X/ ).
 // In the buckets created by FileSinkOperator
@@ -5185,78 +5185,130 @@ private void constructOneLBLocationMap(FileStatus fSta,
   for (FileStatus origBucketStat : origBucketStats) {
 Path origBucketPath = origBucketStat.getPath();
 moveAcidFiles(AcidUtils.DELTA_PREFIX, AcidUtils.deltaFileFilter,
-fs, dst, origBucketPath, createdDeltaDirs, newFiles);
+fs, dst, origBucketPath, createdDeltaDirs, newFiles, conf);
 moveAcidFiles(AcidUtils.DELETE_DELTA_PREFIX, 
AcidUtils.deleteEventDeltaDirFilter,
-fs, dst,origBucketPath, createdDeltaDirs, newFiles);
+fs, dst,origBucketPath, createdDeltaDirs, newFiles, conf);
 moveAcidFiles(AcidUtils.BASE_PREFIX, AcidUtils.baseFileFilter,//for 
Insert Overwrite
-fs, dst, origBucketPath, createdDeltaDirs, newFiles);
+fs, dst, origBucketPath, createdDeltaDirs, newFiles, conf);
   }
 }
   }
 
   private static void moveAcidFiles(String deltaFileType, PathFilter 
pathFilter, FileSystem fs,
 Path dst, Path origBucketPath, Set 
createdDeltaDirs,
-List newFiles) throws HiveException {
-LOG.debug("Acid move looking for " + deltaFileType + " files in bucket " + 
origBucketPath);
+List newFiles, HiveConf conf) throws 
HiveException {
 
-FileStatus[] deltaStats = null;
-try {
-  deltaStats = fs.listStatus(origBucketPath, pathFilter);
-} catch (IOException e) {
-  throw new HiveException("Unable to look for " + deltaFileType + " files 
in original bucket " +
-  origBucketPath.toUri().toString(), e);
-}
-LOG.debug("Acid move found " + deltaStats.length + " " + deltaFileType + " 
files");
-
-for (FileStatus deltaStat : deltaStats) {
-  Path deltaPath = deltaStat.getPath();
-  // Create the delta directory.  Don

[hive] branch master updated: HIVE-26683: Sum windowing function returns wrong value when all nulls. (#3800)

2022-12-05 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 10805bc997d HIVE-26683: Sum windowing function returns wrong value 
when all nulls. (#3800)
10805bc997d is described below

commit 10805bc997d7cd136b85fca9200cf165ffe2eae5
Author: scarlin-cloudera <55709772+scarlin-cloud...@users.noreply.github.com>
AuthorDate: Mon Dec 5 08:58:15 2022 -0800

HIVE-26683: Sum windowing function returns wrong value when all nulls. 
(#3800)

* HIVE-26683: Sum windowing function returns wrong value when all nulls.

The sum windowing function is returning an incorrect value when all the
"following" rows are null.  The correct value for sum when all the rows
are null is "null".

A new member variable had to be added to track for nulls. It uses the
same algorithm that is used for sums. The sums are tracked by keeping
a running sum across all the rows and subtracting off the running sum
outside the window. Likewise, we keep track of a running non null row
count for the current row and subtract the non null row count of the
row that is leaving the window.

* empty
---
 .../hadoop/hive/ql/udf/generic/GenericUDAFSum.java | 106 +++---
 .../clientpositive/windowing_sum_following_null.q  |  30 +
 .../llap/windowing_sum_following_null.q.out| 124 +
 3 files changed, 220 insertions(+), 40 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
index 6ce8734e8f0..40c7a7d7b5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
@@ -139,9 +139,17 @@ public class GenericUDAFSum extends 
AbstractGenericUDAFResolver {
*/
   public static abstract class GenericUDAFSumEvaluator extends GenericUDAFEvaluator {
 static abstract class SumAgg extends AbstractAggregationBuffer {
-  boolean empty;
   T sum;
   HashSet uniqueObjects; // Unique rows.
+  // HIVE-26683: Tracks the number of non null rows. If all the rows are 
null, then the sum of
+  // them is null. The count is needed for tracking in windowing frames. 
Windowing frames
+  // keep a running count of the sum and subtract off entries as the 
window moves. In order
+  // to process nulls within this same framework, we track the number of 
non null rows and
+  // also subtract off the number of entries as the window moves. If the 
current running count
+  // of non null rows is  and the number of non null rows in the entry 
leaving the window
+  // is also  then we know all the entries within the window are null 
and can return null
+  // for the sum.
+  long nonNullCount;
 }
 
 protected PrimitiveObjectInspector inputOI;
@@ -267,9 +275,9 @@ public class GenericUDAFSum extends 
AbstractGenericUDAFResolver {
 @Override
 public void reset(AggregationBuffer agg) throws HiveException {
   SumAgg bdAgg = (SumAgg) agg;
-  bdAgg.empty = true;
   bdAgg.sum = new HiveDecimalWritable(0);
   bdAgg.uniqueObjects = null;
+  bdAgg.nonNullCount = 0;
 }
 
 boolean warned = false;
@@ -279,7 +287,7 @@ public class GenericUDAFSum extends 
AbstractGenericUDAFResolver {
   assert (parameters.length == 1);
   try {
 if (isEligibleValue((SumHiveDecimalWritableAgg) agg, parameters[0])) {
-  ((SumHiveDecimalWritableAgg)agg).empty = false;
+  ((SumHiveDecimalWritableAgg)agg).nonNullCount++;
   ((SumHiveDecimalWritableAgg)agg).sum.mutateAdd(
   PrimitiveObjectInspectorUtils.getHiveDecimal(parameters[0], 
inputOI));
 }
@@ -303,12 +311,12 @@ public class GenericUDAFSum extends 
AbstractGenericUDAFResolver {
   return;
 }
 
-myagg.empty = false;
 if (isWindowingDistinct()) {
   throw new HiveException("Distinct windowing UDAF doesn't support 
merge and terminatePartial");
 } else {
   // If partial is NULL, then there was an overflow and myagg.sum will 
be marked as not set.
   
myagg.sum.mutateAdd(PrimitiveObjectInspectorUtils.getHiveDecimal(partial, 
inputOI));
+  myagg.nonNullCount++;
 }
   }
 }
@@ -316,7 +324,7 @@ public class GenericUDAFSum extends 
AbstractGenericUDAFResolver {
 @Override
 public Object terminate(AggregationBuffer agg) throws HiveException {
   SumHiveDecimalWritableAgg myagg = (SumHiveDecimalWritableAgg) agg;
-  if (myagg.empty || myagg.sum == null || !myagg.sum.isSet()) {
+  if (myagg.nonNullCount == 0 || myagg.sum == null || !myagg.sum.isSet()) {
 return n

[hive] branch master updated: HIVE-26208: Exception in Vectorization with Decimal64 to Decimal casting (#3344)

2022-06-20 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new a694ee7b6da HIVE-26208: Exception in Vectorization with Decimal64 to 
Decimal casting (#3344)
a694ee7b6da is described below

commit a694ee7b6da82a9d11312d32524bebd70d292a94
Author: scarlin-cloudera <55709772+scarlin-cloud...@users.noreply.github.com>
AuthorDate: Mon Jun 20 11:15:04 2022 -0700

HIVE-26208: Exception in Vectorization with Decimal64 to Decimal casting 
(#3344)

The exception was happening because the code that casts the Decimal64 to
Decimal was not being added in filtered expressions.

Once this code was added, it caused a regression in check_constraint.q. The
reason for this was because we do not want to convert Decimal64 to Decimal
if the expression explicitly handles decimal64 types. A method was added
to these classes that will prevent the conversion in these cases.
---
 .../Decimal64ColumnArithmeticDecimal64Column.txt   |  7 +++-
 .../Decimal64ColumnArithmeticDecimal64Scalar.txt   |  5 +++
 ...al64ColumnArithmeticDecimal64ScalarUnscaled.txt |  4 ++
 .../ExpressionTemplates/Decimal64ColumnBetween.txt |  5 +++
 .../Decimal64ColumnCompareDecimal64Column.txt  |  7 +++-
 .../Decimal64ColumnCompareDecimal64Scalar.txt  |  7 +++-
 .../Decimal64ColumnDivideDecimal64Column.txt   |  5 +++
 .../Decimal64ColumnDivideDecimal64Scalar.txt   |  5 +++
 .../ExpressionTemplates/Decimal64ColumnScaleUp.txt |  4 ++
 .../Decimal64ScalarArithmeticDecimal64Column.txt   |  7 +++-
 ...al64ScalarArithmeticDecimal64ColumnUnscaled.txt |  5 +++
 .../Decimal64ScalarCompareDecimal64Column.txt  |  7 +++-
 ...FilterDecimal64ColumnCompareDecimal64Column.txt |  7 +++-
 ...FilterDecimal64ColumnCompareDecimal64Scalar.txt |  7 +++-
 ...FilterDecimal64ScalarCompareDecimal64Column.txt |  7 +++-
 .../vector/expressions/CastLongToDecimal64.java|  5 +++
 .../expressions/ConvertDecimal64ToDecimal.java |  5 +++
 .../vector/expressions/Decimal64ColumnInList.java  |  5 +++
 .../expressions/FilterDecimal64ColumnBetween.java  |  5 +++
 .../expressions/FilterDecimal64ColumnInList.java   |  5 +++
 .../FilterDecimal64ColumnNotBetween.java   |  5 +++
 .../IfExprDecimal64ColumnDecimal64Column.java  |  7 +++-
 .../IfExprDecimal64ColumnDecimal64Scalar.java  |  5 +++
 .../IfExprDecimal64ScalarDecimal64Column.java  |  5 +++
 .../IfExprDecimal64ScalarDecimal64Scalar.java  |  5 +++
 .../exec/vector/expressions/VectorExpression.java  | 12 +-
 .../hive/ql/optimizer/physical/Vectorizer.java |  7 +++-
 .../test/queries/clientpositive/vector_decimal_7.q | 18 +
 .../clientpositive/llap/vector_decimal_7.q.out | 45 ++
 29 files changed, 211 insertions(+), 12 deletions(-)

diff --git 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
index 8f6e6be5903..bd11b5ddd1b 100644
--- 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
+++ 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Column.txt
@@ -177,4 +177,9 @@ public class  extends VectorExpression {
 VectorExpressionDescriptor.InputExpressionType.COLUMN,
 VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
   }
-}
\ No newline at end of file
+
+  @Override
+  public boolean shouldConvertDecimal64ToDecimal() {
+return false;
+  }
+}
diff --git 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
index 5a54fd48820..0bf4e155e39 100644
--- 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
+++ 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64Scalar.txt
@@ -209,4 +209,9 @@ public class  extends VectorExpression {
 VectorExpressionDescriptor.InputExpressionType.COLUMN,
 VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
   }
+
+  @Override
+  public boolean shouldConvertDecimal64ToDecimal() {
+return false;
+  }
 }
diff --git 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64ScalarUnscaled.txt
 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64ScalarUnscaled.txt
index 6573611600b..662baa4edc3 100644
--- 
a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64ScalarUnscaled.txt
+++ 
b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnArithmeticDecimal64ScalarUnscaled.txt
@@ -61,4 +61,8 @@ public class  extends  {
 .setUnscaled(true)

[hive] branch master updated: HIVE-26269 Class cast exception when vectorization is enabled for certain case when cases (#3329)

2022-06-15 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new fdea1bd1ba3 HIVE-26269 Class cast exception when vectorization is 
enabled for certain case when cases (#3329)
fdea1bd1ba3 is described below

commit fdea1bd1ba3c4b2b27ef2bf0a463ca91d4d44653
Author: Ramesh Kumar 
AuthorDate: Wed Jun 15 10:31:44 2022 -0700

HIVE-26269 Class cast exception when vectorization is enabled for certain 
case when cases (#3329)
---
 .../hive/ql/exec/vector/VectorizationContext.java  |  19 +-
 .../queries/clientpositive/vector_case_when_3.q|   9 +
 .../clientpositive/llap/vector_case_when_3.q.out   | 288 +
 3 files changed, 312 insertions(+), 4 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 6a897939819..6d0e4899e68 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -2407,7 +2407,7 @@ import com.google.common.annotations.VisibleForTesting;
 } else if (udf instanceof GenericUDFToString) {
   ve = getCastToString(childExpr, returnType);
 } else if (udf instanceof GenericUDFToDecimal) {
-  ve = getCastToDecimal(childExpr, returnType);
+  ve = getCastToDecimal(childExpr, mode, returnType);
 } else if (udf instanceof GenericUDFToChar) {
   ve = getCastToChar(childExpr, returnType);
 } else if (udf instanceof GenericUDFToVarchar) {
@@ -3232,8 +3232,8 @@ import com.google.common.annotations.VisibleForTesting;
 return null;
   }
 
-  private VectorExpression getCastToDecimal(List childExpr, 
TypeInfo returnType)
-  throws HiveException {
+  private VectorExpression getCastToDecimal(List childExpr, 
VectorExpressionDescriptor.Mode mode,
+  TypeInfo returnType) throws HiveException {
 ExprNodeDesc child = childExpr.get(0);
 String inputType = childExpr.get(0).getTypeString();
 if (child instanceof ExprNodeConstantDesc) {
@@ -3278,7 +3278,18 @@ import com.google.common.annotations.VisibleForTesting;
 int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
 DataTypePhysicalVariation dataTypePhysicalVariation = 
getDataTypePhysicalVariation(colIndex);
 if (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) 
{
-
+  // try to scale up the expression so we can match the return type 
scale
+  if (tryDecimal64Cast && ((DecimalTypeInfo)returnType).precision() <= 
18) {
+List children = new ArrayList<>();
+int scaleDiff = ((DecimalTypeInfo)returnType).scale() - 
((DecimalTypeInfo)childExpr.get(0).getTypeInfo()).scale();
+ExprNodeDesc newConstant = new ExprNodeConstantDesc(new 
DecimalTypeInfo(scaleDiff, 0),
+HiveDecimal.create(POWEROFTENTABLE[scaleDiff]));
+children.add(child);
+children.add(newConstant);
+ExprNodeGenericFuncDesc newScaledExpr = new 
ExprNodeGenericFuncDesc(returnType,
+new GenericUDFOPScaleUpDecimal64(), " ScaleUp ", children);
+return getVectorExpression(newScaledExpr, mode);
+  }
   // Do Decimal64 conversion instead.
   return createDecimal64ToDecimalConversion(colIndex, returnType);
 } else {
diff --git a/ql/src/test/queries/clientpositive/vector_case_when_3.q 
b/ql/src/test/queries/clientpositive/vector_case_when_3.q
new file mode 100644
index 000..35a157a9941
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_case_when_3.q
@@ -0,0 +1,9 @@
+set hive.explain.user=false;
+set hive.fetch.task.conversion=none;
+set hive.vectorized.execution.enabled=true;
+create external table test_decimal(rattag string, newclt_all decimal(15,2)) 
stored as orc;
+insert into test_decimal values('a', '10.20');
+explain vectorization detail select sum(case when rattag='a' then 
newclt_all*0.3 else newclt_all end) from test_decimal;
+select sum(case when rattag='a' then newclt_all*0.3 else newclt_all end) from 
test_decimal;
+explain vectorization detail select sum(case when rattag='Y' then 
newclt_all*0.3 else newclt_all end) from test_decimal;
+select sum(case when rattag='Y' then newclt_all*0.3 else newclt_all end) from 
test_decimal;
diff --git a/ql/src/test/results/clientpositive/llap/vector_case_when_3.q.out 
b/ql/src/test/results/clientpositive/llap/vector_case_when_3.q.out
new file mode 100644
index 000..ddfe1d85aa9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vector_case_when_3.q.out
@@ -0,0 +1,288 @@
+PREHOOK: query: create external table test_decimal(rattag string, newclt_all 
decimal(15,2)) stored as orc
+PREHOOK: type: C

[hive] branch master updated: HIVE-26239 Shutdown Hash table load executor service threads when they are interrupted (#3302)

2022-05-27 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 2a2889a178f HIVE-26239 Shutdown Hash table load executor service 
threads when they are interrupted (#3302)
2a2889a178f is described below

commit 2a2889a178fa0196c4ddccb38b32a9ba2c58b8f9
Author: Ramesh Kumar 
AuthorDate: Fri May 27 08:48:19 2022 -0700

HIVE-26239 Shutdown Hash table load executor service threads when they are 
interrupted (#3302)
---
 .../mapjoin/fast/VectorMapJoinFastHashTableLoader.java   | 16 +++-
 1 file changed, 11 insertions(+), 5 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java
index 0aa4736742d..0a5a4f308d5 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.exec.MemoryMonitorInfo;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError;
+import 
org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hive.common.util.FixedSizedObjectPool;
 import org.apache.tez.common.counters.TezCounter;
 import org.slf4j.Logger;
@@ -294,7 +295,10 @@ public class VectorMapJoinFastHashTableLoader implements 
org.apache.hadoop.hive.
 LOG.info("Finished loading the queue for input: {} waiting {} minutes 
for TPool shutdown", inputName, 2);
 addQueueDoneSentinel();
 loadExecService.shutdown();
-loadExecService.awaitTermination(2, TimeUnit.MINUTES);
+
+if (!loadExecService.awaitTermination(2, TimeUnit.MINUTES)) {
+  throw new HiveException("Failed to complete the hash table loader. 
Loading timed out.");
+}
 batchPool.clear();
 LOG.info("Total received entries: {} Threads {} HT entries: {}", 
receivedEntries, numLoadThreads, totalEntries.get());
 
@@ -314,12 +318,14 @@ public class VectorMapJoinFastHashTableLoader implements 
org.apache.hadoop.hive.
   } catch (InterruptedException e) {
 Thread.currentThread().interrupt();
 throw new HiveException(e);
-  } catch (IOException e) {
-throw new HiveException(e);
-  } catch (SerDeException e) {
-throw new HiveException(e);
+  } catch (HiveException e) {
+throw e;
   } catch (Exception e) {
 throw new HiveException(e);
+  } finally {
+if (loadExecService != null && !loadExecService.isTerminated()) {
+  loadExecService.shutdownNow();
+}
   }
 }
   }



[hive] branch master updated: HIVE-26219 Encapsulate the API change for FileUtils.isActionPermittedForFileHierarchy (#3278)

2022-05-10 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new 4622860b8c HIVE-26219 Encapsulate the API change for 
FileUtils.isActionPermittedForFileHierarchy (#3278)
4622860b8c is described below

commit 4622860b8c7dbddaf4c556e65c5039c60da15e82
Author: Ramesh Kumar 
AuthorDate: Tue May 10 16:17:25 2022 -0700

HIVE-26219 Encapsulate the API change for 
FileUtils.isActionPermittedForFileHierarchy (#3278)
---
 .../src/java/org/apache/hadoop/hive/common/FileUtils.java  | 14 ++
 1 file changed, 14 insertions(+)

diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java 
b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index f3290f69a1..e9410ec4d0 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -532,6 +532,20 @@ public final class FileUtils {
 return isPermitted;
   }
 
+  public static boolean isActionPermittedForFileHierarchy(FileSystem fs, 
FileStatus fileStatus,
+  String userName, FsAction action, boolean recurse) throws Exception {
+UserGroupInformation proxyUser = null;
+boolean isPermitted;
+try {
+  proxyUser = getProxyUser(userName);
+  FileSystem fsAsUser = getFsAsUser(fs, proxyUser);
+  isPermitted = isActionPermittedForFileHierarchy(fs, fileStatus, 
userName, action, recurse, fsAsUser);
+} finally {
+  closeFs(proxyUser);
+}
+return isPermitted;
+  }
+
   @SuppressFBWarnings(value = "DLS_DEAD_LOCAL_STORE", justification = 
"Intended, dir privilege all-around bug")
   public static boolean isActionPermittedForFileHierarchy(FileSystem fs, 
FileStatus fileStatus,
   String userName, FsAction action, boolean recurse, FileSystem fsAsUser) 
throws Exception {



[hive] branch master updated: HVIE-26199 Reduce FileSystem init during user impersonation (#3264)

2022-05-10 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new f298ebc51d HVIE-26199 Reduce FileSystem init during user impersonation 
(#3264)
f298ebc51d is described below

commit f298ebc51d8073e2bcefae8bdf331fab2b91d4d0
Author: Ramesh Kumar 
AuthorDate: Tue May 10 07:44:50 2022 -0700

HVIE-26199 Reduce FileSystem init during user impersonation (#3264)
---
 .../org/apache/hadoop/hive/common/FileUtils.java   | 92 +++---
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   | 16 +++-
 .../plugin/sqlstd/SQLAuthorizationUtils.java   | 20 +++--
 3 files changed, 91 insertions(+), 37 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java 
b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index e92b700195..f3290f69a1 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -67,6 +67,8 @@ import org.apache.hive.common.util.SuppressFBWarnings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.security.auth.login.LoginException;
+
 /**
  * Collection of file manipulation utilities common across Hive.
  */
@@ -408,10 +410,16 @@ public final class FileUtils {
 return getPathOrParentThatExists(fs, parentPath);
   }
 
-  public static void checkFileAccessWithImpersonation(final FileSystem fs, 
final FileStatus stat,
-  final FsAction action, final String user)
-  throws IOException, AccessControlException, InterruptedException, 
Exception {
-checkFileAccessWithImpersonation(fs, stat, action, user, null);
+  public static void checkFileAccessWithImpersonation(final FileSystem fs, 
final FileStatus stat, final FsAction action,
+  final String user) throws Exception {
+UserGroupInformation proxyUser = null;
+try {
+  proxyUser = getProxyUser(user);
+  FileSystem fsAsUser = FileUtils.getFsAsUser(fs, proxyUser);
+  checkFileAccessWithImpersonation(fs, stat, action, user, null, fsAsUser);
+} finally {
+  closeFs(proxyUser);
+}
   }
 
   /**
@@ -435,9 +443,9 @@ public final class FileUtils {
* @throws InterruptedException
* @throws Exception
*/
-  public static void checkFileAccessWithImpersonation(final FileSystem fs,
-  final FileStatus stat, final FsAction action, final String user, final 
List children)
-  throws IOException, AccessControlException, InterruptedException, 
Exception {
+  public static void checkFileAccessWithImpersonation(final FileSystem fs, 
final FileStatus stat, final FsAction action,
+  final String user, final List children, final FileSystem 
fsAsUser)
+  throws IOException, AccessControlException, InterruptedException, 
Exception {
 UserGroupInformation ugi = Utils.getUGI();
 String currentUser = ugi.getShortUserName();
 
@@ -449,25 +457,17 @@ public final class FileUtils {
 }
 
 // Otherwise, try user impersonation. Current user must be configured to 
do user impersonation.
-UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(
-user, UserGroupInformation.getLoginUser());
-try {
-  proxyUser.doAs(new PrivilegedExceptionAction() {
-@Override
-public Object run() throws Exception {
-  FileSystem fsAsUser = FileSystem.get(fs.getUri(), fs.getConf());
-  ShimLoader.getHadoopShims().checkFileAccess(fsAsUser, stat, action);
-  addChildren(fsAsUser, stat.getPath(), children);
-  return null;
-}
-  });
-} finally {
-  FileSystem.closeAllForUGI(proxyUser);
-}
+UserGroupInformation proxyUser = 
UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser());
+proxyUser.doAs(new PrivilegedExceptionAction() {
+  @Override public Object run() throws Exception {
+ShimLoader.getHadoopShims().checkFileAccess(fsAsUser, stat, action);
+addChildren(fsAsUser, stat.getPath(), children);
+return null;
+  }
+});
   }
 
-  private static void addChildren(FileSystem fsAsUser, Path path, 
List children)
-  throws IOException {
+  private static void addChildren(FileSystem fsAsUser, Path path, 
List children) throws IOException {
 if (children != null) {
   FileStatus[] listStatus;
   try {
@@ -480,6 +480,33 @@ public final class FileUtils {
 }
   }
 
+  public static UserGroupInformation getProxyUser(final String user) throws 
LoginException, IOException {
+UserGroupInformation ugi = Utils.getUGI();
+String currentUser = ugi.getShortUserName();
+UserGroupInformation proxyUser = null;
+if (user != null && !user.equals(currentUser)) {
+  proxyUser = UserGroupInformation.createProxyUser(user, 
UserGroupInformation.getLoginUser());
+}
+return p

[hive] branch master updated: HIVE-26096: Select on single column MultiDelimitSerDe table throws AIOBE (#3158)

2022-04-08 Thread rameshkumar
This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
 new d06f891353 HIVE-26096: Select on single column MultiDelimitSerDe table 
throws AIOBE (#3158)
d06f891353 is described below

commit d06f8913533d7a003f85e03a359e3ad50409745e
Author: Naresh P R 
AuthorDate: Fri Apr 8 17:22:47 2022 -0700

HIVE-26096: Select on single column MultiDelimitSerDe table throws AIOBE 
(#3158)
---
 .../queries/clientpositive/serde_multi_delimit.q   |  8 +
 .../clientpositive/llap/serde_multi_delimit.q.out  | 41 ++
 .../apache/hadoop/hive/serde2/lazy/LazyStruct.java |  2 +-
 3 files changed, 50 insertions(+), 1 deletion(-)

diff --git a/ql/src/test/queries/clientpositive/serde_multi_delimit.q 
b/ql/src/test/queries/clientpositive/serde_multi_delimit.q
index e9e7f789d1..c23f264b3b 100644
--- a/ql/src/test/queries/clientpositive/serde_multi_delimit.q
+++ b/ql/src/test/queries/clientpositive/serde_multi_delimit.q
@@ -67,7 +67,15 @@ LOAD DATA LOCAL INPATH 
"../../data/files/t4_multi_delimit.csv" INTO TABLE t4_mul
 
 SELECT * FROM t4_multi_delimit;
 
+create table test_multidelim(col string)
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.MultiDelimitSerDe'
+with serdeproperties('field.delim'='!^') STORED AS TEXTFILE;
+
+insert into test_multidelim values('aa'),('bb'),('cc'),('dd');
+
+select * from test_multidelim;
 
+DROP TABLE test_multidelim;
 DROP TABLE t1_multi_delimit;
 DROP TABLE t11_csv_serde;
 DROP TABLE t2_multi_delimit;
diff --git a/ql/src/test/results/clientpositive/llap/serde_multi_delimit.q.out 
b/ql/src/test/results/clientpositive/llap/serde_multi_delimit.q.out
index 837f62012c..7bb2fdc9e3 100644
--- a/ql/src/test/results/clientpositive/llap/serde_multi_delimit.q.out
+++ b/ql/src/test/results/clientpositive/llap/serde_multi_delimit.q.out
@@ -235,6 +235,47 @@ POSTHOOK: Input: default@t4_multi_delimit

 áűáűáűáű   
 űűűááá óóó
+PREHOOK: query: create table test_multidelim(col string)
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.MultiDelimitSerDe'
+with serdeproperties('field.delim'='!^') STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_multidelim
+POSTHOOK: query: create table test_multidelim(col string)
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.MultiDelimitSerDe'
+with serdeproperties('field.delim'='!^') STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_multidelim
+PREHOOK: query: insert into test_multidelim values('aa'),('bb'),('cc'),('dd')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@test_multidelim
+POSTHOOK: query: insert into test_multidelim values('aa'),('bb'),('cc'),('dd')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@test_multidelim
+POSTHOOK: Lineage: test_multidelim.col SCRIPT []
+PREHOOK: query: select * from test_multidelim
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_multidelim
+ A masked pattern was here 
+POSTHOOK: query: select * from test_multidelim
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_multidelim
+ A masked pattern was here 
+aa
+bb
+cc
+dd
+PREHOOK: query: DROP TABLE test_multidelim
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@test_multidelim
+PREHOOK: Output: default@test_multidelim
+POSTHOOK: query: DROP TABLE test_multidelim
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@test_multidelim
+POSTHOOK: Output: default@test_multidelim
 PREHOOK: query: DROP TABLE t1_multi_delimit
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@t1_multi_delimit
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyStruct.java 
b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyStruct.java
index 66b15374dd..2848d348e3 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyStruct.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyStruct.java
@@ -300,7 +300,7 @@ public class LazyStruct extends 
LazyNonPrimitive 1 && delimitIndexes[i - 1] != -1) {
 int start = delimitIndexes[i - 1] + fieldDelimit.length;
 startPosition[i] = start - i * diff;
   } else {