hive git commit: HIVE-13784: Hive Metastore start failed on Oracle DB (Hari Subramaniyan, reviewed by Sergey Shelukhin)

2016-05-18 Thread harisankar
Repository: hive
Updated Branches:
  refs/heads/master f3fe47ca5 -> ff38cfb76


HIVE-13784: Hive Metastore start failed on Oracle DB (Hari Subramaniyan, 
reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ff38cfb7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ff38cfb7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ff38cfb7

Branch: refs/heads/master
Commit: ff38cfb76e91a21ab10077d7ac62c6828514b0ad
Parents: f3fe47c
Author: Hari Subramaniyan 
Authored: Wed May 18 12:54:25 2016 -0700
Committer: Hari Subramaniyan 
Committed: Wed May 18 12:54:25 2016 -0700

--
 metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql| 2 +-
 metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/ff38cfb7/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
--
diff --git a/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql 
b/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
index 5bbd197..a456e1e 100644
--- a/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/034-HIVE-13076.oracle.sql
@@ -14,4 +14,4 @@ CREATE TABLE IF NOT EXISTS  KEY_CONSTRAINTS
   ENABLE_VALIDATE_RELY NUMBER NOT NULL
 ) ;
 ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY 
(CONSTRAINT_NAME, POSITION);
-CREATE INDEX CONSTRAINTS_PARENT_TBL_ID_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
+CREATE INDEX CONSTRAINTS_PT_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/ff38cfb7/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
--
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql 
b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
index f3dcccd..bdc79ae 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
@@ -795,7 +795,7 @@ CREATE TABLE KEY_CONSTRAINTS
 
 ALTER TABLE KEY_CONSTRAINTS ADD CONSTRAINT CONSTRAINTS_PK PRIMARY KEY 
(CONSTRAINT_NAME, POSITION);
 
-CREATE INDEX CONSTRAINTS_PARENT_TBL_ID_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
+CREATE INDEX CONSTRAINTS_PT_INDEX ON KEY_CONSTRAINTS(PARENT_TBL_ID);
 
 
 --



hive git commit: HIVE-13343: Need to disable hybrid grace hash join in llap mode except for dynamically partitioned hash join (Vikram Dixit K, reviewed by Gunther Hagleitner)

2016-05-18 Thread vikram
Repository: hive
Updated Branches:
  refs/heads/master 4959ff5bb -> f3fe47ca5


HIVE-13343: Need to disable hybrid grace hash join in llap mode except for 
dynamically partitioned hash join (Vikram Dixit K, reviewed by Gunther 
Hagleitner)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f3fe47ca
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f3fe47ca
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f3fe47ca

Branch: refs/heads/master
Commit: f3fe47ca584e332c3e4bacc0a2e13ecbf7f32fea
Parents: 4959ff5
Author: vikram 
Authored: Wed May 18 10:47:11 2016 -0700
Committer: vikram 
Committed: Wed May 18 10:48:11 2016 -0700

--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  2 +
 .../hive/ql/optimizer/physical/LlapDecider.java | 40 
 .../clientpositive/hybridgrace_hashjoin_1.q |  2 +
 .../llap/bucket_map_join_tez1.q.out | 21 --
 .../llap/bucket_map_join_tez2.q.out |  8 
 .../llap/dynamic_partition_pruning.q.out| 18 -
 .../llap/dynamic_partition_pruning_2.q.out  |  7 
 .../llap/hybridgrace_hashjoin_1.q.out   |  5 +--
 .../llap/hybridgrace_hashjoin_2.q.out   |  6 ---
 .../clientpositive/llap/lvj_mapjoin.q.out   |  2 -
 .../clientpositive/llap/mapjoin_decimal.q.out   |  1 -
 .../test/results/clientpositive/llap/mrr.q.out  |  2 -
 .../llap/tez_bmj_schema_evolution.q.out |  1 -
 .../llap/tez_join_result_complex.q.out  | 17 -
 .../clientpositive/llap/tez_smb_main.q.out  | 11 --
 .../results/clientpositive/llap/tez_union.q.out | 14 ---
 .../vectorized_dynamic_partition_pruning.q.out  | 17 -
 .../tez/hybridgrace_hashjoin_1.q.out|  2 +
 18 files changed, 56 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/f3fe47ca/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
--
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index cbb3a72..9cc8fbe 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2852,6 +2852,8 @@ public class HiveConf extends Configuration {
 "protocol or ZK paths), similar to how ssh refuses a key with bad 
access permissions."),
 LLAP_DAEMON_OUTPUT_SERVICE_PORT("hive.llap.daemon.output.service.port", 
15003,
 "LLAP daemon output service port"),
+LLAP_ENABLE_GRACE_JOIN_IN_LLAP("hive.llap.enable.grace.join.in.llap", 
false,
+"Override if grace join should be allowed to run in llap."),
 
 SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
   "60s", new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/f3fe47ca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
index e6f8f75..4d50c64 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ScriptOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
@@ -105,6 +106,7 @@ public class LlapDecider implements PhysicalPlanResolver {
 private final boolean doSkipUdfCheck;
 private final boolean arePermanentFnsAllowed;
 private final boolean shouldUber;
+private List mapJoinOpList;
 
 public LlapDecisionDispatcher(PhysicalContext pctx, LlapMode mode) {
   conf = pctx.getConf();
@@ -112,6 +114,7 @@ public class LlapDecider implements PhysicalPlanResolver {
   arePermanentFnsAllowed = HiveConf.getBoolVar(conf, 
ConfVars.LLAP_ALLOW_PERMANENT_FNS);
   // Don't user uber in "all" mode - everything can go into LLAP, which is 
better than uber.
   shouldUber = HiveConf.getBoolVar(conf, ConfVars.LLAP_AUTO_ALLOW_UBER) && 
(mode != all);
+  mapJoinOpList = new ArrayList();
 }
 
 @Override
@@ -130,17 +133,21 @@ public class LlapDecider implements PhysicalPlanResolver {
 
 private void handleWork(TezWork tezWork, BaseWork work)
   throws SemanticE

hive git commit: HIVE-13691 No record with CQ_ID=0 found in COMPACTION_QUEUE (Eugene Koifman, reviewed by Wei Zheng)

2016-05-18 Thread ekoifman
Repository: hive
Updated Branches:
  refs/heads/branch-1 c9fb65be9 -> 54ba148d6


HIVE-13691 No record with CQ_ID=0 found in COMPACTION_QUEUE (Eugene Koifman, 
reviewed by Wei Zheng)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/54ba148d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/54ba148d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/54ba148d

Branch: refs/heads/branch-1
Commit: 54ba148d633456c2743293dd9ad908ede0a9accd
Parents: c9fb65b
Author: Eugene Koifman 
Authored: Wed May 18 10:47:07 2016 -0700
Committer: Eugene Koifman 
Committed: Wed May 18 10:47:07 2016 -0700

--
 .../metastore/txn/CompactionTxnHandler.java | 27 +--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 35 
 .../hadoop/hive/ql/txn/compactor/Initiator.java |  5 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java | 84 +---
 4 files changed, 98 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/54ba148d/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
--
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index 8bb8154..f8798b7 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -692,7 +692,7 @@ class CompactionTxnHandler extends TxnHandler {
   }
 
   /**
-   * For any given compactable entity (partition, table if not partitioned) 
the history of compactions
+   * For any given compactable entity (partition; table if not partitioned) 
the history of compactions
* may look like "sssfffaaasffss", for example.  The idea is to retain the 
tail (most recent) of the
* history such that a configurable number of each type of state is present. 
 Any other entries
* can be purged.  This scheme has advantage of always retaining the last 
failure/success even if
@@ -794,7 +794,7 @@ class CompactionTxnHandler extends TxnHandler {
   "CC_DATABASE = " + quoteString(ci.dbname) + " and " +
   "CC_TABLE = " + quoteString(ci.tableName) +
   (ci.partName != null ? "and CC_PARTITION = " + 
quoteString(ci.partName) : "") +
-  " order by CC_ID desc");
+  " and CC_STATE != " + quoteChar(ATTEMPTED_STATE) + " order by CC_ID 
desc");
 int numFailed = 0;
 int numTotal = 0;
 int failedThreshold = 
conf.getIntVar(HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD);
@@ -825,8 +825,8 @@ class CompactionTxnHandler extends TxnHandler {
   /**
* If there is an entry in compaction_queue with ci.id, remove it
* Make entry in completed_compactions with status 'f'.
-   *
-   * but what abount markCleaned() which is called when table is had been 
deleted...
+   * If there is no entry in compaction_queue, it means Initiator failed to 
even schedule a compaction,
+   * which we record as ATTEMPTED_STATE entry in history.
*/
   public void markFailed(CompactionInfo ci) throws MetaException {//todo: this 
should not throw
 //todo: this should take "comment" as parameter to set in CC_META_INFO to 
provide some context for the failure
@@ -846,12 +846,27 @@ class CompactionTxnHandler extends TxnHandler {
   int updCnt = stmt.executeUpdate(s);
 }
 else {
-  throw new IllegalStateException("No record with CQ_ID=" + ci.id + " 
found in COMPACTION_QUEUE");
+  if(ci.id > 0) {
+//the record with valid CQ_ID has disappeared - this is a sign of 
something wrong
+throw new IllegalStateException("No record with CQ_ID=" + ci.id + 
" found in COMPACTION_QUEUE");
+  }
+}
+if(ci.id == 0) {
+  //The failure occurred before we even made an entry in 
COMPACTION_QUEUE
+  //generate ID so that we can make an entry in COMPLETED_COMPACTIONS
+  ci.id = generateCompactionQueueId(stmt);
+  //mostly this indicates that the Initiator is paying attention to 
some table even though
+  //compactions are not happening.
+  ci.state = ATTEMPTED_STATE;
+  //this is not strictly accurate, but 'type' cannot be null.
+  ci.type = CompactionType.MINOR;
+}
+else {
+  ci.state = FAILED_STATE;
 }
 close(rs, stmt, null);
 
 pStmt = dbConn.prepareStatement("insert into 
COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, 
CC_TYPE, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, 
CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?)")

hive git commit: HIVE-13691 No record with CQ_ID=0 found in COMPACTION_QUEUE (Eugene Koifman, reviewed by Wei Zheng)

2016-05-18 Thread ekoifman
Repository: hive
Updated Branches:
  refs/heads/master 3726ce590 -> 4959ff5bb


HIVE-13691 No record with CQ_ID=0 found in COMPACTION_QUEUE (Eugene Koifman, 
reviewed by Wei Zheng)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4959ff5b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4959ff5b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4959ff5b

Branch: refs/heads/master
Commit: 4959ff5bb7dc590e21f680b9d9be0f2270414309
Parents: 3726ce5
Author: Eugene Koifman 
Authored: Wed May 18 10:36:45 2016 -0700
Committer: Eugene Koifman 
Committed: Wed May 18 10:36:45 2016 -0700

--
 .../metastore/txn/CompactionTxnHandler.java | 27 +--
 .../hadoop/hive/metastore/txn/TxnHandler.java   | 36 +
 .../hadoop/hive/ql/txn/compactor/Initiator.java |  5 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java | 84 +---
 4 files changed, 99 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/4959ff5b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
--
diff --git 
a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
 
b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index ab7da68..d2d6462 100644
--- 
a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ 
b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -691,7 +691,7 @@ class CompactionTxnHandler extends TxnHandler {
   }
 
   /**
-   * For any given compactable entity (partition, table if not partitioned) 
the history of compactions
+   * For any given compactable entity (partition; table if not partitioned) 
the history of compactions
* may look like "sssfffaaasffss", for example.  The idea is to retain the 
tail (most recent) of the
* history such that a configurable number of each type of state is present. 
 Any other entries
* can be purged.  This scheme has advantage of always retaining the last 
failure/success even if
@@ -793,7 +793,7 @@ class CompactionTxnHandler extends TxnHandler {
   "CC_DATABASE = " + quoteString(ci.dbname) + " and " +
   "CC_TABLE = " + quoteString(ci.tableName) +
   (ci.partName != null ? "and CC_PARTITION = " + 
quoteString(ci.partName) : "") +
-  " order by CC_ID desc");
+  " and CC_STATE != " + quoteChar(ATTEMPTED_STATE) + " order by CC_ID 
desc");
 int numFailed = 0;
 int numTotal = 0;
 int failedThreshold = 
conf.getIntVar(HiveConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD);
@@ -824,8 +824,8 @@ class CompactionTxnHandler extends TxnHandler {
   /**
* If there is an entry in compaction_queue with ci.id, remove it
* Make entry in completed_compactions with status 'f'.
-   *
-   * but what abount markCleaned() which is called when table is had been 
deleted...
+   * If there is no entry in compaction_queue, it means Initiator failed to 
even schedule a compaction,
+   * which we record as ATTEMPTED_STATE entry in history.
*/
   public void markFailed(CompactionInfo ci) throws MetaException {//todo: this 
should not throw
 //todo: this should take "comment" as parameter to set in CC_META_INFO to 
provide some context for the failure
@@ -845,12 +845,27 @@ class CompactionTxnHandler extends TxnHandler {
   int updCnt = stmt.executeUpdate(s);
 }
 else {
-  throw new IllegalStateException("No record with CQ_ID=" + ci.id + " 
found in COMPACTION_QUEUE");
+  if(ci.id > 0) {
+//the record with valid CQ_ID has disappeared - this is a sign of 
something wrong
+throw new IllegalStateException("No record with CQ_ID=" + ci.id + 
" found in COMPACTION_QUEUE");
+  }
+}
+if(ci.id == 0) {
+  //The failure occurred before we even made an entry in 
COMPACTION_QUEUE
+  //generate ID so that we can make an entry in COMPLETED_COMPACTIONS
+  ci.id = generateCompactionQueueId(stmt);
+  //mostly this indicates that the Initiator is paying attention to 
some table even though
+  //compactions are not happening.
+  ci.state = ATTEMPTED_STATE;
+  //this is not strictly accurate, but 'type' cannot be null.
+  ci.type = CompactionType.MINOR;
+}
+else {
+  ci.state = FAILED_STATE;
 }
 close(rs, stmt, null);
 
 pStmt = dbConn.prepareStatement("insert into 
COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, 
CC_TYPE, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, 
CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?)");
-

hive git commit: HIVE-13730 : Avoid double spilling the same partition when memory threshold is set very low (Wei Zheng, reviewed by Vikram Dixit K)

2016-05-18 Thread weiz
Repository: hive
Updated Branches:
  refs/heads/master 8c4b99a4e -> 3726ce590


HIVE-13730 : Avoid double spilling the same partition when memory threshold is 
set very low (Wei Zheng, reviewed by Vikram Dixit K)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3726ce59
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3726ce59
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3726ce59

Branch: refs/heads/master
Commit: 3726ce590f9dcb0e679ed6faaafa1211e9f881d3
Parents: 8c4b99a
Author: Wei Zheng 
Authored: Wed May 18 09:51:31 2016 -0700
Committer: Wei Zheng 
Committed: Wed May 18 09:51:31 2016 -0700

--
 .../persistence/HybridHashTableContainer.java   | 22 +++-
 1 file changed, 21 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/3726ce59/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index 5552dfb..bb35bae 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -385,6 +385,11 @@ public class HybridHashTableContainer
 memoryUsed += hashPartitions[i].hashMap.memorySize();
   }
 }
+
+if (writeBufferSize * (numPartitions - numPartitionsSpilledOnCreation) > 
memoryThreshold) {
+  LOG.error("There is not enough memory to allocate " +
+  (numPartitions - numPartitionsSpilledOnCreation) + " hash 
partitions.");
+}
 assert numPartitionsSpilledOnCreation != numPartitions : "All partitions 
are directly spilled!" +
 " It is not supported now.";
 LOG.info("Number of partitions created: " + numPartitions);
@@ -558,7 +563,7 @@ public class HybridHashTableContainer
* @return the biggest partition number
*/
   private int biggestPartition() {
-int res = 0;
+int res = -1;
 int maxSize = 0;
 
 // If a partition has been spilled to disk, its size will be 0, i.e. it 
won't be picked
@@ -574,6 +579,17 @@ public class HybridHashTableContainer
 res = i;
   }
 }
+
+// It can happen that although there're some partitions in memory, but 
their sizes are all 0.
+// In that case we just pick one and spill.
+if (res == -1) {
+  for (int i = 0; i < hashPartitions.length; i++) {
+if (!isOnDisk(i)) {
+  return i;
+}
+  }
+}
+
 return res;
   }
 
@@ -585,6 +601,10 @@ public class HybridHashTableContainer
   public long spillPartition(int partitionId) throws IOException {
 HashPartition partition = hashPartitions[partitionId];
 int inMemRowCount = partition.hashMap.getNumValues();
+if (inMemRowCount == 0) {
+  LOG.warn("Trying to spill an empty hash partition! It may be due to " +
+  "hive.auto.convert.join.noconditionaltask.size being set too low.");
+}
 
 File file = FileUtils.createLocalDirsTempFile(
 spillLocalDirs, "partition-" + partitionId + "-", null, false);