You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2017/01/27 01:18:45 UTC
[01/26] phoenix git commit: PHOENIX-3540 Fix Time data type in
Phoenix Spark integration
Repository: phoenix
Updated Branches:
refs/heads/calcite 948d009f9 -> 405499047
PHOENIX-3540 Fix Time data type in Phoenix Spark integration
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/bd2acd54
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/bd2acd54
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/bd2acd54
Branch: refs/heads/calcite
Commit: bd2acd5404ce03fb330a72bbf346546b7f4fbd2b
Parents: 108b78d
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Dec 22 13:17:20 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Dec 22 13:17:20 2016 +0530
----------------------------------------------------------------------
phoenix-spark/src/it/resources/globalSetup.sql | 2 ++
.../org/apache/phoenix/spark/PhoenixSparkIT.scala | 12 ++++++++++++
.../scala/org/apache/phoenix/spark/PhoenixRDD.scala | 15 ++++++++-------
3 files changed, 22 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/it/resources/globalSetup.sql
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/resources/globalSetup.sql b/phoenix-spark/src/it/resources/globalSetup.sql
index 852687e..72f8620 100644
--- a/phoenix-spark/src/it/resources/globalSetup.sql
+++ b/phoenix-spark/src/it/resources/globalSetup.sql
@@ -48,6 +48,8 @@ CREATE TABLE TEST_SMALL_TINY (ID BIGINT NOT NULL PRIMARY KEY, COL1 SMALLINT, COL
UPSERT INTO TEST_SMALL_TINY VALUES (1, 32767, 127)
CREATE TABLE DATE_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 DATE)
UPSERT INTO DATE_TEST VALUES(1, CURRENT_DATE())
+CREATE TABLE TIME_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 TIME)
+UPSERT INTO TIME_TEST VALUES(1, CURRENT_TIME())
CREATE TABLE "space" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR)
UPSERT INTO "space" VALUES ('key1', 'xyz')
CREATE TABLE "small" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR, "salary" INTEGER )
http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 8aeba09..dbcc4f1 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -621,4 +621,16 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
assert(Math.abs(epoch - ts) < 300000)
}
+
+ test("Can load Phoenix Time columns through DataFrame API") {
+ val sqlContext = new SQLContext(sc)
+ val df = sqlContext.read
+ .format("org.apache.phoenix.spark")
+ .options(Map("table" -> "TIME_TEST", "zkUrl" -> quorumAddress))
+ .load
+ val time = df.select("COL1").first().getTimestamp(0).getTime
+ val epoch = new Date().getTime
+ assert(Math.abs(epoch - time) < 86400000)
+ }
+
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 505de1b..204a7ef 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -130,13 +130,14 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
val rowSeq = columns.map { case (name, sqlType) =>
val res = pr.resultMap(name)
- // Special handling for data types
- if(dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
- new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
- }
- else {
- res
- }
+ // Special handling for data types
+ if (dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
+ new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
+ } else if (sqlType == 92) { // 92 is the defined type for Time
+ new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime)
+ } else {
+ res
+ }
}
// Create a Spark Row from the sequence
[10/26] phoenix git commit: PHOENIX-3333 Support Spark 2.0
Posted by ma...@apache.org.
PHOENIX-3333 Support Spark 2.0
Note that the default maven profile will compile for Spark 2.0.2
and Scala 2.11. To switch to the previous behavior, use Maven
profile 'spark16'.
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/a0e5efce
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/a0e5efce
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/a0e5efce
Branch: refs/heads/calcite
Commit: a0e5efcec5a1a732b2dce9794251242c3d66eea6
Parents: 07f9273
Author: Josh Mahonin <jm...@gmail.com>
Authored: Tue Jan 3 09:57:46 2017 -0500
Committer: Josh Mahonin <jm...@gmail.com>
Committed: Tue Jan 3 09:57:46 2017 -0500
----------------------------------------------------------------------
.../apache/phoenix/spark/PhoenixSparkIT.scala | 26 ++++++++++++++++----
.../phoenix/spark/DataFrameFunctions.scala | 6 ++---
.../org/apache/phoenix/spark/PhoenixRDD.scala | 6 ++---
.../phoenix/spark/ProductRDDFunctions.scala | 3 +--
pom.xml | 14 ++++++++---
5 files changed, 39 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 9def354..d53b5ee 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -302,11 +302,21 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
// Load TABLE1, save as TABLE1_COPY
val sqlContext = new SQLContext(sc)
- val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
- "zkUrl" -> quorumAddress))
+ val df = sqlContext
+ .read
+ .format("org.apache.phoenix.spark")
+ .option("table", "TABLE1")
+ .option("zkUrl", quorumAddress)
+ .load()
// Save to TABLE21_COPY
- df.save("org.apache.phoenix.spark", SaveMode.Overwrite, Map("table" -> "TABLE1_COPY", "zkUrl" -> quorumAddress))
+ df
+ .write
+ .format("org.apache.phoenix.spark")
+ .mode(SaveMode.Overwrite)
+ .option("table", "TABLE1_COPY")
+ .option("zkUrl", quorumAddress)
+ .save()
// Verify results
stmt = conn.createStatement()
@@ -632,12 +642,18 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
val epoch = new Date().getTime
assert(Math.abs(epoch - time) < 86400000)
}
+
test("can read all Phoenix data types") {
val sqlContext = new SQLContext(sc)
val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "GIGANTIC_TABLE",
"zkUrl" -> quorumAddress))
- df.save("org.apache.phoenix.spark",SaveMode.Overwrite, Map("table" -> "OUTPUT_GIGANTIC_TABLE",
- "zkUrl" -> quorumAddress))
+
+ df.write
+ .format("org.apache.phoenix.spark")
+ .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", "zkUrl" -> quorumAddress))
+ .mode(SaveMode.Overwrite)
+ .save()
+
df.count() shouldEqual 1
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
index bb2efd5..ddf4fab 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
@@ -18,12 +18,12 @@ import org.apache.hadoop.io.NullWritable
import org.apache.phoenix.mapreduce.PhoenixOutputFormat
import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
import org.apache.phoenix.util.SchemaUtil
-import org.apache.spark.Logging
import org.apache.spark.sql.DataFrame
import scala.collection.JavaConversions._
-class DataFrameFunctions(data: DataFrame) extends Logging with Serializable {
+
+class DataFrameFunctions(data: DataFrame) extends Serializable {
def saveToPhoenix(tableName: String, conf: Configuration = new Configuration,
zkUrl: Option[String] = None, tenantId: Option[String] = None): Unit = {
@@ -39,7 +39,7 @@ class DataFrameFunctions(data: DataFrame) extends Logging with Serializable {
val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig)
// Map the row objects into PhoenixRecordWritable
- val phxRDD = data.mapPartitions{ rows =>
+ val phxRDD = data.rdd.mapPartitions{ rows =>
// Create a within-partition config to retrieve the ColumnInfo list
@transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrlFinal, tenantId)
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 63289a0..01a9077 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -37,7 +37,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
@transient conf: Configuration, dateAsTimestamp: Boolean = false,
tenantId: Option[String] = None
)
- extends RDD[PhoenixRecordWritable](sc, Nil) with Logging {
+ extends RDD[PhoenixRecordWritable](sc, Nil) {
// Make sure to register the Phoenix driver
DriverManager.registerDriver(new PhoenixDriver)
@@ -163,7 +163,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
case t if t.isInstanceOf[PDouble] || t.isInstanceOf[PUnsignedDouble] => DoubleType
// Use Spark system default precision for now (explicit to work with < 1.5)
case t if t.isInstanceOf[PDecimal] =>
- if (columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale)
+ if (columnInfo.getPrecision == null || columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale)
case t if t.isInstanceOf[PTimestamp] || t.isInstanceOf[PUnsignedTimestamp] => TimestampType
case t if t.isInstanceOf[PTime] || t.isInstanceOf[PUnsignedTime] => TimestampType
case t if (t.isInstanceOf[PDate] || t.isInstanceOf[PUnsignedDate]) && !dateAsTimestamp => DateType
@@ -180,7 +180,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
case t if t.isInstanceOf[PFloatArray] || t.isInstanceOf[PUnsignedFloatArray] => ArrayType(FloatType, containsNull = true)
case t if t.isInstanceOf[PDoubleArray] || t.isInstanceOf[PUnsignedDoubleArray] => ArrayType(DoubleType, containsNull = true)
case t if t.isInstanceOf[PDecimalArray] => ArrayType(
- if (columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale), containsNull = true)
+ if (columnInfo.getPrecision == null || columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale), containsNull = true)
case t if t.isInstanceOf[PTimestampArray] || t.isInstanceOf[PUnsignedTimestampArray] => ArrayType(TimestampType, containsNull = true)
case t if t.isInstanceOf[PDateArray] || t.isInstanceOf[PUnsignedDateArray] => ArrayType(TimestampType, containsNull = true)
case t if t.isInstanceOf[PTimeArray] || t.isInstanceOf[PUnsignedTimeArray] => ArrayType(TimestampType, containsNull = true)
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
index b59592b..9b368b6 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
@@ -17,12 +17,11 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.NullWritable
import org.apache.phoenix.mapreduce.PhoenixOutputFormat
import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
-import org.apache.spark.Logging
import org.apache.spark.rdd.RDD
import scala.collection.JavaConversions._
-class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Logging with Serializable {
+class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Serializable {
def saveToPhoenix(tableName: String, cols: Seq[String],
conf: Configuration = new Configuration, zkUrl: Option[String] = None, tenantId: Option[String] = None)
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f27f2cc..4a0292d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,9 +99,9 @@
<avatica.version>1.9.0</avatica.version>
<jettyVersion>8.1.7.v20120910</jettyVersion>
<tephra.version>0.9.0-incubating</tephra.version>
- <spark.version>1.6.1</spark.version>
- <scala.version>2.10.4</scala.version>
- <scala.binary.version>2.10</scala.binary.version>
+ <spark.version>2.0.2</spark.version>
+ <scala.version>2.11.8</scala.version>
+ <scala.binary.version>2.11</scala.binary.version>
<!-- Test Dependencies -->
<mockito-all.version>1.8.5</mockito-all.version>
@@ -982,5 +982,13 @@
</plugins>
</build>
</profile>
+ <profile>
+ <id>spark16</id>
+ <properties>
+ <spark.version>1.6.1</spark.version>
+ <scala.version>2.10.4</scala.version>
+ <scala.binary.version>2.10</scala.binary.version>
+ </properties>
+ </profile>
</profiles>
</project>
[14/26] phoenix git commit: PHOENIX-3584 Expose metrics for
ConnectionQueryServices instances and their allocators in the JVM
Posted by ma...@apache.org.
PHOENIX-3584 Expose metrics for ConnectionQueryServices instances and their allocators in the JVM
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/f5de28b6
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/f5de28b6
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/f5de28b6
Branch: refs/heads/calcite
Commit: f5de28b6d79141c555c994a94af7b4078872d845
Parents: d8f4594
Author: Samarth <sa...@salesforce.com>
Authored: Tue Jan 10 17:36:42 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Jan 10 17:36:42 2017 -0800
----------------------------------------------------------------------
.../phoenix/monitoring/PhoenixMetricsIT.java | 135 +++++++++++++++++++
.../phoenix/monitoring/GlobalClientMetrics.java | 6 +-
.../apache/phoenix/monitoring/MetricType.java | 4 +-
.../query/ConnectionQueryServicesImpl.java | 10 ++
4 files changed, 153 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
index 3af8ce7..16a66df 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
@@ -10,12 +10,14 @@
package org.apache.phoenix.monitoring;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BATCH_SIZE;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BYTES;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_COMMIT_TIME;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_SQL_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_NUM_PARALLEL_SCANS;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_OPEN_PHOENIX_CONNECTIONS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIME;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_REJECTED_TASK_COUNTER;
@@ -28,6 +30,7 @@ import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
import static org.apache.phoenix.monitoring.MetricType.SCAN_BYTES;
import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTION_TIME;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
import static org.apache.phoenix.util.PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -44,10 +47,16 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.phoenix.compile.StatementContext;
import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
import org.apache.phoenix.jdbc.PhoenixResultSet;
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.util.PhoenixRuntime;
@@ -67,6 +76,8 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
.newArrayList(MetricType.MUTATION_COMMIT_TIME.name());
private static final List<String> readMetricsToSkip = Lists.newArrayList(MetricType.TASK_QUEUE_WAIT_TIME.name(),
MetricType.TASK_EXECUTION_TIME.name(), MetricType.TASK_END_TO_END_TIME.name());
+ private static final String CUSTOM_URL_STRING = "SESSION";
+ private static final AtomicInteger numConnections = new AtomicInteger(0);
@BeforeClass
public static void doSetup() throws Exception {
@@ -76,6 +87,8 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
// disable renewing leases as this will force spooling to happen.
props.put(QueryServices.RENEW_LEASE_ENABLED, String.valueOf(false));
setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+ // need the non-test driver for some tests that check number of hconnections, etc.
+ DriverManager.registerDriver(PhoenixDriver.INSTANCE);
}
@Test
@@ -827,5 +840,127 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
}
}
}
+
+ @Test
+ public void testGetConnectionsForSameUrlConcurrently() throws Exception {
+ // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+ String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+ String url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+ ExecutorService exec = Executors.newFixedThreadPool(10);
+ try {
+ GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+ GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+ assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ List<Callable<Connection>> callables = new ArrayList<>(100);
+ List<Future<Connection>> futures = new ArrayList<>(100);
+ int expectedHConnections = numConnections.get() > 0 ? 0 : 1;
+ for (int i = 1; i <= 100; i++) {
+ Callable<Connection> c = new GetConnectionCallable(url);
+ callables.add(c);
+ futures.add(exec.submit(c));
+ }
+ for (int i = 0; i < futures.size(); i++) {
+ Connection c = futures.get(i).get();
+ try {
+ c.close();
+ } catch (Exception ignore) {}
+ }
+ assertEquals(expectedHConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(expectedHConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ } finally {
+ exec.shutdownNow();
+ }
+ }
+
+ @Test
+ public void testGetConnectionsForDifferentTenantsConcurrently() throws Exception {
+ // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+ String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+ String url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+ ExecutorService exec = Executors.newFixedThreadPool(10);
+ try {
+ GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+ GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+ assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ int expectedHConnections = numConnections.get() > 0 ? 0 : 1;
+ List<Callable<Connection>> callables = new ArrayList<>(100);
+ List<Future<Connection>> futures = new ArrayList<>(100);
+ for (int i = 1; i <= 100; i++) {
+ String tenantUrl = url + ';' + TENANT_ID_ATTRIB + '=' + i;
+ Callable<Connection> c = new GetConnectionCallable(tenantUrl + ";");
+ callables.add(c);
+ futures.add(exec.submit(c));
+ }
+ for (int i = 0; i < futures.size(); i++) {
+ Connection c = futures.get(i).get();
+ try {
+ c.close();
+ } catch (Exception ignore) {}
+ }
+ assertEquals(expectedHConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(expectedHConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ } finally {
+ exec.shutdownNow();
+ }
+ }
+
+ @Test
+ public void testGetConnectionsWithDifferentJDBCParamsConcurrently() throws Exception {
+ DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+ ExecutorService exec = Executors.newFixedThreadPool(4);
+ // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+ String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+ String baseUrl = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+ int numConnections = 20;
+ List<Callable<Connection>> callables = new ArrayList<>(numConnections);
+ List<Future<Connection>> futures = new ArrayList<>(numConnections);
+ try {
+ GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+ GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+ assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ for (int i = 1; i <= numConnections; i++) {
+ String customUrl = baseUrl + ':' + CUSTOM_URL_STRING + '=' + i;
+ Callable<Connection> c = new GetConnectionCallable(customUrl + ";");
+ callables.add(c);
+ futures.add(exec.submit(c));
+ }
+ for (int i = 0; i < futures.size(); i++) {
+ futures.get(i).get();
+ }
+ assertEquals(numConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+ assertEquals(numConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+ } finally {
+ exec.shutdownNow();
+ for (int i = 0; i < futures.size(); i++) {
+ try {
+ Connection c = futures.get(i).get();
+ // close the query services instance because we created a lot of HConnections.
+ c.unwrap(PhoenixConnection.class).getQueryServices().close();
+ c.close();
+ } catch (Exception ignore) {}
+ }
+ }
+ }
+
+ private static class GetConnectionCallable implements Callable<Connection> {
+ private final String url;
+ GetConnectionCallable(String url) {
+ this.url = url;
+ }
+ @Override
+ public Connection call() throws Exception {
+ Connection c = DriverManager.getConnection(url);
+ if (!url.contains(CUSTOM_URL_STRING)) {
+ // check to detect whether a connection was established using the PhoenixDriver
+ // This is used in our tests to figure out whether a new hconnection and query
+ // services will be created.
+ numConnections.incrementAndGet();
+ }
+ return c;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
index c3a7261..fab4d27 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
@@ -17,6 +17,7 @@
*/
package org.apache.phoenix.monitoring;
+import static org.apache.phoenix.monitoring.MetricType.HCONNECTIONS_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.OPEN_PHOENIX_CONNECTIONS_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.QUERY_FAILED_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
@@ -26,6 +27,7 @@ import static org.apache.phoenix.monitoring.MetricType.MUTATION_BYTES;
import static org.apache.phoenix.monitoring.MetricType.MUTATION_COMMIT_TIME;
import static org.apache.phoenix.monitoring.MetricType.MUTATION_SQL_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.NUM_PARALLEL_SCANS;
+import static org.apache.phoenix.monitoring.MetricType.QUERY_SERVICES_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.QUERY_TIME;
import static org.apache.phoenix.monitoring.MetricType.QUERY_TIMEOUT_COUNTER;
import static org.apache.phoenix.monitoring.MetricType.TASK_REJECTED_COUNTER;
@@ -73,7 +75,9 @@ public enum GlobalClientMetrics {
GLOBAL_QUERY_TIMEOUT_COUNTER(QUERY_TIMEOUT_COUNTER),
GLOBAL_FAILED_QUERY_COUNTER(QUERY_FAILED_COUNTER),
GLOBAL_SPOOL_FILE_COUNTER(SPOOL_FILE_COUNTER),
- GLOBAL_OPEN_PHOENIX_CONNECTIONS(OPEN_PHOENIX_CONNECTIONS_COUNTER);
+ GLOBAL_OPEN_PHOENIX_CONNECTIONS(OPEN_PHOENIX_CONNECTIONS_COUNTER),
+ GLOBAL_QUERY_SERVICES_COUNTER(QUERY_SERVICES_COUNTER),
+ GLOBAL_HCONNECTIONS_COUNTER(HCONNECTIONS_COUNTER);
private static final boolean isGlobalMetricsEnabled = QueryServicesOptions.withDefaults().isGlobalMetricsEnabled();
private GlobalMetric metric;
http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
index 6cfe977..b420b75 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
@@ -41,7 +41,9 @@ public enum MetricType {
CACHE_REFRESH_SPLITS_COUNTER("Number of times cache was refreshed because of splits"),
WALL_CLOCK_TIME_MS("Wall clock time elapsed for the overall query execution"),
RESULT_SET_TIME_MS("Wall clock time elapsed for reading all records using resultSet.next()"),
- OPEN_PHOENIX_CONNECTIONS_COUNTER("Number of open phoenix connections");
+ OPEN_PHOENIX_CONNECTIONS_COUNTER("Number of open phoenix connections"),
+ QUERY_SERVICES_COUNTER("Number of ConnectionQueryServicesImpl instantiated"),
+ HCONNECTIONS_COUNTER("Number of HConnections created by phoenix driver");
private final String description;
http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index be34f66..f2eb8e4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -26,6 +26,8 @@ import static org.apache.phoenix.coprocessor.MetaDataProtocol.getVersion;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_ENABLED;
import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE;
@@ -151,6 +153,8 @@ import org.apache.phoenix.iterate.TableResultIterator.RenewLeaseStatus;
import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.monitoring.GlobalClientMetrics;
+import org.apache.phoenix.monitoring.GlobalMetric;
import org.apache.phoenix.parse.PFunction;
import org.apache.phoenix.parse.PSchema;
import org.apache.phoenix.protobuf.ProtobufUtil;
@@ -392,6 +396,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
QueryServices.TRANSACTIONS_ENABLED,
QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
+ GLOBAL_HCONNECTIONS_COUNTER.increment();
+ logger.info("HConnnection established. Details: " + connection + " " + Throwables.getStackTraceAsString(new Exception()));
// only initialize the tx service client if needed and if we succeeded in getting a connection
// to HBase
if (transactionsEnabled) {
@@ -457,6 +463,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
return;
}
closed = true;
+ GLOBAL_QUERY_SERVICES_COUNTER.decrement();
SQLException sqlE = null;
try {
// Attempt to return any unused sequences.
@@ -473,6 +480,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
try {
// close the HBase connection
if (connection != null) connection.close();
+ GLOBAL_HCONNECTIONS_COUNTER.decrement();
} finally {
if (renewLeaseExecutor != null) {
renewLeaseExecutor.shutdownNow();
@@ -2360,6 +2368,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
boolean hConnectionEstablished = false;
boolean success = false;
try {
+ GLOBAL_QUERY_SERVICES_COUNTER.increment();
+ logger.info("An instance of ConnectionQueryServices was created: " + Throwables.getStackTraceAsString(new Exception()));
openConnection();
hConnectionEstablished = true;
boolean isDoNotUpgradePropSet = UpgradeUtil.isNoUpgradeSet(props);
[07/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow
incremental index rebuilds(addendum)
Posted by ma...@apache.org.
PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds(addendum)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e906841f
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e906841f
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e906841f
Branch: refs/heads/calcite
Commit: e906841fb2c96615f0d50b06c48bdeda78379c94
Parents: 83827cd
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 12:28:14 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 12:28:14 2016 +0530
----------------------------------------------------------------------
.../end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/e906841f/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
index 5e16b05..4b2371c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
@@ -36,10 +36,10 @@ import com.google.common.collect.Maps;
* Tests for the {@link IndexToolForPartialBuildWithNamespaceEnabled}
*/
@RunWith(Parameterized.class)
-public class IndexToolForPartialBuildWithNamespaceEnabled extends IndexToolForPartialBuildIT {
+public class IndexToolForPartialBuildWithNamespaceEnabledIT extends IndexToolForPartialBuildIT {
- public IndexToolForPartialBuildWithNamespaceEnabled(boolean localIndex, boolean isNamespaceEnabled) {
+ public IndexToolForPartialBuildWithNamespaceEnabledIT(boolean localIndex, boolean isNamespaceEnabled) {
super(localIndex);
this.isNamespaceEnabled=isNamespaceEnabled;
}
[03/26] phoenix git commit: PHOENIX-3505 Avoid NPE on close() in
OrderedResultIterator
Posted by ma...@apache.org.
PHOENIX-3505 Avoid NPE on close() in OrderedResultIterator
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e23634a3
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e23634a3
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e23634a3
Branch: refs/heads/calcite
Commit: e23634a358929516ce210fe06d668ce475eccccb
Parents: c504604
Author: Josh Elser <el...@apache.org>
Authored: Wed Nov 23 11:16:35 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Sat Dec 24 23:11:10 2016 -0500
----------------------------------------------------------------------
.../phoenix/iterate/OrderedResultIterator.java | 5 ++-
.../iterate/OrderedResultIteratorTest.java | 41 ++++++++++++++++++++
2 files changed, 45 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/e23634a3/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
index 8dcb2e8..da75bb7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
@@ -279,7 +279,10 @@ public class OrderedResultIterator implements PeekingResultIterator {
@Override
public void close() throws SQLException {
- resultIterator.close();
+ // Guard against resultIterator being null
+ if (null != resultIterator) {
+ resultIterator.close();
+ }
resultIterator = PeekingResultIterator.EMPTY_ITERATOR;
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/e23634a3/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
new file mode 100644
index 0000000..50ed8e9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.phoenix.iterate;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.OrderByExpression;
+import org.junit.Test;
+
+/**
+ * Test class for {@link OrderedResultIterator}.
+ */
+public class OrderedResultIteratorTest {
+
+ @Test
+ public void testNullIteratorOnClose() throws SQLException {
+ ResultIterator delegate = ResultIterator.EMPTY_ITERATOR;
+ List<OrderByExpression> orderByExpressions = Collections.singletonList(null);
+ int thresholdBytes = Integer.MAX_VALUE;
+ OrderedResultIterator iterator = new OrderedResultIterator(delegate, orderByExpressions, thresholdBytes);
+ // Should not throw an exception
+ iterator.close();
+ }
+
+}
[05/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow
incremental index rebuilds
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index 82b353c..e594e0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -17,10 +17,17 @@
*/
package org.apache.phoenix.mapreduce.index;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.ResultSet;
import java.sql.SQLException;
+import java.util.ArrayList;
import java.util.List;
import org.apache.commons.cli.CommandLine;
@@ -36,11 +43,15 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
import org.apache.hadoop.hbase.util.Bytes;
@@ -48,9 +59,13 @@ import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.phoenix.compile.PostIndexDDLCompiler;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.index.IndexMaintainer;
import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.mapreduce.CsvBulkImportUtil;
import org.apache.phoenix.mapreduce.util.ColumnInfoToStringEncoderDecoder;
@@ -62,9 +77,9 @@ import org.apache.phoenix.schema.PIndexState;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.PTable.IndexType;
import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ByteUtil;
import org.apache.phoenix.util.ColumnInfo;
import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
import org.apache.phoenix.util.PhoenixRuntime;
import org.apache.phoenix.util.QueryUtil;
import org.apache.phoenix.util.SchemaUtil;
@@ -72,6 +87,8 @@ import org.apache.phoenix.util.TransactionUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.collect.Lists;
+
/**
* An MR job to populate the index table from the data table.
*
@@ -85,7 +102,11 @@ public class IndexTool extends Configured implements Tool {
private static final Option DATA_TABLE_OPTION = new Option("dt", "data-table", true,
"Data table name (mandatory)");
private static final Option INDEX_TABLE_OPTION = new Option("it", "index-table", true,
- "Index table name(mandatory)");
+ "Index table name(not required in case of partial rebuilding)");
+
+ private static final Option PARTIAL_REBUILD_OPTION = new Option("pr", "partial-rebuild", false,
+ "To build indexes for a data table from least disabledTimeStamp");
+
private static final Option DIRECT_API_OPTION = new Option("direct", "direct", false,
"If specified, we avoid the bulk load (optional)");
private static final Option RUN_FOREGROUND_OPTION =
@@ -105,6 +126,7 @@ public class IndexTool extends Configured implements Tool {
options.addOption(SCHEMA_NAME_OPTION);
options.addOption(DATA_TABLE_OPTION);
options.addOption(INDEX_TABLE_OPTION);
+ options.addOption(PARTIAL_REBUILD_OPTION);
options.addOption(DIRECT_API_OPTION);
options.addOption(RUN_FOREGROUND_OPTION);
options.addOption(OUTPUT_PATH_OPTION);
@@ -139,18 +161,18 @@ public class IndexTool extends Configured implements Tool {
+ "parameter");
}
- if (!cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())) {
- throw new IllegalStateException(INDEX_TABLE_OPTION.getLongOpt() + " is a mandatory "
- + "parameter");
- }
-
- if (!cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt())) {
- throw new IllegalStateException(OUTPUT_PATH_OPTION.getLongOpt() + " is a mandatory "
- + "parameter");
- }
-
- if (!cmdLine.hasOption(DIRECT_API_OPTION.getOpt())
- && cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt())) {
+ if (!(cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt()) || cmdLine.hasOption(DIRECT_API_OPTION.getOpt()))
+ && !cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt())) {
+ throw new IllegalStateException(OUTPUT_PATH_OPTION.getLongOpt() + " is a mandatory " + "parameter");
+ }
+
+ if (cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt()) && cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())) {
+ throw new IllegalStateException("Index name should not be passed with " + PARTIAL_REBUILD_OPTION.getLongOpt());
+ }
+
+ if (!(cmdLine.hasOption(DIRECT_API_OPTION.getOpt())) && cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())
+ && cmdLine.hasOption(RUN_FOREGROUND_OPTION
+ .getOpt())) {
throw new IllegalStateException(RUN_FOREGROUND_OPTION.getLongOpt()
+ " is applicable only for " + DIRECT_API_OPTION.getLongOpt());
}
@@ -167,50 +189,155 @@ public class IndexTool extends Configured implements Tool {
formatter.printHelp("help", options);
System.exit(exitCode);
}
+
+ class JobFactory {
+ Connection connection;
+ Configuration configuration;
+ private Path outputPath;
- @Override
- public int run(String[] args) throws Exception {
- Connection connection = null;
- try {
- CommandLine cmdLine = null;
- try {
- cmdLine = parseOptions(args);
- } catch (IllegalStateException e) {
- printHelpAndExit(e.getMessage(), getOptions());
+ public JobFactory(Connection connection, Configuration configuration, Path outputPath) {
+ this.connection = connection;
+ this.configuration = configuration;
+ this.outputPath = outputPath;
+
+ }
+
+ public Job getJob(String schemaName, String indexTable, String dataTable, boolean useDirectApi, boolean isPartialBuild) throws Exception {
+ if (isPartialBuild) {
+ return configureJobForPartialBuild(schemaName, dataTable);
+ } else {
+ return configureJobForAysncIndex(schemaName, indexTable, dataTable, useDirectApi);
}
- final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
- final String schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
- final String dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
- final String indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
+ }
+
+ private Job configureJobForPartialBuild(String schemaName, String dataTable) throws Exception {
final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
- final String qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
-
+ final PTable pdataTable = PhoenixRuntime.getTable(connection, qDataTable);
connection = ConnectionUtil.getInputConnection(configuration);
- if (!isValidIndexTable(connection, qDataTable, indexTable)) {
- throw new IllegalArgumentException(String.format(
- " %s is not an index table for %s ", qIndexTable, qDataTable));
+ long minDisableTimestamp = HConstants.LATEST_TIMESTAMP;
+ PTable indexWithMinDisableTimestamp = null;
+
+ //Get Indexes in building state, minDisabledTimestamp
+ List<String> disableIndexes = new ArrayList<String>();
+ List<PTable> disabledPIndexes = new ArrayList<PTable>();
+ for (PTable index : pdataTable.getIndexes()) {
+ if (index.getIndexState().equals(PIndexState.BUILDING)) {
+ disableIndexes.add(index.getTableName().getString());
+ disabledPIndexes.add(index);
+ if (minDisableTimestamp > index.getIndexDisableTimestamp()) {
+ minDisableTimestamp = index.getIndexDisableTimestamp();
+ indexWithMinDisableTimestamp = index;
+ }
+ }
+ }
+
+ if (indexWithMinDisableTimestamp == null) {
+ throw new Exception("There is no index for a datatable to be rebuild:" + qDataTable);
+ }
+ if (minDisableTimestamp == 0) {
+ throw new Exception("It seems Index " + indexWithMinDisableTimestamp
+ + " has disable timestamp as 0 , please run IndexTool with IndexName to build it first");
+ // TODO probably we can initiate the job by ourself or can skip them while making the list for partial build with a warning
+ }
+
+ long maxTimestamp = getMaxRebuildAsyncDate(schemaName, disableIndexes);
+
+ //serialize index maintaienr in job conf with Base64 TODO: Need to find better way to serialize them in conf.
+ List<IndexMaintainer> maintainers = Lists.newArrayListWithExpectedSize(disabledPIndexes.size());
+ for (PTable index : disabledPIndexes) {
+ maintainers.add(index.getIndexMaintainer(pdataTable, connection.unwrap(PhoenixConnection.class)));
+ }
+ ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+ IndexMaintainer.serializeAdditional(pdataTable, indexMetaDataPtr, disabledPIndexes, connection.unwrap(PhoenixConnection.class));
+ PhoenixConfigurationUtil.setIndexMaintainers(configuration, indexMetaDataPtr);
+
+ //Prepare raw scan
+ Scan scan = IndexManagementUtil.newLocalStateScan(maintainers);
+ scan.setTimeRange(minDisableTimestamp - 1, maxTimestamp);
+ scan.setRaw(true);
+ scan.setCacheBlocks(false);
+ if (pdataTable.isTransactional()) {
+ long maxTimeRange = pdataTable.getTimeStamp() + 1;
+ scan.setAttribute(BaseScannerRegionObserver.TX_SCN,
+ Bytes.toBytes(Long.valueOf(Long.toString(TransactionUtil.convertToNanoseconds(maxTimeRange)))));
+ }
+
+
+ String physicalTableName=pdataTable.getPhysicalName().getString();
+ final String jobName = String.format("Phoenix Indexes build for " + pdataTable.getName().toString());
+
+ PhoenixConfigurationUtil.setInputTableName(configuration, qDataTable);
+ PhoenixConfigurationUtil.setPhysicalTableName(configuration, physicalTableName);
+
+ //TODO: update disable indexes
+ PhoenixConfigurationUtil.setDisableIndexes(configuration, StringUtils.join(",",disableIndexes));
+
+ final Job job = Job.getInstance(configuration, jobName);
+ if (outputPath != null) {
+ FileOutputFormat.setOutputPath(job, outputPath);
+ }
+ job.setJarByClass(IndexTool.class);
+ TableMapReduceUtil.initTableMapperJob(physicalTableName, scan, PhoenixIndexPartialBuildMapper.class, null,
+ null, job);
+ TableMapReduceUtil.initCredentials(job);
+ TableInputFormat.configureSplitTable(job, TableName.valueOf(physicalTableName));
+ return configureSubmittableJobUsingDirectApi(job, true);
+ }
+
+ private long getMaxRebuildAsyncDate(String schemaName, List<String> disableIndexes) throws SQLException {
+ Long maxRebuilAsyncDate=HConstants.LATEST_TIMESTAMP;
+ Long maxDisabledTimeStamp=0L;
+ if (disableIndexes == null || disableIndexes.isEmpty()) { return 0; }
+ List<String> quotedIndexes = new ArrayList<String>(disableIndexes.size());
+ for (String index : disableIndexes) {
+ quotedIndexes.add("'" + index + "'");
+ }
+ ResultSet rs = connection.createStatement()
+ .executeQuery("SELECT MAX(" + ASYNC_REBUILD_TIMESTAMP + "),MAX("+INDEX_DISABLE_TIMESTAMP+") FROM " + SYSTEM_CATALOG_NAME + " ("
+ + ASYNC_REBUILD_TIMESTAMP + " BIGINT) WHERE " + TABLE_SCHEM
+ + (schemaName != null && schemaName.length() > 0 ? "='" + schemaName + "'" : " IS NULL")
+ + " and " + TABLE_NAME + " IN (" + StringUtils.join(",", quotedIndexes) + ")");
+ if (rs.next()) {
+ maxRebuilAsyncDate = rs.getLong(1);
+ maxDisabledTimeStamp = rs.getLong(2);
+ }
+ // Do check if table is disabled again after user invoked async rebuilding during the run of the job
+ if (maxRebuilAsyncDate > maxDisabledTimeStamp) {
+ return maxRebuilAsyncDate;
+ } else {
+ throw new RuntimeException(
+ "Inconsistent state we have one or more index tables which are disabled after the async is called!!");
}
+
+ }
+ private Job configureJobForAysncIndex(String schemaName, String indexTable, String dataTable, boolean useDirectApi)
+ throws Exception {
+ final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+ final String qIndexTable;
+ if (schemaName != null && !schemaName.isEmpty()) {
+ qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
+ } else {
+ qIndexTable = indexTable;
+ }
final PTable pdataTable = PhoenixRuntime.getTable(connection, qDataTable);
+
final PTable pindexTable = PhoenixRuntime.getTable(connection, qIndexTable);
-
+
+ long maxTimeRange = pindexTable.getTimeStamp() + 1;
// this is set to ensure index tables remains consistent post population.
- long maxTimeRange = pindexTable.getTimeStamp()+1;
+
if (pdataTable.isTransactional()) {
configuration.set(PhoenixConfigurationUtil.TX_SCN_VALUE,
Long.toString(TransactionUtil.convertToNanoseconds(maxTimeRange)));
}
configuration.set(PhoenixConfigurationUtil.CURRENT_SCN_VALUE,
Long.toString(maxTimeRange));
-
+
// check if the index type is LOCAL, if so, derive and set the physicalIndexName that is
// computed from the qDataTable name.
String physicalIndexTable = pindexTable.getPhysicalName().getString();
- boolean isLocalIndexBuild = false;
- if (IndexType.LOCAL.equals(pindexTable.getIndexType())) {
- physicalIndexTable = qDataTable;
- isLocalIndexBuild = true;
- }
+
final PhoenixConnection pConnection = connection.unwrap(PhoenixConnection.class);
final PostIndexDDLCompiler ddlCompiler =
@@ -224,18 +351,15 @@ public class IndexTool extends Configured implements Tool {
configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, upsertQuery);
PhoenixConfigurationUtil.setPhysicalTableName(configuration, physicalIndexTable);
- PhoenixConfigurationUtil.setOutputTableName(configuration, indexTable);
+ PhoenixConfigurationUtil.setDisableIndexes(configuration, indexTable);
PhoenixConfigurationUtil.setUpsertColumnNames(configuration,
indexColumns.toArray(new String[indexColumns.size()]));
final List<ColumnInfo> columnMetadataList =
PhoenixRuntime.generateColumnInfo(connection, qIndexTable, indexColumns);
ColumnInfoToStringEncoderDecoder.encode(configuration, columnMetadataList);
-
- final Path outputPath = CsvBulkImportUtil
- .getOutputPath(new Path(cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt())), physicalIndexTable);
FileSystem.get(configuration).delete(outputPath, true);
- final String jobName = String.format(INDEX_JOB_NAME_TEMPLATE, dataTable, indexTable);
+ final String jobName = String.format(INDEX_JOB_NAME_TEMPLATE, pdataTable.getName().toString(), indexTable);
final Job job = Job.getInstance(configuration, jobName);
job.setJarByClass(IndexTool.class);
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
@@ -245,17 +369,146 @@ public class IndexTool extends Configured implements Tool {
selectQuery);
TableMapReduceUtil.initCredentials(job);
- boolean useDirectApi = cmdLine.hasOption(DIRECT_API_OPTION.getOpt());
+
if (useDirectApi) {
- configureSubmittableJobUsingDirectApi(job, outputPath,
- cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt()));
+ return configureSubmittableJobUsingDirectApi(job, false);
} else {
- configureRunnableJobUsingBulkLoad(job, outputPath, isLocalIndexBuild);
- // Without direct API, we need to update the index state to ACTIVE from client.
- IndexToolUtil.updateIndexState(connection, qDataTable, indexTable,
- PIndexState.ACTIVE);
+ return configureRunnableJobUsingBulkLoad(job, outputPath);
+
+ }
+
+ }
+
+ /**
+ * Submits the job and waits for completion.
+ * @param job
+ * @param outputPath
+ * @return
+ * @throws Exception
+ */
+ private Job configureRunnableJobUsingBulkLoad(Job job, Path outputPath) throws Exception {
+ job.setMapperClass(PhoenixIndexImportMapper.class);
+ job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+ job.setMapOutputValueClass(KeyValue.class);
+ final Configuration configuration = job.getConfiguration();
+ final String physicalIndexTable =
+ PhoenixConfigurationUtil.getPhysicalTableName(configuration);
+ final HTable htable = new HTable(configuration, physicalIndexTable);
+ HFileOutputFormat.configureIncrementalLoad(job, htable);
+ return job;
+
+ }
+
+ /**
+ * Uses the HBase Front Door Api to write to index table. Submits the job and either returns or
+ * waits for the job completion based on runForeground parameter.
+ *
+ * @param job
+ * @param outputPath
+ * @param runForeground - if true, waits for job completion, else submits and returns
+ * immediately.
+ * @return
+ * @throws Exception
+ */
+ private Job configureSubmittableJobUsingDirectApi(Job job, boolean isPartialRebuild)
+ throws Exception {
+ if (!isPartialRebuild) {
+ //Don't configure mapper for partial build as it is configured already
+ job.setMapperClass(PhoenixIndexImportDirectMapper.class);
+ }
+ job.setReducerClass(PhoenixIndexImportDirectReducer.class);
+ Configuration conf = job.getConfiguration();
+ HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
+ // Set the Physical Table name for use in DirectHTableWriter#write(Mutation)
+ conf.set(TableOutputFormat.OUTPUT_TABLE,
+ PhoenixConfigurationUtil.getPhysicalTableName(job.getConfiguration()));
+ //Set the Output classes
+ job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+ job.setMapOutputValueClass(IntWritable.class);
+ job.setOutputKeyClass(NullWritable.class);
+ job.setOutputValueClass(NullWritable.class);
+ TableMapReduceUtil.addDependencyJars(job);
+ job.setNumReduceTasks(1);
+ return job;
+ }
+
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+ Connection connection = null;
+ HTable htable = null;
+ try {
+ CommandLine cmdLine = null;
+ try {
+ cmdLine = parseOptions(args);
+ } catch (IllegalStateException e) {
+ printHelpAndExit(e.getMessage(), getOptions());
+ }
+ final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
+ final String schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
+ final String dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
+ final String indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
+ final boolean isPartialBuild = cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt());
+ final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+ boolean useDirectApi = cmdLine.hasOption(DIRECT_API_OPTION.getOpt());
+ String basePath=cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt());
+ boolean isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
+ connection = ConnectionUtil.getInputConnection(configuration);
+ byte[][] splitKeysBeforeJob = null;
+ boolean isLocalIndexBuild = false;
+ PTable pindexTable = null;
+ if (indexTable != null) {
+ if (!isValidIndexTable(connection, qDataTable,indexTable)) {
+ throw new IllegalArgumentException(String.format(
+ " %s is not an index table for %s ", indexTable, qDataTable));
+ }
+ pindexTable = PhoenixRuntime.getTable(connection, schemaName != null && !schemaName.isEmpty()
+ ? SchemaUtil.getQualifiedTableName(schemaName, indexTable) : indexTable);
+ htable = (HTable)connection.unwrap(PhoenixConnection.class).getQueryServices()
+ .getTable(pindexTable.getPhysicalName().getBytes());
+ if (IndexType.LOCAL.equals(pindexTable.getIndexType())) {
+ isLocalIndexBuild = true;
+ splitKeysBeforeJob = htable.getRegionLocator().getStartKeys();
+ }
+ }
+
+ PTable pdataTable = PhoenixRuntime.getTableNoCache(connection, qDataTable);
+ Path outputPath = null;
+ if (basePath != null) {
+ outputPath = CsvBulkImportUtil.getOutputPath(new Path(basePath), pindexTable == null
+ ? pdataTable.getPhysicalName().getString() : pindexTable.getPhysicalName().getString());
+ FileSystem.get(configuration).delete(outputPath, true);
+ }
+
+ Job job = new JobFactory(connection, configuration, outputPath).getJob(schemaName, indexTable, dataTable,
+ useDirectApi, isPartialBuild);
+ if (!isForeground && useDirectApi) {
+ LOG.info("Running Index Build in Background - Submit async and exit");
+ job.submit();
+ return 0;
+ }
+ LOG.info("Running Index Build in Foreground. Waits for the build to complete. This may take a long time!.");
+ boolean result = job.waitForCompletion(true);
+
+ if (result) {
+ if (!useDirectApi && indexTable != null) {
+ if (isLocalIndexBuild) {
+ validateSplitForLocalIndex(splitKeysBeforeJob, htable);
+ }
+ LOG.info("Loading HFiles from {}", outputPath);
+ LoadIncrementalHFiles loader = new LoadIncrementalHFiles(configuration);
+ loader.doBulkLoad(outputPath, htable);
+ htable.close();
+ // Without direct API, we need to update the index state to ACTIVE from client.
+ IndexToolUtil.updateIndexState(connection, qDataTable, indexTable, PIndexState.ACTIVE);
+ FileSystem.get(configuration).delete(outputPath, true);
+ }
+ return 0;
+ } else {
+ LOG.error("IndexTool job failed! Check logs for errors..");
+ return -1;
}
- return 0;
} catch (Exception ex) {
LOG.error("An exception occurred while performing the indexing job: "
+ ExceptionUtils.getMessage(ex) + " at:\n" + ExceptionUtils.getStackTrace(ex));
@@ -265,6 +518,9 @@ public class IndexTool extends Configured implements Tool {
if (connection != null) {
connection.close();
}
+ if (htable != null) {
+ htable.close();
+ }
} catch (SQLException sqle) {
LOG.error("Failed to close connection ", sqle.getMessage());
throw new RuntimeException("Failed to close connection");
@@ -272,91 +528,18 @@ public class IndexTool extends Configured implements Tool {
}
}
- /**
- * Submits the job and waits for completion.
- * @param job
- * @param outputPath
- * @return
- * @throws Exception
- */
- private void configureRunnableJobUsingBulkLoad(Job job, Path outputPath, boolean isLocalIndexBuild) throws Exception {
- job.setMapperClass(PhoenixIndexImportMapper.class);
- job.setMapOutputKeyClass(ImmutableBytesWritable.class);
- job.setMapOutputValueClass(KeyValue.class);
- final Configuration configuration = job.getConfiguration();
- final String physicalIndexTable =
- PhoenixConfigurationUtil.getPhysicalTableName(configuration);
- final HTable htable = new HTable(configuration, physicalIndexTable);
- HFileOutputFormat.configureIncrementalLoad(job, htable);
- byte[][] splitKeysBeforeJob = null;
- if(isLocalIndexBuild) {
- splitKeysBeforeJob = htable.getRegionLocator().getStartKeys();
- }
- boolean status = job.waitForCompletion(true);
- if (!status) {
- LOG.error("IndexTool job failed!");
- htable.close();
- throw new Exception("IndexTool job failed: " + job.toString());
- } else {
- if (isLocalIndexBuild
- && !IndexUtil.matchingSplitKeys(splitKeysBeforeJob, htable.getRegionLocator()
- .getStartKeys())) {
- String errMsg = "The index to build is local index and the split keys are not matching"
- + " before and after running the job. Please rerun the job otherwise"
- + " there may be inconsistencies between actual data and index data";
- LOG.error(errMsg);
- throw new Exception(errMsg);
- }
- }
-
- LOG.info("Loading HFiles from {}", outputPath);
- LoadIncrementalHFiles loader = new LoadIncrementalHFiles(configuration);
- loader.doBulkLoad(outputPath, htable);
- htable.close();
-
- FileSystem.get(configuration).delete(outputPath, true);
- }
- /**
- * Uses the HBase Front Door Api to write to index table. Submits the job and either returns or
- * waits for the job completion based on runForeground parameter.
- *
- * @param job
- * @param outputPath
- * @param runForeground - if true, waits for job completion, else submits and returns
- * immediately.
- * @return
- * @throws Exception
- */
- private void configureSubmittableJobUsingDirectApi(Job job, Path outputPath, boolean runForeground)
- throws Exception {
- job.setMapperClass(PhoenixIndexImportDirectMapper.class);
- job.setReducerClass(PhoenixIndexImportDirectReducer.class);
- Configuration conf = job.getConfiguration();
- HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
- // Set the Physical Table name for use in DirectHTableWriter#write(Mutation)
- conf.set(TableOutputFormat.OUTPUT_TABLE,
- PhoenixConfigurationUtil.getPhysicalTableName(job.getConfiguration()));
-
- //Set the Output classes
- job.setMapOutputValueClass(IntWritable.class);
- job.setOutputKeyClass(NullWritable.class);
- job.setOutputValueClass(NullWritable.class);
- TableMapReduceUtil.addDependencyJars(job);
- job.setNumReduceTasks(1);
-
- if (!runForeground) {
- LOG.info("Running Index Build in Background - Submit async and exit");
- job.submit();
- return;
- }
- LOG.info("Running Index Build in Foreground. Waits for the build to complete. This may take a long time!.");
- boolean result = job.waitForCompletion(true);
- if (!result) {
- LOG.error("IndexTool job failed!");
- throw new Exception("IndexTool job failed: " + job.toString());
+
+ private boolean validateSplitForLocalIndex(byte[][] splitKeysBeforeJob, HTable htable) throws Exception {
+ if (splitKeysBeforeJob != null
+ && !IndexUtil.matchingSplitKeys(splitKeysBeforeJob, htable.getRegionLocator().getStartKeys())) {
+ String errMsg = "The index to build is local index and the split keys are not matching"
+ + " before and after running the job. Please rerun the job otherwise"
+ + " there may be inconsistencies between actual data and index data";
+ LOG.error(errMsg);
+ throw new Exception(errMsg);
}
- FileSystem.get(conf).delete(outputPath, true);
+ return true;
}
/**
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
index 1058670..2dc7551 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
@@ -48,11 +48,13 @@ public class IndexToolUtil {
*/
public static void updateIndexState(Configuration configuration,PIndexState state) throws SQLException {
final String masterTable = PhoenixConfigurationUtil.getInputTableName(configuration);
- final String indexTable = PhoenixConfigurationUtil.getOutputTableName(configuration);
+ final String[] indexTables = PhoenixConfigurationUtil.getDisableIndexes(configuration).split(",");
final Properties overrideProps = new Properties();
final Connection connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
try {
- updateIndexState(connection, masterTable, indexTable , state);
+ for (String indexTable : indexTables) {
+ updateIndexState(connection, masterTable, indexTable, state);
+ }
} finally {
if(connection != null) {
connection.close();
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index 9c64efc..15e55dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -99,7 +99,7 @@ public class PhoenixIndexImportDirectMapper extends
this.pStatement = connection.prepareStatement(upsertQuery);
} catch (SQLException e) {
- throw new RuntimeException(e.getMessage());
+ throw new RuntimeException(e);
}
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
new file mode 100644
index 0000000..47a38a7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
@@ -0,0 +1,182 @@
+/*
+ * 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.phoenix.mapreduce.index;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.PhoenixJobCounters;
+import org.apache.phoenix.mapreduce.util.ConnectionUtil;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Mapper that hands over rows from data table to the index table.
+ */
+public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWritable, IntWritable> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PhoenixIndexPartialBuildMapper.class);
+
+ private PhoenixConnection connection;
+
+ private DirectHTableWriter writer;
+
+ private int batchSize;
+
+ private List<Mutation> mutations ;
+
+ private ImmutableBytesPtr maintainers;
+
+ @Override
+ protected void setup(final Context context) throws IOException, InterruptedException {
+ super.setup(context);
+ final Configuration configuration = context.getConfiguration();
+ writer = new DirectHTableWriter(configuration);
+
+ try {
+ final Properties overrideProps = new Properties();
+ String scn = configuration.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE);
+ String txScnValue = configuration.get(PhoenixConfigurationUtil.TX_SCN_VALUE);
+ if(txScnValue==null && scn!=null) {
+ overrideProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn);
+ }
+ connection = ConnectionUtil.getOutputConnection(configuration, overrideProps).unwrap(PhoenixConnection.class);
+ connection.setAutoCommit(false);
+ // Get BatchSize
+ ConnectionQueryServices services = connection.getQueryServices();
+ int maxSize =
+ services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,
+ QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+ batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
+ LOG.info("Mutation Batch Size = " + batchSize);
+ this.mutations = Lists.newArrayListWithExpectedSize(batchSize);
+ maintainers=new ImmutableBytesPtr(PhoenixConfigurationUtil.getIndexMaintainers(configuration));
+ } catch (SQLException e) {
+ throw new RuntimeException(e.getMessage());
+ }
+ }
+
+ @Override
+ protected void map(ImmutableBytesWritable row, Result value, Context context)
+ throws IOException, InterruptedException {
+ context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
+ try {
+ byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(maintainers);
+ byte[] uuidValue = ServerCacheClient.generateId();
+ Put put = null;
+ Delete del = null;
+ for (Cell cell : value.rawCells()) {
+ if (KeyValue.Type.codeToType(cell.getTypeByte()) == KeyValue.Type.Put) {
+ if (put == null) {
+ put = new Put(CellUtil.cloneRow(cell));
+ put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+ put.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+ put.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
+ mutations.add(put);
+ }
+ put.add(cell);
+ } else {
+ if (del == null) {
+ del = new Delete(CellUtil.cloneRow(cell));
+ del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+ del.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+ del.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
+ mutations.add(del);
+ }
+ del.addDeleteMarker(cell);
+ }
+ }
+ // Write Mutation Batch
+ if (context.getCounter(PhoenixJobCounters.INPUT_RECORDS).getValue() % batchSize == 0) {
+ writeBatch(mutations, context);
+ mutations.clear();
+ }
+ // Make sure progress is reported to Application Master.
+ context.progress();
+ } catch (SQLException e) {
+ LOG.error(" Error {} while read/write of a record ", e.getMessage());
+ context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void writeBatch(List<Mutation> mutations, Context context)
+ throws IOException, SQLException, InterruptedException {
+ writer.write(mutations);
+ context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(mutations.size());
+ }
+
+ @Override
+ protected void cleanup(Context context) throws IOException, InterruptedException {
+ try {
+ // Write the last & final Mutation Batch
+ if (!mutations.isEmpty()) {
+ writeBatch(mutations, context);
+ }
+ // We are writing some dummy key-value as map output here so that we commit only one
+ // output to reducer.
+ context.write(new ImmutableBytesWritable(UUID.randomUUID().toString().getBytes()),
+ new IntWritable(0));
+ super.cleanup(context);
+ } catch (SQLException e) {
+ LOG.error(" Error {} while read/write of a record ", e.getMessage());
+ context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+ throw new RuntimeException(e);
+ } finally {
+ if (connection != null) {
+ try {
+ connection.close();
+ } catch (SQLException e) {
+ LOG.error("Error {} while closing connection in the PhoenixIndexMapper class ",
+ e.getMessage());
+ }
+ }
+ if (writer != null) {
+ writer.close();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
index 2264acd..f3e4450 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
@@ -30,7 +30,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.NullDBWritable;
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
@@ -98,6 +100,12 @@ public final class PhoenixConfigurationUtil {
public static final String MAPREDUCE_OUTPUT_CLUSTER_QUORUM = "phoneix.mapreduce.output.cluster.quorum";
+ public static final String INDEX_DISABLED_TIMESTAMP_VALUE = "phoenix.mr.index.disableTimestamp";
+
+ public static final String INDEX_MAINTAINERS = "phoenix.mr.index.maintainers";
+
+ public static final String DISABLED_INDEXES = "phoenix.mr.index.disabledIndexes";
+
public enum SchemaType {
TABLE,
QUERY;
@@ -428,4 +436,27 @@ public final class PhoenixConfigurationUtil {
return ReflectionUtils.newInstance(processorClass, conf);
}
+
+ public static byte[] getIndexMaintainers(final Configuration configuration){
+ Preconditions.checkNotNull(configuration);
+ return Base64.decode(configuration.get(INDEX_MAINTAINERS));
+ }
+
+ public static void setIndexMaintainers(final Configuration configuration,
+ final ImmutableBytesWritable indexMetaDataPtr) {
+ Preconditions.checkNotNull(configuration);
+ Preconditions.checkNotNull(indexMetaDataPtr);
+ configuration.set(INDEX_MAINTAINERS, Base64.encodeBytes(indexMetaDataPtr.get()));
+ }
+
+ public static void setDisableIndexes(Configuration configuration, String indexName) {
+ Preconditions.checkNotNull(configuration);
+ Preconditions.checkNotNull(indexName);
+ configuration.set(DISABLED_INDEXES, indexName);
+ }
+
+ public static String getDisableIndexes(Configuration configuration) {
+ Preconditions.checkNotNull(configuration);
+ return configuration.get(DISABLED_INDEXES);
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
index fcf817a..11328c2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -23,12 +23,14 @@ public class AlterIndexStatement extends SingleTableStatement {
private final String dataTableName;
private final boolean ifExists;
private final PIndexState indexState;
+ private boolean async;
- public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState) {
+ public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async) {
super(indexTableNode,0);
this.dataTableName = dataTableName;
this.ifExists = ifExists;
this.indexState = indexState;
+ this.async = async;
}
public String getTableName() {
@@ -48,4 +50,8 @@ public class AlterIndexStatement extends SingleTableStatement {
return indexState;
}
+ public boolean isAsync() {
+ return async;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 0c60e39..4b65c6a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -374,8 +374,12 @@ public class ParseNodeFactory {
return new DropIndexStatement(indexName, tableName, ifExists);
}
+ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+ return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+ }
+
public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
- return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+ return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, false);
}
public TraceStatement trace(boolean isTraceOn, double samplingRate) {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 4a565be..1e002d2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -127,10 +127,13 @@ public interface QueryServices extends SQLCloseable {
// A master switch if to enable auto rebuild an index which failed to be updated previously
public static final String INDEX_FAILURE_HANDLING_REBUILD_ATTRIB = "phoenix.index.failure.handling.rebuild";
+ public static final String INDEX_FAILURE_HANDLING_REBUILD_PERIOD = "phoenix.index.failure.handling.rebuild.period";
// Time interval to check if there is an index needs to be rebuild
public static final String INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB =
"phoenix.index.failure.handling.rebuild.interval";
+
+ public static final String INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE = "phoenix.index.rebuild.batch.perTable";
// A master switch if to block writes when index build failed
public static final String INDEX_FAILURE_BLOCK_WRITE = "phoenix.index.failure.block.write";
@@ -227,6 +230,7 @@ public interface QueryServices extends SQLCloseable {
public static final String CLIENT_CACHE_ENCODING = "phoenix.table.client.cache.encoding";
public static final String AUTO_UPGRADE_ENABLED = "phoenix.autoupgrade.enabled";
+
/**
* Get executor service used for parallel scans
*/
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 7f183e9..a3b9b32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -27,6 +27,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_CREATED_DATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
@@ -304,16 +305,27 @@ public class MetaDataClient {
TENANT_ID + "," +
TABLE_SCHEM + "," +
TABLE_NAME + "," +
- INDEX_STATE +
+ INDEX_STATE + "," +
+ ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
+ ") VALUES (?, ?, ?, ?, ?)";
+
+ private static final String UPDATE_INDEX_REBUILD_ASYNC_STATE =
+ "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+ TENANT_ID + "," +
+ TABLE_SCHEM + "," +
+ TABLE_NAME + "," +
+ ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
") VALUES (?, ?, ?, ?)";
+
private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
"UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
TENANT_ID + "," +
TABLE_SCHEM + "," +
TABLE_NAME + "," +
INDEX_STATE + "," +
- INDEX_DISABLE_TIMESTAMP +
- ") VALUES (?, ?, ?, ?, ?)";
+ INDEX_DISABLE_TIMESTAMP +","+
+ ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
+ ") VALUES (?, ?, ?, ?, ?, ?)";
//TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
// the new major release is out.
private static final String INSERT_COLUMN_CREATE_TABLE =
@@ -3469,7 +3481,13 @@ public class MetaDataClient {
String dataTableName = statement.getTableName();
String schemaName = statement.getTable().getName().getSchemaName();
String indexName = statement.getTable().getName().getTableName();
+ boolean isAsync = statement.isAsync();
PIndexState newIndexState = statement.getIndexState();
+ if (isAsync && newIndexState != PIndexState.REBUILD) { throw new SQLExceptionInfo.Builder(
+ SQLExceptionCode.ASYNC_NOT_ALLOWED)
+ .setMessage(" ASYNC building of index is allowed only with REBUILD index state")
+ .setSchemaName(schemaName).setTableName(indexName).build().buildException(); }
+
if (newIndexState == PIndexState.REBUILD) {
newIndexState = PIndexState.BUILDING;
}
@@ -3480,15 +3498,16 @@ public class MetaDataClient {
try {
if(newIndexState == PIndexState.ACTIVE){
tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE_TO_ACTIVE);
- } else {
+ }else{
tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE);
}
tableUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
tableUpsert.setString(2, schemaName);
tableUpsert.setString(3, indexName);
tableUpsert.setString(4, newIndexState.getSerializedValue());
+ tableUpsert.setLong(5, 0);
if(newIndexState == PIndexState.ACTIVE){
- tableUpsert.setLong(5, 0);
+ tableUpsert.setLong(6, 0);
}
tableUpsert.execute();
} finally {
@@ -3515,9 +3534,25 @@ public class MetaDataClient {
addTableToCache(result);
// Set so that we get the table below with the potentially modified rowKeyOrderOptimizable flag set
indexRef.setTable(result.getTable());
+ if (newIndexState == PIndexState.BUILDING && isAsync) {
+ try {
+ tableUpsert = connection.prepareStatement(UPDATE_INDEX_REBUILD_ASYNC_STATE);
+ tableUpsert.setString(1,
+ connection.getTenantId() == null ? null : connection.getTenantId().getString());
+ tableUpsert.setString(2, schemaName);
+ tableUpsert.setString(3, indexName);
+ tableUpsert.setLong(4, result.getTable().getTimeStamp());
+ tableUpsert.execute();
+ connection.commit();
+ } finally {
+ if (tableUpsert != null) {
+ tableUpsert.close();
+ }
+ }
+ }
}
}
- if (newIndexState == PIndexState.BUILDING) {
+ if (newIndexState == PIndexState.BUILDING && !isAsync) {
PTable index = indexRef.getTable();
// First delete any existing rows of the index
Long scn = connection.getSCN();
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 9622880..2a0f3b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -42,9 +42,13 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.WritableUtils;
@@ -54,6 +58,10 @@ import org.apache.phoenix.compile.IndexStatementRewriter;
import org.apache.phoenix.compile.StatementContext;
import org.apache.phoenix.compile.WhereCompiler;
import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
import org.apache.phoenix.exception.SQLExceptionCode;
import org.apache.phoenix.exception.SQLExceptionInfo;
import org.apache.phoenix.execute.TupleProjector;
@@ -67,10 +75,12 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
import org.apache.phoenix.index.IndexMaintainer;
import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
import org.apache.phoenix.jdbc.PhoenixStatement;
import org.apache.phoenix.parse.ParseNode;
import org.apache.phoenix.parse.SQLParser;
import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.protobuf.ProtobufUtil;
import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
import org.apache.phoenix.schema.ColumnNotFoundException;
@@ -78,6 +88,7 @@ import org.apache.phoenix.schema.ColumnRef;
import org.apache.phoenix.schema.KeyValueSchema;
import org.apache.phoenix.schema.PColumn;
import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PIndexState;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.PTableType;
import org.apache.phoenix.schema.TableNotFoundException;
@@ -87,12 +98,13 @@ import org.apache.phoenix.schema.tuple.Tuple;
import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PLong;
import org.apache.phoenix.schema.types.PVarbinary;
import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.tephra.TxConstants;
import com.google.common.collect.Lists;
-
-import org.apache.tephra.TxConstants;
+import com.google.protobuf.ServiceException;
public class IndexUtil {
public static final String INDEX_COLUMN_NAME_SEP = ":";
@@ -673,13 +685,48 @@ public class IndexUtil {
HConstants.NO_NONCE, HConstants.NO_NONCE);
}
+ public static MetaDataMutationResult disableIndexWithTimestamp(String indexTableName, long minTimeStamp,
+ HTableInterface metaTable, boolean blockWriteRebuildIndex) throws ServiceException, Throwable {
+ byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+ // Mimic the Put that gets generated by the client on an update of the index state
+ Put put = new Put(indexTableKey);
+ if (blockWriteRebuildIndex)
+ put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+ PIndexState.ACTIVE.getSerializedBytes());
+ else
+ put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+ PIndexState.DISABLE.getSerializedBytes());
+ put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
+ PLong.INSTANCE.toBytes(minTimeStamp));
+ put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP_BYTES,
+ PLong.INSTANCE.toBytes(0));
+ final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
+
+ final Map<byte[], MetaDataResponse> results = metaTable.coprocessorService(MetaDataService.class, indexTableKey,
+ indexTableKey, new Batch.Call<MetaDataService, MetaDataResponse>() {
+ @Override
+ public MetaDataResponse call(MetaDataService instance) throws IOException {
+ ServerRpcController controller = new ServerRpcController();
+ BlockingRpcCallback<MetaDataResponse> rpcCallback = new BlockingRpcCallback<MetaDataResponse>();
+ UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
+ for (Mutation m : tableMetadata) {
+ MutationProto mp = ProtobufUtil.toProto(m);
+ builder.addTableMetadataMutations(mp.toByteString());
+ }
+ instance.updateIndexState(controller, builder.build(), rpcCallback);
+ if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
+ return rpcCallback.get();
+ }
+ });
+ if (results.isEmpty()) { throw new IOException("Didn't get expected result size"); }
+ MetaDataResponse tmpResponse = results.values().iterator().next();
+ return MetaDataMutationResult.constructFromProto(tmpResponse);
+ }
+
public static boolean matchingSplitKeys(byte[][] splitKeys1, byte[][] splitKeys2) throws IOException {
- if (splitKeys1 != null && splitKeys2 != null
- && splitKeys1.length == splitKeys2.length) {
+ if (splitKeys1 != null && splitKeys2 != null && splitKeys1.length == splitKeys2.length) {
for (int i = 0; i < splitKeys1.length; i++) {
- if (Bytes.compareTo(splitKeys1[i], splitKeys2[i]) != 0) {
- return false;
- }
+ if (Bytes.compareTo(splitKeys1[i], splitKeys2[i]) != 0) { return false; }
}
} else {
return false;
[08/26] phoenix git commit: PHOENIX-3544 Fix UNSIGNED_DATE and
UNSIGNED_TIME type in phoenix-spark
Posted by ma...@apache.org.
PHOENIX-3544 Fix UNSIGNED_DATE and UNSIGNED_TIME type in phoenix-spark
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/b3269950
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/b3269950
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/b3269950
Branch: refs/heads/calcite
Commit: b32699505d18b5b93b4072e0b146df6b8ddaa7fb
Parents: e906841
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 16:54:13 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 16:54:13 2016 +0530
----------------------------------------------------------------------
phoenix-spark/src/it/resources/globalSetup.sql | 5 ++++-
.../it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala | 8 ++++++++
.../src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala | 5 ++---
3 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/it/resources/globalSetup.sql
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/resources/globalSetup.sql b/phoenix-spark/src/it/resources/globalSetup.sql
index 72f8620..28eb0f7 100644
--- a/phoenix-spark/src/it/resources/globalSetup.sql
+++ b/phoenix-spark/src/it/resources/globalSetup.sql
@@ -57,4 +57,7 @@ UPSERT INTO "small" VALUES ('key1', 'foo', 10000)
UPSERT INTO "small" VALUES ('key2', 'bar', 20000)
UPSERT INTO "small" VALUES ('key3', 'xyz', 30000)
-CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true
\ No newline at end of file
+CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true
+CREATE TABLE IF NOT EXISTS GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100))
+ CREATE TABLE IF NOT EXISTS OUTPUT_GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100))
+ upsert into GIGANTIC_TABLE values(0,2,3,4,-5,6,7,8,9.3,10.4,11.5,12.6,13.7,true,CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),'This is random textA','a','a','a')
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index dbcc4f1..9def354 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -632,5 +632,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
val epoch = new Date().getTime
assert(Math.abs(epoch - time) < 86400000)
}
+ test("can read all Phoenix data types") {
+ val sqlContext = new SQLContext(sc)
+ val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "GIGANTIC_TABLE",
+ "zkUrl" -> quorumAddress))
+ df.save("org.apache.phoenix.spark",SaveMode.Overwrite, Map("table" -> "OUTPUT_GIGANTIC_TABLE",
+ "zkUrl" -> quorumAddress))
+ df.count() shouldEqual 1
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 204a7ef..63289a0 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -129,11 +129,10 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
// Create a sequence of column data
val rowSeq = columns.map { case (name, sqlType) =>
val res = pr.resultMap(name)
-
// Special handling for data types
- if (dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
+ if (dateAsTimestamp && (sqlType == 91 || sqlType == 19)) { // 91 is the defined type for Date and 19 for UNSIGNED_DATE
new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
- } else if (sqlType == 92) { // 92 is the defined type for Time
+ } else if (sqlType == 92 || sqlType == 18) { // 92 is the defined type for Time and 18 for UNSIGNED_TIME
new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime)
} else {
res
[11/26] phoenix git commit: PHOENIX-3557 Provide a good error message
when an invalid test pattern was provided
Posted by ma...@apache.org.
PHOENIX-3557 Provide a good error message when an invalid test pattern was provided
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e4c6f5cd
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e4c6f5cd
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e4c6f5cd
Branch: refs/heads/calcite
Commit: e4c6f5cde940931bb4be82c7aa7959de58857ad8
Parents: a0e5efc
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 13:37:59 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 5 22:19:13 2017 -0500
----------------------------------------------------------------------
.../org/apache/phoenix/end2end/End2EndTestDriver.java | 12 +++++++++++-
1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4c6f5cd/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
index a505c1e..feb506f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
import org.apache.commons.cli.CommandLine;
import org.apache.hadoop.hbase.ClassFinder;
@@ -38,6 +39,8 @@ import org.junit.runner.Description;
import org.junit.runner.JUnitCore;
import org.junit.runner.Result;
import org.junit.runner.notification.Failure;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* This class drives the End2End tests suite execution against an
@@ -45,6 +48,7 @@ import org.junit.runner.notification.Failure;
*/
public class End2EndTestDriver extends AbstractHBaseTool {
+ private static final Logger LOG = LoggerFactory.getLogger(End2EndTestDriver.class);
private static final String SHORT_REGEX_ARG = "r";
private static final String SKIP_TESTS = "n";
@@ -73,7 +77,13 @@ public class End2EndTestDriver extends AbstractHBaseTool {
}
public void setPattern(String pattern) {
- testFilterRe = Pattern.compile(pattern);
+ try {
+ testFilterRe = Pattern.compile(pattern);
+ } catch (PatternSyntaxException e) {
+ LOG.error("Failed to find tests using pattern '" + pattern
+ + "'. Is it a valid Java regular expression?", e);
+ throw e;
+ }
}
@Override
[18/26] phoenix git commit: PHOENIX-3468 Double quote SYSTEM, USER,
DATE keywords in IT tests(Rajeshbabu)
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index eb81ae3..64935d2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -57,7 +57,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
Connection conn = DriverManager.getConnection(getUrl(), props);
conn.setAutoCommit(true);
- PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.PTSDB_NAME + " (inst,host,date) VALUES(?,'b',CURRENT_DATE())");
+ PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.PTSDB_NAME + " (inst,host,\"DATE\") VALUES(?,'b',CURRENT_DATE())");
stmt.setString(1, "a");
stmt.execute();
stmt.execute();
@@ -316,8 +316,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
PreparedStatement stmt = null;
try {
conn = DriverManager.getConnection(getUrl(), props);
- stmt = conn.prepareStatement("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
- stmt.execute();
+ conn.createStatement().execute("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
} finally {
closeStmtAndConn(stmt, conn);
}
@@ -357,8 +356,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
PreparedStatement stmt = null;
try {
conn = DriverManager.getConnection(getUrl(), props);
- stmt = conn.prepareStatement("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
- stmt.execute();
+ conn.createStatement().execute("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
} finally {
closeStmtAndConn(stmt, conn);
}
@@ -455,8 +453,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
PreparedStatement stmt = null;
try {
conn = DriverManager.getConnection(getUrl(), props);
- stmt = conn.prepareStatement("create table UpsertFloat (k varchar primary key, v float)");
- stmt.execute();
+ conn.createStatement().execute("create table UpsertFloat (k varchar primary key, v float)");
} finally {
closeStmtAndConn(stmt, conn);
}
@@ -496,8 +493,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
String tableName = BaseTest.generateUniqueName();
try {
conn = DriverManager.getConnection(getUrl(), props);
- pstmt = conn.prepareStatement("create table " + tableName + " (k varchar primary key, v integer)");
- pstmt.execute();
+ conn.createStatement().execute("create table " + tableName + " (k varchar primary key, v integer)");
} finally {
closeStmtAndConn(pstmt, conn);
}
@@ -587,8 +583,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
PreparedStatement stmt = null;
try {
conn = DriverManager.getConnection(getUrl(), props);
- stmt = conn.prepareStatement("create table UpsertTimestamp (k varchar, v unsigned_date not null, constraint pk primary key (k,v desc))");
- stmt.execute();
+ conn.createStatement().execute("create table UpsertTimestamp (k varchar, v unsigned_date not null, constraint pk primary key (k,v desc))");
} finally {
closeStmtAndConn(stmt, conn);
}
@@ -629,9 +624,8 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
PreparedStatement stmt = null;
try {
conn = DriverManager.getConnection(getUrl(), props);
- stmt = conn.prepareStatement("create table UpsertDateVal (k varchar, v date not null, t timestamp" +
+ conn.createStatement().execute("create table UpsertDateVal (k varchar, v date not null, t timestamp" +
", tt time constraint pk primary key (k,v desc))");
- stmt.execute();
} finally {
closeStmtAndConn(stmt, conn);
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
index f62e716..8c1f536 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
@@ -58,14 +58,14 @@ public class JoinQueryCompilerTest extends BaseConnectionlessQueryTest {
" \"item_id\" varchar(10), " +
" price integer, " +
" quantity integer, " +
- " date timestamp)");
+ " \"date\" timestamp)");
conn.createStatement().execute("create table " + JOIN_CUSTOMER_TABLE_FULL_NAME +
" (\"customer_id\" varchar(10) not null primary key, " +
" name varchar, " +
" phone varchar(12), " +
" address varchar, " +
" loc_id varchar(5), " +
- " date date)");
+ " \"date\" date)");
conn.createStatement().execute("create table " + JOIN_ITEM_TABLE_FULL_NAME +
" (\"item_id\" varchar(10) not null primary key, " +
" name varchar, " +
@@ -86,7 +86,7 @@ public class JoinQueryCompilerTest extends BaseConnectionlessQueryTest {
@Test
public void testExplainPlan() throws Exception {
Connection conn = DriverManager.getConnection(getUrl());
- String query = "EXPLAIN SELECT s.\"supplier_id\", \"order_id\", c.name, i.name, quantity, o.\"DATE\" FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o LEFT JOIN "
+ String query = "EXPLAIN SELECT s.\"supplier_id\", \"order_id\", c.name, i.name, quantity, o.\"date\" FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o LEFT JOIN "
+ JOIN_CUSTOMER_TABLE_FULL_NAME + " c ON o.\"customer_id\" = c.\"customer_id\" AND c.name LIKE 'C%' LEFT JOIN "
+ JOIN_ITEM_TABLE_FULL_NAME + " i ON o.\"item_id\" = i.\"item_id\" RIGHT JOIN "
+ JOIN_SUPPLIER_TABLE_FULL_NAME + " s ON s.\"supplier_id\" = i.\"supplier_id\" WHERE i.name LIKE 'T%'";
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 8daefc0..04cd93f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -17,6 +17,8 @@
*/
package org.apache.phoenix.compile;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
import static org.apache.phoenix.util.TestUtil.assertDegenerate;
import static org.junit.Assert.assertArrayEquals;
@@ -500,7 +502,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
"SELECT count(1) FROM atable GROUP BY organization_id,substr(entity_id,1,3),entity_id",
"SELECT count(1) FROM atable GROUP BY entity_id,organization_id",
"SELECT count(1) FROM atable GROUP BY substr(entity_id,1,3),organization_id",
- "SELECT count(1) FROM ptsdb GROUP BY host,inst,round(date,'HOUR')",
+ "SELECT count(1) FROM ptsdb GROUP BY host,inst,round(\"DATE\",'HOUR')",
"SELECT count(1) FROM atable GROUP BY organization_id",
};
List<Object> binds = Collections.emptyList();
@@ -1539,7 +1541,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
try{
Statement statement = conn1.createStatement();
statement.execute("create table example (id integer not null,fn varchar,"
- + "ln varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
+ + "\"ln\" varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
try {
statement.execute("create local index my_idx on example (fn) DEFAULT_COLUMN_FAMILY='F'");
fail();
@@ -1701,7 +1703,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
Integer.MAX_VALUE + "," + Long.MAX_VALUE + "," +
(Integer.MAX_VALUE - 1) + "," + (Long.MAX_VALUE - 1) + "," +
((long)Integer.MAX_VALUE + 1) + "," + oneMoreThanMaxLong +
- " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " LIMIT 1";
+ " FROM " + "\""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_STATS_TABLE + "\"" + " LIMIT 1";
List<Object> binds = Collections.emptyList();
QueryPlan plan = getQueryPlan(query, binds);
RowProjector p = plan.getProjector();
[25/26] phoenix git commit: PHOENIX-3614 Use unique name for index in
TransactionIT#testNonTxToTxTable
Posted by ma...@apache.org.
PHOENIX-3614 Use unique name for index in TransactionIT#testNonTxToTxTable
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/069c371a
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/069c371a
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/069c371a
Branch: refs/heads/calcite
Commit: 069c371aec23a6b87519615eda7eafb895cb1af0
Parents: a44d317
Author: Samarth <sa...@salesforce.com>
Authored: Thu Jan 26 15:49:33 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Jan 26 15:49:33 2017 -0800
----------------------------------------------------------------------
.../src/it/java/org/apache/phoenix/tx/TransactionIT.java | 7 ++++---
1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/069c371a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index bde5cc8..1399f6c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -255,6 +255,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
@Test
public void testNonTxToTxTable() throws Exception {
String nonTxTableName = generateUniqueName();
+ String indexName = generateUniqueName() + "_IDX";
Connection conn = DriverManager.getConnection(getUrl());
conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
@@ -263,7 +264,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (3, 'b')");
conn.commit();
- conn.createStatement().execute("CREATE INDEX IDX ON " + nonTxTableName + "(v)");
+ conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + nonTxTableName + "(v)");
// Reset empty column value to an empty value like it is pre-transactions
HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
@@ -276,7 +277,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
- htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("IDX"));
+ htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName));
assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
@@ -289,7 +290,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (5, 'd')");
rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
- assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "IDX")).isTransactional());
+ assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, indexName)).isTransactional());
assertTrue(rs.next());
assertEquals(1,rs.getInt(1));
assertTrue(rs.next());
[19/26] phoenix git commit: PHOENIX-3468 Double quote SYSTEM, USER,
DATE keywords in IT tests(Rajeshbabu)
Posted by ma...@apache.org.
PHOENIX-3468 Double quote SYSTEM, USER,DATE keywords in IT tests(Rajeshbabu)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/895cb1df
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/895cb1df
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/895cb1df
Branch: refs/heads/calcite
Commit: 895cb1dfca01a9cfffe7d25eaa524f944cb53952
Parents: a675211
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 18:39:39 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 18:39:39 2017 +0530
----------------------------------------------------------------------
.../AlterMultiTenantTableWithViewsIT.java | 2 +-
.../apache/phoenix/end2end/AlterTableIT.java | 16 ++---
.../phoenix/end2end/AlterTableWithViewsIT.java | 2 +-
.../phoenix/end2end/ArithmeticQueryIT.java | 2 +-
.../phoenix/end2end/ArrayFillFunctionIT.java | 28 ++++-----
.../end2end/BaseTenantSpecificTablesIT.java | 4 +-
.../phoenix/end2end/CoalesceFunctionIT.java | 60 +++++++++---------
.../org/apache/phoenix/end2end/DateTimeIT.java | 6 +-
.../org/apache/phoenix/end2end/DeleteIT.java | 26 ++++----
.../phoenix/end2end/EvaluationOfORIT.java | 5 +-
.../apache/phoenix/end2end/GroupByCaseIT.java | 10 +--
.../org/apache/phoenix/end2end/GroupByIT.java | 6 +-
.../apache/phoenix/end2end/HashJoinMoreIT.java | 47 +++++++-------
.../phoenix/end2end/SortMergeJoinMoreIT.java | 37 +++++------
.../org/apache/phoenix/end2end/SortOrderIT.java | 8 +--
.../end2end/TenantSpecificTablesDDLIT.java | 12 ++--
.../end2end/TenantSpecificTablesDMLIT.java | 66 ++++++++++----------
.../apache/phoenix/end2end/UpsertValuesIT.java | 20 +++---
.../phoenix/compile/JoinQueryCompilerTest.java | 6 +-
.../phoenix/compile/QueryCompilerTest.java | 8 ++-
20 files changed, 178 insertions(+), 193 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index adadca7..d6f3a7f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -656,7 +656,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
}
public static void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
- PreparedStatement p = conn.prepareStatement("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME=? AND TABLE_TYPE=?");
+ PreparedStatement p = conn.prepareStatement("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_NAME=? AND TABLE_TYPE=?");
p.setString(1, tableName);
p.setString(2, tableType.getSerializedValue());
ResultSet rs = p.executeQuery();
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index f51c3cb..fee7316 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -996,7 +996,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
assertEquals("COL3",rs.getString(4));
assertFalse(rs.next());
- rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+ rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
+ "WHERE TENANT_ID IS NULL AND\n"
+ "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ dataTableName + "') AND\n"
+ "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1004,7 +1004,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
assertEquals(4,rs.getInt(1));
assertFalse(rs.next());
- rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+ rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
+ "WHERE TENANT_ID IS NULL AND\n"
+ "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ indexTableName + "') AND\n"
+ "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1051,7 +1051,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
assertEquals("COL4",rs.getString(4));
assertFalse(rs.next());
- rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+ rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
+ "WHERE TENANT_ID IS NULL AND\n"
+ "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ dataTableName + "') AND\n"
+ "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1059,7 +1059,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
assertEquals(3,rs.getInt(1));
assertFalse(rs.next());
- rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+ rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
+ "WHERE TENANT_ID IS NULL AND\n"
+ "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ indexTableName + "') AND\n"
+ "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1977,7 +1977,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
Statement stmt = conn.createStatement();
stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR)");
- ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM SYSTEM.CATALOG " +
+ ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM \"SYSTEM\".\"CATALOG\" " +
"WHERE table_name = '"
+ dataTableFullName + "' AND STORE_NULLS IS NOT NULL");
assertTrue(rs.next());
@@ -1987,7 +1987,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
stmt.execute("ALTER TABLE " + dataTableFullName + " SET STORE_NULLS = true");
- rs = stmt.executeQuery("SELECT STORE_NULLS FROM SYSTEM.CATALOG " +
+ rs = stmt.executeQuery("SELECT STORE_NULLS FROM \"SYSTEM\".\"CATALOG\" " +
"WHERE table_name = '" + dataTableFullName
+ "' AND STORE_NULLS IS NOT NULL");
assertTrue(rs.next());
@@ -2102,7 +2102,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
assertTrue(t.isMultiTenant());
// check table metadata updated server side
- ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM SYSTEM.CATALOG "
+ ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM \"SYSTEM\".\"CATALOG\""
+ "WHERE table_name = '"
+ dataTableFullName + "' AND DISABLE_WAL IS NOT NULL AND MULTI_TENANT IS NOT NULL");
assertTrue(rs.next());
@@ -2162,7 +2162,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
}
private void assertIsRowTimestampSet(String schemaName, String tableName, String columnName) throws SQLException {
- String sql = "SELECT IS_ROW_TIMESTAMP FROM SYSTEM.CATALOG WHERE "
+ String sql = "SELECT IS_ROW_TIMESTAMP FROM \"SYSTEM\".\"CATALOG\" WHERE "
+ "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ tableName + "') AND\n"
+ "COLUMN_FAMILY IS NULL AND COLUMN_NAME = ?";
try(Connection conn = DriverManager.getConnection(getUrl())) {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 52dae00..211145e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -585,7 +585,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
public static String getSystemCatalogEntriesForTable(Connection conn, String tableName, String message) throws Exception {
StringBuilder sb = new StringBuilder(message);
sb.append("\n\n\n");
- ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME='"+ tableName +"'");
+ ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_NAME='"+ tableName +"'");
ResultSetMetaData metaData = rs.getMetaData();
int rowNum = 0;
while (rs.next()) {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
index c297441..6126746 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
@@ -1050,7 +1050,7 @@ public class ArithmeticQueryIT extends ParallelStatsDisabledIT {
conn.createStatement().execute(dml);
conn.commit();
- ResultSet rs = conn.createStatement().executeQuery("SELECT 1.2E3 FROM SYSTEM.CATALOG LIMIT 1");
+ ResultSet rs = conn.createStatement().executeQuery("SELECT 1.2E3 FROM \"SYSTEM\".\"CATALOG\" LIMIT 1");
assertTrue(rs.next());
assertTrue(rs.getObject(1) instanceof Double);
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
index dfa0beb..5133e5d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
@@ -44,10 +44,10 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
tableName = generateUniqueName();
Connection conn = DriverManager.getConnection(getUrl());
String ddl = "CREATE TABLE " + tableName
- + " (region_name VARCHAR PRIMARY KEY,length1 INTEGER, length2 INTEGER,\"DATE\" DATE,time TIME,timestamp TIMESTAMP,varchar VARCHAR,integer INTEGER,double DOUBLE,bigint BIGINT,char CHAR(15),double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], varchars2 VARCHAR[])";
+ + " (region_name VARCHAR PRIMARY KEY,length1 INTEGER, length2 INTEGER,\"DATE\" DATE,\"time\" TIME,\"timestamp\" TIMESTAMP,\"varchar\" VARCHAR,\"integer\" INTEGER,\"double\" DOUBLE,\"bigint\" BIGINT,\"char\" CHAR(15),double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], varchars2 VARCHAR[])";
conn.createStatement().execute(ddl);
String dml = "UPSERT INTO " + tableName
- + "(region_name,length1,length2,\"DATE\",time,timestamp,varchar,integer,double,bigint,char,double1,char1,nullcheck,chars2,varchars2) VALUES('SF Bay Area',"
+ + "(region_name,length1,length2,\"DATE\",\"time\",\"timestamp\",\"varchar\",\"integer\",\"double\",\"bigint\",\"char\",double1,char1,nullcheck,chars2,varchars2) VALUES('SF Bay Area',"
+
"0," +
"-3," +
@@ -76,7 +76,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(varchar,5) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"varchar\",5) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
String[] strings = new String[]{"foo", "foo", "foo", "foo", "foo"};
@@ -93,7 +93,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(integer,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"integer\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{34, 34, 34, 34};
@@ -111,7 +111,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(double,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"double\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{23.45, 23.45, 23.45, 23.45};
@@ -129,7 +129,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(bigint,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"bigint\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{34567l, 34567l, 34567l, 34567l};
@@ -147,7 +147,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(char,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"char\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -164,7 +164,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(varchar,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"varchar\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -198,7 +198,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(time,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"time\",3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{new Time(1432102334184l), new Time(1432102334184l), new Time(1432102334184l)};
@@ -215,7 +215,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(timestamp,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+ "SELECT ARRAY_FILL(\"timestamp\",3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
Object[] objects = new Object[]{new Timestamp(1432102334184l), new Timestamp(1432102334184l), new Timestamp(1432102334184l)};
@@ -232,7 +232,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(timestamp,length2) FROM " + tableName
+ "SELECT ARRAY_FILL(\"timestamp\",length2) FROM " + tableName
+ " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
@@ -250,7 +250,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT ARRAY_FILL(timestamp,length1) FROM " + tableName
+ "SELECT ARRAY_FILL(\"timestamp\",length1) FROM " + tableName
+ " WHERE region_name = 'SF Bay Area'");
assertTrue(rs.next());
@@ -513,7 +513,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery(
- "SELECT region_name FROM " + tableName + " WHERE varchar=ANY(ARRAY_FILL('foo',3))");
+ "SELECT region_name FROM " + tableName + " WHERE \"varchar\"=ANY(ARRAY_FILL('foo',3))");
assertTrue(rs.next());
assertEquals("SF Bay Area", rs.getString(1));
@@ -555,7 +555,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
ResultSet rs;
rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
- + " WHERE ARRAY['foo','foo','foo','foo','foo']=ARRAY_FILL(varchar,5)");
+ + " WHERE ARRAY['foo','foo','foo','foo','foo']=ARRAY_FILL(\"varchar\",5)");
assertTrue(rs.next());
assertEquals("SF Bay Area", rs.getString(1));
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
index 0749a20..5e9208b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
@@ -56,7 +56,7 @@ public abstract class BaseTenantSpecificTablesIT extends ParallelStatsEnabledIT
PARENT_TABLE_NAME_NO_TENANT_TYPE_ID = "P_" + BaseTest.generateUniqueName();
TENANT_TABLE_NAME_NO_TENANT_TYPE_ID = "V_" + BaseTest.generateUniqueName();
PARENT_TABLE_DDL = "CREATE TABLE " + PARENT_TABLE_NAME + " ( \n" +
- " user VARCHAR ,\n" +
+ " \"user\" VARCHAR ,\n" +
" tenant_id VARCHAR NOT NULL,\n" +
" tenant_type_id VARCHAR(3) NOT NULL, \n" +
" id INTEGER NOT NULL\n" +
@@ -65,7 +65,7 @@ public abstract class BaseTenantSpecificTablesIT extends ParallelStatsEnabledIT
" tenant_col VARCHAR) AS SELECT *\n" +
" FROM " + PARENT_TABLE_NAME + " WHERE tenant_type_id= '" + TENANT_TYPE_ID + "'";
PARENT_TABLE_DDL_NO_TENANT_TYPE_ID = "CREATE TABLE " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " ( \n" +
- " user VARCHAR ,\n" +
+ " \"user\" VARCHAR ,\n" +
" tenant_id VARCHAR NOT NULL,\n" +
" id INTEGER NOT NULL,\n" +
" CONSTRAINT pk PRIMARY KEY (tenant_id, id)) MULTI_TENANT=true, IMMUTABLE_ROWS=true";
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index 0666f77..828051f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -69,16 +69,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String tableName = generateUniqueName();
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
- + " COUNT BIGINT "
+ + " \"COUNT\" BIGINT "
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
conn.commit();
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(SUM(COUNT), CAST(0 AS BIGINT)) " //explicitly def long
+ + "COALESCE(SUM(\"COUNT\"), CAST(0 AS BIGINT)) " //explicitly def long
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -93,16 +93,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String tableName = generateUniqueName();
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
- + " COUNT BIGINT "
+ + " \"COUNT\" BIGINT "
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
conn.commit();
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(SUM(COUNT), 0) " // no long def
+ + "COALESCE(SUM(\"COUNT\"), 0) " // no long def
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -117,16 +117,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String tableName = generateUniqueName();
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
- + " COUNT BIGINT "
+ + " \"COUNT\" BIGINT "
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
conn.commit();
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(SUM(COUNT), SUM(ID)) " // second param as expression
+ + "COALESCE(SUM(\"COUNT\"), SUM(ID)) " // second param as expression
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -142,16 +142,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
- + " COUNT BIGINT " //first parameter to coalesce
+ + " \"COUNT\" BIGINT " //first parameter to coalesce
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
conn.commit();
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 0) " //second param is int
+ + "COALESCE(NTH_VALUE(\"COUNT\", 100) WITHIN GROUP (ORDER BY \"COUNT\" DESC), 0) " //second param is int
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -166,17 +166,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String tableName = generateUniqueName();
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
- + " COUNT UNSIGNED_INT " //first parameter to coalesce
+ + " \"COUNT\" UNSIGNED_INT " //first parameter to coalesce
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + " (ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + " (ID, \"COUNT\") VALUES(2, null)");
conn.commit();
//second param to coalesce is signed int
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + " COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 1) "
+ + " COALESCE(NTH_VALUE(\"COUNT\", 100) WITHIN GROUP (ORDER BY COUNT DESC), 1) "
+ " FROM " + tableName
+ " GROUP BY ID");
@@ -193,21 +193,21 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String ddl = "CREATE TABLE " + tableName + "("
+ " ID BIGINT NOT NULL, "
+ " DATE TIMESTAMP NOT NULL, "
- + " COUNT BIGINT "
+ + " \"COUNT\" BIGINT "
+ " CONSTRAINT pk PRIMARY KEY(ID, DATE))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 1)");
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 2)");
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(2, CURRENT_TIME(), 1)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(1, CURRENT_TIME(), 1)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(1, CURRENT_TIME(), 2)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(2, CURRENT_TIME(), 1)");
conn.commit();
//second param to coalesce is signed int
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
+ " COALESCE("
- + " NTH_VALUE(COUNT, 2000)" // should evaluate null
- + " WITHIN GROUP (ORDER BY COUNT DESC),"
+ + " NTH_VALUE(\"COUNT\", 2000)" // should evaluate null
+ + " WITHIN GROUP (ORDER BY \"COUNT\" DESC),"
+ " 0)"
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -224,17 +224,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String ddl = "CREATE TABLE " + tableName + "("
+ " ID UNSIGNED_INT NOT NULL, "
- + " COUNT UNSIGNED_INT "
+ + " \"COUNT\" UNSIGNED_INT "
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
conn.commit();
try {
conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(MIN(COUNT), -1) " // invalid value for UNSIGNED_INT
+ + "COALESCE(MIN(\"COUNT\"), -1) " // invalid value for UNSIGNED_INT
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -251,17 +251,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String ddl = "CREATE TABLE " + tableName + "("
+ " ID INTEGER NOT NULL, "
- + " COUNT UNSIGNED_INT " //first parameter to coalesce
+ + " \"COUNT\" UNSIGNED_INT " //first parameter to coalesce
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(-2, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(-2, null)");
conn.commit();
try {
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(MIN(COUNT), ID) "
+ + "COALESCE(MIN(\"COUNT\"), ID) "
+ "FROM " + tableName
+ " GROUP BY ID");
@@ -280,16 +280,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
String ddl = "CREATE TABLE " + tableName + "("
+ " ID DOUBLE NOT NULL, "
- + " COUNT INTEGER " //first parameter to coalesce
+ + " \"COUNT\" INTEGER " //first parameter to coalesce
+ " CONSTRAINT pk PRIMARY KEY(ID))";
conn.createStatement().execute(ddl);
- conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2.0, null)");
+ conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2.0, null)");
conn.commit();
ResultSet rs = conn.createStatement().executeQuery(
"SELECT "
- + "COALESCE(MIN(COUNT), ID) "
+ + "COALESCE(MIN(\"COUNT\"), ID) "
+ "FROM " + tableName
+ " GROUP BY ID");
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index 74cc068..ad45d5e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -395,7 +395,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
Statement stmt = conn.createStatement();
ResultSet rs =
stmt.executeQuery(String
- .format("SELECT %s FROM SYSTEM.CATALOG LIMIT 1", invocation));
+ .format("SELECT %s FROM \"SYSTEM\".\"CATALOG\" LIMIT 1", invocation));
assertTrue(rs.next());
int returnValue = rs.getInt(1);
assertFalse(rs.next());
@@ -815,7 +815,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
public void testLiteralDateTimestampComparison() throws Exception {
ResultSet rs =
conn.createStatement().executeQuery(
- "select DATE '2016-05-10 00:00:00' > TIMESTAMP '2016-05-11 00:00:00'");
+ "select \"DATE\" '2016-05-10 00:00:00' > \"TIMESTAMP\" '2016-05-11 00:00:00'");
assertTrue(rs.next());
assertEquals(false, rs.getBoolean(1));
@@ -826,7 +826,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
public void testLiteralDateTimestampComparison2() throws Exception {
ResultSet rs =
conn.createStatement().executeQuery(
- "select TIMESTAMP '2016-05-10 00:00:00' > DATE '2016-05-11 00:00:00'");
+ "select \"TIMESTAMP\" '2016-05-10 00:00:00' > \"DATE\" '2016-05-11 00:00:00'");
assertTrue(rs.next());
assertEquals(false, rs.getBoolean(1));
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 9f514b3..09e1021 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -299,11 +299,11 @@ public class DeleteIT extends ParallelStatsDisabledIT {
"HOST CHAR(2) NOT NULL," +
"DOMAIN VARCHAR NOT NULL, " +
"FEATURE VARCHAR NOT NULL, " +
- "DATE DATE NOT NULL, \n" +
+ "\"DATE\" DATE NOT NULL, \n" +
"USAGE.CORE BIGINT," +
"USAGE.DB BIGINT," +
"STATS.ACTIVE_VISITOR INTEGER " +
- "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE))" + (isSalted ? " SALT_BUCKETS=3" : "");
+ "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\"))" + (isSalted ? " SALT_BUCKETS=3" : "");
stm.execute(s);
String localIndexName = generateUniqueName();
String indexName = generateUniqueName();
@@ -315,7 +315,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
stm.close();
PreparedStatement psInsert = con
- .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+ .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
psInsert.setString(1, "AA");
psInsert.setString(2, "BB");
psInsert.setString(3, "CC");
@@ -380,18 +380,18 @@ public class DeleteIT extends ParallelStatsDisabledIT {
"HOST CHAR(2) NOT NULL," +
"DOMAIN VARCHAR NOT NULL, " +
"FEATURE VARCHAR NOT NULL, " +
- "DATE DATE NOT NULL, \n" +
+ "\"DATE\" DATE NOT NULL, \n" +
"USAGE.CORE BIGINT," +
"USAGE.DB BIGINT," +
"STATS.ACTIVE_VISITOR INTEGER " +
- "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE)) IMMUTABLE_ROWS=true");
- stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (DATE, FEATURE)");
- stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (DATE, FEATURE, USAGE.DB)");
+ "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\")) IMMUTABLE_ROWS=true");
+ stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (\"DATE\", FEATURE)");
+ stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (\"DATE\", FEATURE, USAGE.DB)");
stm.close();
Date date = new Date(0);
PreparedStatement psInsert = con
- .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+ .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
psInsert.setString(1, "AA");
psInsert.setString(2, "BB");
psInsert.setString(3, "CC");
@@ -405,7 +405,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
con.commit();
}
- psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, DATE) = (?,?,?,?)");
+ psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)");
psInsert.setString(1, "AA");
psInsert.setString(2, "BB");
psInsert.setString(3, "CC");
@@ -427,7 +427,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
stm.execute("DROP INDEX " + indexName2 + " ON " + tableName);
stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (USAGE.DB)");
- stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, DATE)");
+ stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, \"DATE\")");
try{
psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE USAGE.DB=2");
} catch(Exception e) {
@@ -465,15 +465,15 @@ public class DeleteIT extends ParallelStatsDisabledIT {
" HOST CHAR(2) NOT NULL," +
"DOMAIN VARCHAR NOT NULL, " +
"FEATURE VARCHAR NOT NULL, " +
- "DATE DATE NOT NULL, \n" +
+ "\"DATE\" DATE NOT NULL, \n" +
"USAGE.CORE BIGINT," +
"USAGE.DB BIGINT," +
"STATS.ACTIVE_VISITOR INTEGER " +
- "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE))");
+ "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\"))");
stm.close();
PreparedStatement psInsert = con
- .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+ .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
psInsert.setString(1, "AA");
psInsert.setString(2, "BB");
psInsert.setString(3, "CC");
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
index 1e450de..8edbaa6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
@@ -39,7 +39,7 @@ public class EvaluationOfORIT extends ParallelStatsDisabledIT{
public void testFalseOrFalse() throws SQLException {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
Connection conn = DriverManager.getConnection(getUrl(), props);
- ResultSet rs = conn.createStatement().executeQuery("SELECT (FALSE OR FALSE) AS B FROM SYSTEM.CATALOG LIMIT 1");
+ ResultSet rs = conn.createStatement().executeQuery("SELECT (FALSE OR FALSE) AS B FROM \"SYSTEM\".\"CATALOG\" LIMIT 1");
assertTrue(rs.next());
assertFalse(rs.getBoolean(1));
conn.close();
@@ -53,8 +53,7 @@ public class EvaluationOfORIT extends ParallelStatsDisabledIT{
conn.setAutoCommit(false);
String create = "CREATE TABLE " + tableName + " ( ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR(50))";
- PreparedStatement createStmt = conn.prepareStatement(create);
- createStmt.execute();
+ conn.createStatement().execute(create);
PreparedStatement stmt = conn.prepareStatement(
"upsert into " +
tableName + " VALUES (?, ?)");
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
index 56a94b3..5ceb412 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
@@ -412,21 +412,21 @@ public class GroupByCaseIT extends ParallelStatsDisabledIT {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
Connection conn = DriverManager.getConnection(getUrl(), props);
String tableName = generateUniqueName();
- ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+ ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
assertTrue(rs.next());
assertEquals(0,rs.getInt(1));
initAvgGroupTable(conn, tableName, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=20 ");
testAvgGroupByOrderPreserving(conn, tableName, 13);
- rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+ rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
assertTrue(rs.next());
assertEquals(13,rs.getInt(1));
conn.setAutoCommit(true);
- conn.createStatement().execute("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME);
- rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+ conn.createStatement().execute("DELETE FROM " + "\"SYSTEM\".\"STATS\"");
+ rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
assertTrue(rs.next());
assertEquals(0,rs.getInt(1));
TestUtil.doMajorCompaction(conn, tableName);
- rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+ rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
assertTrue(rs.next());
assertEquals(13,rs.getInt(1));
testAvgGroupByOrderPreserving(conn, tableName, 13);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
index f0c781f..b9fa15b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
@@ -576,7 +576,7 @@ public class GroupByIT extends BaseQueryIT {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
Connection conn = DriverManager.getConnection(getUrl(), props);
- String ddl = "create table test1(time integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(time, hostname))";
+ String ddl = "create table test1(\"time\" integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(\"time\", hostname))";
conn.createStatement().execute(ddl);
conn.close();
@@ -596,7 +596,7 @@ public class GroupByIT extends BaseQueryIT {
props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
conn = DriverManager.getConnection(getUrl(), props);
ResultSet rs;
- stmt = conn.prepareStatement("select time/10 as tm, hostname, avg(usage) from test1 group by hostname, tm");
+ stmt = conn.prepareStatement("select \"time\"/10 as tm, hostname, avg(usage) from test1 group by hostname, tm");
rs = stmt.executeQuery();
assertTrue(rs.next());
assertEquals(143985345, rs.getInt(1));
@@ -612,7 +612,7 @@ public class GroupByIT extends BaseQueryIT {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
Connection conn = DriverManager.getConnection(getUrl(), props);
- String ddl = "create table test1(time integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(time, hostname))";
+ String ddl = "create table test1(\"time\" integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(\"time\", hostname))";
conn.createStatement().execute(ddl);
conn.close();
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
index 52e8006..c76eb67 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
@@ -236,13 +236,8 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
+ " colA2 VARCHAR " + "CONSTRAINT PK PRIMARY KEY" + "(pkA)" + ")";
String ddlB = "CREATE TABLE " + tableB + " (pkB INTEGER NOT NULL PRIMARY KEY, " + " colB INTEGER)";
- stmt = conn.prepareStatement(ddlA);
- stmt.execute();
- stmt.close();
-
- stmt = conn.prepareStatement(ddlB);
- stmt.execute();
- stmt.close();
+ conn.createStatement().execute(ddlA);
+ conn.createStatement().execute(ddlB);
String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
stmt = conn.prepareStatement(upsertA);
@@ -498,15 +493,15 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
" TENANTID UNSIGNED_INT NOT NULL" +
",ID UNSIGNED_INT NOT NULL" +
",FOO UNSIGNED_INT NOT NULL" +
- ",TIMESTAMP UNSIGNED_LONG NOT NULL" +
+ ",\"TIMESTAMP\" UNSIGNED_LONG NOT NULL" +
",CODES INTEGER ARRAY[] NOT NULL" +
",V UNSIGNED_LONG" +
- " CONSTRAINT pk PRIMARY KEY (TENANTID, ID, FOO, TIMESTAMP, CODES))" +
+ " CONSTRAINT pk PRIMARY KEY (TENANTID, ID, FOO, \"TIMESTAMP\" , CODES))" +
" DEFAULT_COLUMN_FAMILY ='E'," +
" MULTI_TENANT=true");
PreparedStatement upsertStmt = conn.prepareStatement(
"upsert into INVENTORY "
- + "(tenantid, id, foo, timestamp, codes) "
+ + "(tenantid, id, foo, \"TIMESTAMP\" , codes) "
+ "values (?, ?, ?, ?, ?)");
upsertStmt.setInt(1, 15);
upsertStmt.setInt(2, 5);
@@ -672,16 +667,16 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
"CREATE TABLE IF NOT EXISTS EVENT_COUNT (\n" +
" BUCKET VARCHAR,\n" +
" TIMESTAMP_DATE TIMESTAMP,\n" +
- " TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+ " \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
" LOCATION VARCHAR,\n" +
" A VARCHAR,\n" +
" B VARCHAR,\n" +
" C VARCHAR,\n" +
" D UNSIGNED_LONG,\n" +
" E FLOAT\n" +
- " CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP DESC, LOCATION, A, B, C)\n" +
+ " CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\" DESC, LOCATION, A, B, C)\n" +
") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
- PreparedStatement stmt = conn.prepareStatement("UPSERT INTO EVENT_COUNT(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
+ PreparedStatement stmt = conn.prepareStatement("UPSERT INTO EVENT_COUNT(BUCKET, \"TIMESTAMP\", LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
stmt.setString(1, "5SEC");
stmt.setString(3, "Tr/Bal");
stmt.setString(4, "A1");
@@ -733,7 +728,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
"CREATE TABLE IF NOT EXISTS " + t[i] + " (\n" +
" BUCKET VARCHAR,\n" +
" TIMESTAMP_DATE TIMESTAMP,\n" +
- " TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+ " \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
" SRC_LOCATION VARCHAR,\n" +
" DST_LOCATION VARCHAR,\n" +
" B VARCHAR,\n" +
@@ -742,9 +737,9 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
" G UNSIGNED_LONG,\n" +
" H UNSIGNED_LONG,\n" +
" I UNSIGNED_LONG\n" +
- " CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
+ " CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\"" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
- stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
+ stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
stmt.setString(1, "5SEC");
stmt.setString(3, "Tr/Bal");
stmt.setString(4, "Tr/Bal");
@@ -772,25 +767,25 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
stmt.execute();
String q =
- "SELECT C.BUCKET, C.TIMESTAMP FROM (\n" +
- " SELECT E.BUCKET as BUCKET, L.BUCKET as LBUCKET, E.TIMESTAMP as TIMESTAMP, L.TIMESTAMP as LTIMESTAMP FROM\n" +
- " (SELECT BUCKET, TIMESTAMP FROM EVENT_COUNT\n" +
+ "SELECT C.BUCKET, C.\"TIMESTAMP\" FROM (\n" +
+ " SELECT E.BUCKET as BUCKET, L.BUCKET as LBUCKET, E.\"TIMESTAMP\" as TIMESTAMP, L.\"TIMESTAMP\" as LTIMESTAMP FROM\n" +
+ " (SELECT BUCKET, \"TIMESTAMP\" FROM EVENT_COUNT\n" +
" WHERE BUCKET = '5SEC' AND LOCATION = 'Tr/Bal'\n" +
- " AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
+ " AND \"TIMESTAMP\" <= 1462993520000000000 AND \"TIMESTAMP\" > 1462993420000000000\n" +
" ) E\n" +
" JOIN\n" +
- " (SELECT BUCKET, TIMESTAMP FROM "+ t[i] +"\n" +
+ " (SELECT BUCKET, \"TIMESTAMP\" FROM "+ t[i] +"\n" +
" WHERE BUCKET = '5SEC' AND SRC_LOCATION = 'Tr/Bal' AND SRC_LOCATION = DST_LOCATION\n" +
- " AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
+ " AND \"TIMESTAMP\" <= 1462993520000000000 AND \"TIMESTAMP\" > 1462993420000000000\n" +
" ) L\n" +
- " ON L.BUCKET = E.BUCKET AND L.TIMESTAMP = E.TIMESTAMP\n" +
+ " ON L.BUCKET = E.BUCKET AND L.\"TIMESTAMP\" = E.\"TIMESTAMP\"\n" +
" ) C\n" +
- " GROUP BY C.BUCKET, C.TIMESTAMP";
+ " GROUP BY C.BUCKET, C.\"TIMESTAMP\"";
String p = i == 0 ?
"CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
" SERVER FILTER BY FIRST KEY ONLY\n" +
- " SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]\n" +
+ " SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, \"E.TIMESTAMP\"]\n" +
"CLIENT MERGE SORT\n" +
" PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
" CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
@@ -799,7 +794,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
:
"CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
" SERVER FILTER BY FIRST KEY ONLY\n" +
- " SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]\n" +
+ " SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, \"E.TIMESTAMP\"]\n" +
"CLIENT MERGE SORT\n" +
" PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
" CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',1462993420000000001,'Tr/Bal'] - [1,'5SEC',1462993520000000000,'Tr/Bal']\n" +
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index a08ba39..992e55f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -186,13 +186,8 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
+ " colA2 VARCHAR " + "CONSTRAINT PK PRIMARY KEY" + "(pkA)" + ")";
String ddlB = "CREATE TABLE " + tableB + " (pkB INTEGER NOT NULL PRIMARY KEY, " + " colB INTEGER)";
- stmt = conn.prepareStatement(ddlA);
- stmt.execute();
- stmt.close();
-
- stmt = conn.prepareStatement(ddlB);
- stmt.execute();
- stmt.close();
+ conn.createStatement().execute(ddlA);
+ conn.createStatement().execute(ddlB);
String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
stmt = conn.prepareStatement(upsertA);
@@ -313,16 +308,16 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
"CREATE TABLE IF NOT EXISTS " + eventCountTableName + " (\n" +
" BUCKET VARCHAR,\n" +
" TIMESTAMP_DATE TIMESTAMP,\n" +
- " TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+ " \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
" LOCATION VARCHAR,\n" +
" A VARCHAR,\n" +
" B VARCHAR,\n" +
" C VARCHAR,\n" +
" D UNSIGNED_LONG,\n" +
" E FLOAT\n" +
- " CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP DESC, LOCATION, A, B, C)\n" +
+ " CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\" DESC, LOCATION, A, B, C)\n" +
") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
- PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + eventCountTableName + "(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
+ PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + eventCountTableName + "(BUCKET, \"TIMESTAMP\", LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
stmt.setString(1, "5SEC");
stmt.setString(3, "Tr/Bal");
stmt.setString(4, "A1");
@@ -374,7 +369,7 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
"CREATE TABLE IF NOT EXISTS " + t[i] + " (\n" +
" BUCKET VARCHAR,\n" +
" TIMESTAMP_DATE TIMESTAMP,\n" +
- " TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+ " \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
" SRC_LOCATION VARCHAR,\n" +
" DST_LOCATION VARCHAR,\n" +
" B VARCHAR,\n" +
@@ -383,9 +378,9 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
" G UNSIGNED_LONG,\n" +
" H UNSIGNED_LONG,\n" +
" I UNSIGNED_LONG\n" +
- " CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
+ " CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\"" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
- stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
+ stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
stmt.setString(1, "5SEC");
stmt.setString(3, "Tr/Bal");
stmt.setString(4, "Tr/Bal");
@@ -421,10 +416,10 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
" GROUP BY BUCKET, TIMESTAMP, LOCATION\n" +
" ) E\n" +
" JOIN\n" +
- " (SELECT BUCKET, TIMESTAMP FROM "+ t[i] +"\n" +
+ " (SELECT BUCKET, \"TIMESTAMP\" FROM "+ t[i] +"\n" +
" WHERE BUCKET = '5SEC' AND SRC_LOCATION = 'Tr/Bal' AND SRC_LOCATION = DST_LOCATION\n" +
- " AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
- " GROUP BY BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION\n" +
+ " AND \"TIMESTAMP\" <= 1462993520000000000 AND \"TIMESTAMP\" > 1462993420000000000\n" +
+ " GROUP BY BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION\n" +
" ) L\n" +
" ON L.BUCKET = E.BUCKET AND L.TIMESTAMP = E.TIMESTAMP\n" +
" ) C\n" +
@@ -441,10 +436,10 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
"AND (SKIP MERGE)\n" +
" CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
" SERVER FILTER BY FIRST KEY ONLY AND SRC_LOCATION = DST_LOCATION\n" +
- " SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
- " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
+ " SERVER DISTINCT PREFIX FILTER OVER [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
+ " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
" CLIENT MERGE SORT\n" +
- " CLIENT SORTED BY [BUCKET, TIMESTAMP]\n" +
+ " CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]"
:
@@ -458,8 +453,8 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
"AND (SKIP MERGE)\n" +
" CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',1462993420000000001,'Tr/Bal'] - [1,'5SEC',1462993520000000000,'Tr/Bal']\n" +
" SERVER FILTER BY FIRST KEY ONLY AND SRC_LOCATION = DST_LOCATION\n" +
- " SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
- " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
+ " SERVER DISTINCT PREFIX FILTER OVER [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
+ " SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
" CLIENT MERGE SORT\n" +
"CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
"CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]";
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index c810844..655dbb1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -370,7 +370,7 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
@Test
public void queryDescDateWithExplicitOrderBy() throws Exception {
String table = generateUniqueName();
- String ddl = "CREATE table " + table + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 DATE NOT NULL, c3 CHAR(1) NOT NULL " +
+ String ddl = "CREATE table " + table + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 \"DATE\" NOT NULL, c3 CHAR(1) NOT NULL " +
"constraint pk primary key (c1, c2, d1 DESC, c3))";
Object[] row1 = {"1", "2", date(10, 11, 2001), "3"};
Object[] row2 = {"1", "2", date(10, 11, 2003), "3"};
@@ -408,7 +408,7 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
@Test
public void lessThanLeadingDescCompositePK() throws Exception {
String table = generateUniqueName();
- String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, date))";
+ String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, \"DATE\"))";
Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2012)}, {3, date(1, 1, 2013)}, {2, date(1, 1, 2011)}};
Object[][] expectedRows = new Object[][]{{1, date(1, 1, 2012)}};
runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("id", "<", "2"),
@@ -418,10 +418,10 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
@Test
public void lessThanTrailingDescCompositePK() throws Exception {
String table = generateUniqueName();
- String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, date))";
+ String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, \"DATE\"))";
Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2002)}, {3, date(1, 1, 2003)}, {2, date(1, 1, 2001)}};
Object[][] expectedRows = new Object[][]{{2, date(1, 1, 2001)}};
- runQueryTest(ddl, upsert("id", "DATE"), insertedRows, expectedRows, new WhereCondition("DATE", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"),
+ runQueryTest(ddl, upsert("id", "\"DATE\""), insertedRows, expectedRows, new WhereCondition("\"DATE\"", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"),
table);
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index b8f8177..e4479ac 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -152,7 +152,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
public void testCreationOfParentTableFailsOnTenantSpecificConnection() throws Exception {
try {
createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL, "CREATE TABLE " + generateUniqueName() + "( \n" +
- " user VARCHAR ,\n" +
+ " \"user\" VARCHAR ,\n" +
" id INTEGER not null primary key desc\n" +
" ) ");
fail();
@@ -178,7 +178,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
}
String newDDL =
"CREATE TABLE DIFFSCHEMA." + PARENT_TABLE_NAME + " ( \n" +
- " user VARCHAR ,\n" +
+ " \"user\" VARCHAR ,\n" +
" tenant_id VARCHAR(5) NOT NULL,\n" +
" tenant_type_id VARCHAR(3) NOT NULL, \n" +
" id INTEGER NOT NULL\n" +
@@ -199,7 +199,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
@Test(expected=ColumnAlreadyExistsException.class)
public void testTenantSpecificTableCannotOverrideParentCol() throws SQLException {
createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL, "CREATE VIEW " + generateUniqueName() + " ( \n" +
- " user INTEGER) AS SELECT *\n" +
+ " \"user\" INTEGER) AS SELECT *\n" +
" FROM " + PARENT_TABLE_NAME);
}
@@ -285,7 +285,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
// try removing a non-PK col, which is allowed
try {
- conn.createStatement().execute("alter table " + PARENT_TABLE_NAME + " drop column user");
+ conn.createStatement().execute("alter table " + PARENT_TABLE_NAME + " drop column \"user\"");
}
catch (SQLException expected) {
fail("We should be able to drop a non pk base table column");
@@ -525,7 +525,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
// make sure tenants see parent table's columns and their own
rs = meta.getColumns(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME), null);
assertTrue(rs.next());
- assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "user", 1);
+ assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "\"user\"", 1);
assertTrue(rs.next());
// (tenant_id column is not visible in tenant-specific connection)
assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "tenant_type_id", 2);
@@ -538,7 +538,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
rs = meta.getColumns(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID), null);
assertTrue(rs.next());
- assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "user", 1);
+ assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "\"user\"", 1);
assertTrue(rs.next());
// (tenant_id column is not visible in tenant-specific connection)
assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "id", 2);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
index 76625c0..55d38fd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
@@ -174,12 +174,12 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
- ResultSet rs = conn.createStatement().executeQuery("select user from " + TENANT_TABLE_NAME);
+ ResultSet rs = conn.createStatement().executeQuery("select \"user\" from " + TENANT_TABLE_NAME);
assertTrue("Expected 1 row in result set", rs.next());
assertEquals("Billy Gibbons", rs.getString(1));
assertFalse("Expected 1 row in result set", rs.next());
@@ -202,9 +202,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
conn.setAutoCommit(true);
@@ -231,14 +231,14 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
Connection tsConn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
tsConn.setAutoCommit(true);
- tsConn.createStatement().executeUpdate("create index idx1 on " + TENANT_TABLE_NAME + "(user)");
- int count = tsConn.createStatement().executeUpdate("delete from " + TENANT_TABLE_NAME + " where user='Billy Gibbons'");
+ tsConn.createStatement().executeUpdate("create index idx1 on " + TENANT_TABLE_NAME + "(\"user\")");
+ int count = tsConn.createStatement().executeUpdate("delete from " + TENANT_TABLE_NAME + " where \"user\"='Billy Gibbons'");
assertEquals("Expected 1 row have been deleted", 1, count);
ResultSet rs = tsConn.createStatement().executeQuery("select * from " + TENANT_TABLE_NAME);
assertFalse("Expected no rows in result set", rs.next());
@@ -261,9 +261,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('AC/DC', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
Connection tsConn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
tsConn.setAutoCommit(true);
@@ -289,9 +289,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
analyzeTable(tsConn, PARENT_TABLE_NAME);
tsConn.createStatement().execute("delete from " + TENANT_TABLE_NAME);
@@ -315,9 +315,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('AC/DC', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
tsConn.createStatement().execute("drop view " + TENANT_TABLE_NAME_NO_TENANT_TYPE_ID);
@@ -340,12 +340,12 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'aaa', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
analyzeTable(tsConn, TENANT_TABLE_NAME);
- int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user) select id+100, user from " + TENANT_TABLE_NAME);
+ int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, \"user\") select id+100, \"user\" from " + TENANT_TABLE_NAME);
tsConn.commit();
assertEquals("Expected 1 row to have been inserted", 1, count);
@@ -371,14 +371,14 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
conn.setAutoCommit(true);
conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'aaa', 1, 'Bon Scott')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
analyzeTable(tsConn, TENANT_TABLE_NAME);
tsConn.setAutoCommit(true);
- int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user)"
- + "select id+100, user from " + anotherTableName + " where id=2");
+ int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, \"user\")"
+ + "select id+100, \"user\" from " + anotherTableName + " where id=2");
assertEquals("Expected 1 row to have been inserted", 1, count);
ResultSet rs = tsConn.createStatement().executeQuery("select count(*) from " + TENANT_TABLE_NAME);
rs.next();
@@ -395,7 +395,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
Connection conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
try {
- int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, user) values (1, 'Bon Scott')");
+ int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, \"user\") values (1, 'Bon Scott')");
conn.commit();
assertEquals("Expected 1 row to have been inserted", 1, count);
ResultSet rs = conn.createStatement().executeQuery("select count(*) from " + TENANT_TABLE_NAME);
@@ -414,10 +414,10 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
try {
ResultSet rs = conn.createStatement().executeQuery("select * from " + PARENT_TABLE_NAME);
assertFalse(rs.next());
- conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_type_id, id, user) values ('" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+ conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_type_id, id, \"user\") values ('" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
conn.commit();
analyzeTable(conn, PARENT_TABLE_NAME);
- rs = conn.createStatement().executeQuery("select user from " + PARENT_TABLE_NAME);
+ rs = conn.createStatement().executeQuery("select \"user\" from " + PARENT_TABLE_NAME);
assertTrue(rs.next());
assertEquals(rs.getString(1),"Billy Gibbons");
assertFalse(rs.next());
[20/26] phoenix git commit: PHOENIX-3608 KeyRange interset should
return EMPTY_RANGE when one of it is NULL_RANGE(Rajeshbabu)
Posted by ma...@apache.org.
PHOENIX-3608 KeyRange interset should return EMPTY_RANGE when one of it is NULL_RANGE(Rajeshbabu)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e7ef25ec
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e7ef25ec
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e7ef25ec
Branch: refs/heads/calcite
Commit: e7ef25eca2468e6d0a154b5e3539219f07748f22
Parents: 895cb1d
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 19:05:06 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 19:05:06 2017 +0530
----------------------------------------------------------------------
.../src/main/java/org/apache/phoenix/query/KeyRange.java | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7ef25ec/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
index f4bf793..babce9d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -356,10 +356,9 @@ public class KeyRange implements Writable {
boolean newUpperInclusive;
// Special case for null, is it is never included another range
// except for null itself.
- if (this == IS_NULL_RANGE) {
- if (range == IS_NULL_RANGE) {
+ if (this == IS_NULL_RANGE && range == IS_NULL_RANGE) {
return IS_NULL_RANGE;
- }
+ } else if(this == IS_NULL_RANGE || range == IS_NULL_RANGE) {
return EMPTY_RANGE;
}
if (lowerUnbound()) {
[22/26] phoenix git commit: PHOENIX-3611 Cache for client connections
will expire (and close) entries in LRU fashion.
Posted by ma...@apache.org.
PHOENIX-3611 Cache for client connections will expire (and close) entries in LRU fashion.
Signed-off-by: Andrew Purtell <ap...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/9b7f3ca5
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/9b7f3ca5
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/9b7f3ca5
Branch: refs/heads/calcite
Commit: 9b7f3ca5b2e6bbf0757d99fbe48b3065e79d6066
Parents: 4e4f7dd
Author: Geoffrey <gj...@salesforce.com>
Authored: Thu Jan 19 16:08:20 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Jan 20 16:16:50 2017 -0800
----------------------------------------------------------------------
.../org/apache/phoenix/jdbc/PhoenixDriver.java | 99 +++++++++++++-------
.../org/apache/phoenix/query/QueryServices.java | 5 +-
.../phoenix/query/QueryServicesOptions.java | 2 +
3 files changed, 70 insertions(+), 36 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
index fa31dd9..b2acacf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -23,20 +23,13 @@ import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.annotation.concurrent.GuardedBy;
+import com.google.common.cache.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.phoenix.exception.SQLExceptionCode;
import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -50,7 +43,6 @@ import org.apache.phoenix.query.QueryServicesOptions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -147,13 +139,43 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
}
// One entry per cluster here
- private final ConcurrentMap<ConnectionInfo,ConnectionQueryServices> connectionQueryServicesMap = new ConcurrentHashMap<ConnectionInfo,ConnectionQueryServices>(3);
+ private final Cache<ConnectionInfo, ConnectionQueryServices> connectionQueryServicesCache =
+ initializeConnectionCache();
public PhoenixDriver() { // for Squirrel
// Use production services implementation
super();
}
-
+
+ private Cache<ConnectionInfo, ConnectionQueryServices> initializeConnectionCache() {
+ Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+ int maxCacheSize = config.getInt(QueryServices.CLIENT_CONNECTION_CACHE_MAX_SIZE,
+ QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_CACHE_MAX_SIZE);
+ int maxCacheDuration = config.getInt(QueryServices.CLIENT_CONNECTION_CACHE_MAX_DURATION_MILLISECONDS,
+ QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION);
+ RemovalListener<ConnectionInfo, ConnectionQueryServices> cacheRemovalListener =
+ new RemovalListener<ConnectionInfo, ConnectionQueryServices>() {
+ @Override
+ public void onRemoval(RemovalNotification<ConnectionInfo, ConnectionQueryServices> notification) {
+ String connInfoIdentifier = notification.getKey().toString();
+ logger.debug("Expiring " + connInfoIdentifier + " because of "
+ + notification.getCause().name());
+
+ try {
+ notification.getValue().close();
+ }
+ catch (SQLException se) {
+ logger.error("Error while closing expired cache connection " + connInfoIdentifier, se);
+ }
+ }
+ };
+ return CacheBuilder.newBuilder()
+ .maximumSize(maxCacheSize)
+ .expireAfterAccess(maxCacheDuration, TimeUnit.MILLISECONDS)
+ .removalListener(cacheRemovalListener)
+ .build();
+ }
+
// writes guarded by "this"
private volatile QueryServices services;
@@ -206,38 +228,49 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
}
@Override
- protected ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+ protected ConnectionQueryServices getConnectionQueryServices(String url, final Properties info) throws SQLException {
try {
lockInterruptibly(LockMode.READ);
checkClosed();
ConnectionInfo connInfo = ConnectionInfo.create(url);
- QueryServices services = getQueryServices();
- // Also performs the Kerberos login if the URL/properties request this
- ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps(), info);
- ConnectionQueryServices connectionQueryServices = connectionQueryServicesMap.get(normalizedConnInfo);
- if (connectionQueryServices == null) {
- if (normalizedConnInfo.isConnectionless()) {
- connectionQueryServices = new ConnectionlessQueryServicesImpl(services, normalizedConnInfo, info);
- } else {
- connectionQueryServices = new ConnectionQueryServicesImpl(services, normalizedConnInfo, info);
- }
- ConnectionQueryServices prevValue = connectionQueryServicesMap.putIfAbsent(normalizedConnInfo, connectionQueryServices);
- if (prevValue != null) {
- connectionQueryServices = prevValue;
- }
- }
- boolean success = false;
SQLException sqlE = null;
+ boolean success = false;
+ final QueryServices services = getQueryServices();
+ ConnectionQueryServices connectionQueryServices = null;
+ // Also performs the Kerberos login if the URL/properties request this
+ final ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps(), info);
try {
+ connectionQueryServices =
+ connectionQueryServicesCache.get(normalizedConnInfo, new Callable<ConnectionQueryServices>() {
+ @Override
+ public ConnectionQueryServices call() throws Exception {
+ ConnectionQueryServices connectionQueryServices;
+ if (normalizedConnInfo.isConnectionless()) {
+ connectionQueryServices = new ConnectionlessQueryServicesImpl(services, normalizedConnInfo, info);
+ } else {
+ connectionQueryServices = new ConnectionQueryServicesImpl(services, normalizedConnInfo, info);
+ }
+
+ return connectionQueryServices;
+ }
+ });
+
connectionQueryServices.init(url, info);
success = true;
- } catch (SQLException e) {
+ } catch (ExecutionException ee){
+ if (ee.getCause() instanceof SQLException) {
+ sqlE = (SQLException) ee.getCause();
+ } else {
+ throw new SQLException(ee);
+ }
+ }
+ catch (SQLException e) {
sqlE = e;
}
finally {
if (!success) {
// Remove from map, as initialization failed
- connectionQueryServicesMap.remove(normalizedConnInfo);
+ connectionQueryServicesCache.invalidate(normalizedConnInfo);
if (sqlE != null) {
throw sqlE;
}
@@ -319,8 +352,4 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
}
}
- @VisibleForTesting
- protected ConcurrentMap<ConnectionInfo,ConnectionQueryServices> getCachedConnections() {
- return this.connectionQueryServicesMap;
- }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 14d9887..233007f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -233,7 +233,10 @@ public interface QueryServices extends SQLCloseable {
public static final String CLIENT_CACHE_ENCODING = "phoenix.table.client.cache.encoding";
public static final String AUTO_UPGRADE_ENABLED = "phoenix.autoupgrade.enabled";
-
+
+ public static final String CLIENT_CONNECTION_CACHE_MAX_SIZE = "phoenix.client.connection.cache.max.size";
+ public static final String CLIENT_CONNECTION_CACHE_MAX_DURATION_MILLISECONDS =
+ "phoenix.client.connection.max.duration";
/**
* Get executor service used for parallel scans
*/
http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 0e8b9d5..df203b5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -262,6 +262,8 @@ public class QueryServicesOptions {
public static final String DEFAULT_CLIENT_CACHE_ENCODING = PTableRefFactory.Encoding.OBJECT.toString();
public static final boolean DEFAULT_AUTO_UPGRADE_ENABLED = true;
+ public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_SIZE = 100;
+ public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION = 86400000;
@SuppressWarnings("serial")
public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {
[13/26] phoenix git commit: PHOENIX-3563 Ensure we release ZooKeeper
resources allocated by the Tephra client embedded in the Phoenix connection
Posted by ma...@apache.org.
PHOENIX-3563 Ensure we release ZooKeeper resources allocated by the Tephra client embedded in the Phoenix connection
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/d8f45949
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/d8f45949
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/d8f45949
Branch: refs/heads/calcite
Commit: d8f4594989c0b73945aaffec5649a0b62ac59724
Parents: a164f03
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Jan 4 16:48:44 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Sat Jan 7 10:03:23 2017 -0800
----------------------------------------------------------------------
.../query/ConnectionQueryServicesImpl.java | 37 ++++++++++++--------
1 file changed, 23 insertions(+), 14 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d8f45949/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index f66b358..be34f66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -254,6 +254,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
private final boolean returnSequenceValues ;
private HConnection connection;
+ private ZKClientService txZKClientService;
private TransactionServiceClient txServiceClient;
private volatile boolean initialized;
private volatile int nSequenceSaltBuckets;
@@ -370,15 +371,16 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
// Create instance of the tephra zookeeper client
- ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-
- ZKClientService zkClientService = ZKClientServices.delegate(
- ZKClients.reWatchOnExpire(
- ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
- )
+ txZKClientService = ZKClientServices.delegate(
+ ZKClients.reWatchOnExpire(
+ ZKClients.retryOnFailure(
+ new TephraZKClientService(zkQuorumServersString, timeOut, null,
+ ArrayListMultimap.<String, byte[]>create()),
+ RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+ )
);
- zkClientService.startAndWait();
- ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
+ txZKClientService.startAndWait();
+ ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(txZKClientService);
PooledClientProvider pooledClientProvider = new PooledClientProvider(
config, zkDiscoveryService);
this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
@@ -389,11 +391,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
boolean transactionsEnabled = props.getBoolean(
QueryServices.TRANSACTIONS_ENABLED,
QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
- // only initialize the tx service client if needed
+ this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
+ // only initialize the tx service client if needed and if we succeeded in getting a connection
+ // to HBase
if (transactionsEnabled) {
initTxServiceClient();
}
- this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
} catch (IOException e) {
throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
.setRootCause(e).build().buildException();
@@ -463,14 +466,20 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
} finally {
try {
childServices.clear();
- if (renewLeaseExecutor != null) {
- renewLeaseExecutor.shutdownNow();
- }
synchronized (latestMetaDataLock) {
latestMetaData = null;
latestMetaDataLock.notifyAll();
}
- if (connection != null) connection.close();
+ try {
+ // close the HBase connection
+ if (connection != null) connection.close();
+ } finally {
+ if (renewLeaseExecutor != null) {
+ renewLeaseExecutor.shutdownNow();
+ }
+ // shut down the tx client service if we created one to support transactions
+ if (this.txZKClientService != null) this.txZKClientService.stopAndWait();
+ }
} catch (IOException e) {
if (sqlE == null) {
sqlE = ServerUtil.parseServerException(e);
[17/26] phoenix git commit: PHOENIX-3134 varbinary fields bulk load
difference between MR/psql and upserts
Posted by ma...@apache.org.
PHOENIX-3134 varbinary fields bulk load difference between MR/psql and upserts
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/a6752119
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/a6752119
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/a6752119
Branch: refs/heads/calcite
Commit: a675211909415ca376e432d25f8a8822aadf5712
Parents: 7332fc3
Author: Ankit Singhal <an...@gmail.com>
Authored: Wed Jan 18 13:27:17 2017 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Wed Jan 18 13:27:17 2017 +0530
----------------------------------------------------------------------
.../expression/function/EncodeFormat.java | 4 +-
.../phoenix/mapreduce/CsvBulkImportUtil.java | 7 +-
.../phoenix/mapreduce/CsvBulkLoadTool.java | 12 ++-
.../org/apache/phoenix/query/QueryServices.java | 3 +
.../phoenix/query/QueryServicesOptions.java | 9 ++-
.../apache/phoenix/schema/types/PBinary.java | 6 +-
.../apache/phoenix/schema/types/PVarbinary.java | 5 +-
.../org/apache/phoenix/util/PhoenixRuntime.java | 11 +++
.../phoenix/util/csv/CsvUpsertExecutor.java | 25 ++++++
.../phoenix/util/json/JsonUpsertExecutor.java | 44 +++++++++++
.../mapreduce/CsvBulkImportUtilTest.java | 8 +-
.../util/AbstractUpsertExecutorTest.java | 82 +++++++++++++++++---
.../phoenix/util/csv/CsvUpsertExecutorTest.java | 26 +++----
.../util/json/JsonUpsertExecutorTest.java | 6 ++
14 files changed, 207 insertions(+), 41 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
index ca6cb66..8130228 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
@@ -20,5 +20,7 @@ package org.apache.phoenix.expression.function;
public enum EncodeFormat {
HEX, //format for encoding HEX value to bytes
- BASE62 //format for encoding a base 10 long value to base 62 string
+ BASE62, //format for encoding a base 10 long value to base 62 string
+ BASE64, //format for encoding a base 10 long value to base 64 string
+ ASCII // Plain Text
};
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
index 9289dbf..ff9ff72 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
import com.google.common.annotations.VisibleForTesting;
@@ -38,15 +39,19 @@ public class CsvBulkImportUtil {
* @param quoteChar quote character for the CSV input
* @param escapeChar escape character for the CSV input
* @param arrayDelimiter array delimiter character, can be null
+ * @param binaryEncoding
*/
public static void initCsvImportJob(Configuration conf, char fieldDelimiter, char quoteChar,
- char escapeChar, String arrayDelimiter) {
+ char escapeChar, String arrayDelimiter, String binaryEncoding) {
setChar(conf, CsvToKeyValueMapper.FIELD_DELIMITER_CONFKEY, fieldDelimiter);
setChar(conf, CsvToKeyValueMapper.QUOTE_CHAR_CONFKEY, quoteChar);
setChar(conf, CsvToKeyValueMapper.ESCAPE_CHAR_CONFKEY, escapeChar);
if (arrayDelimiter != null) {
conf.set(CsvToKeyValueMapper.ARRAY_DELIMITER_CONFKEY, arrayDelimiter);
}
+ if(binaryEncoding!=null){
+ conf.set(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING, binaryEncoding);
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
index 8ed66b8..14b8c34 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
@@ -35,6 +35,7 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
static final Option QUOTE_OPT = new Option("q", "quote", true, "Supply a custom phrase delimiter, defaults to double quote character");
static final Option ESCAPE_OPT = new Option("e", "escape", true, "Supply a custom escape character, default is a backslash");
static final Option ARRAY_DELIMITER_OPT = new Option("a", "array-delimiter", true, "Array element delimiter (optional)");
+ static final Option binaryEncodingOption = new Option("b", "binaryEncoding", true, "Specifies binary encoding");
@Override
protected Options getOptions() {
@@ -43,6 +44,7 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
options.addOption(QUOTE_OPT);
options.addOption(ESCAPE_OPT);
options.addOption(ARRAY_DELIMITER_OPT);
+ options.addOption(binaryEncodingOption);
return options;
}
@@ -79,13 +81,19 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
}
escapeChar = escapeString.charAt(0);
}
-
+
+ String binaryEncoding = null;
+ if (cmdLine.hasOption(binaryEncodingOption.getOpt())) {
+ binaryEncoding = cmdLine.getOptionValue(binaryEncodingOption.getOpt());
+ }
+
CsvBulkImportUtil.initCsvImportJob(
conf,
delimiterChar,
quoteChar,
escapeChar,
- cmdLine.getOptionValue(ARRAY_DELIMITER_OPT.getOpt()));
+ cmdLine.getOptionValue(ARRAY_DELIMITER_OPT.getOpt()),
+ binaryEncoding);
}
@Override
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 1e002d2..14d9887 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -225,6 +225,9 @@ public interface QueryServices extends SQLCloseable {
public static final String INDEX_POPULATION_SLEEP_TIME = "phoenix.index.population.wait.time";
public static final String LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB = "phoenix.client.localIndexUpgrade";
public static final String LIMITED_QUERY_SERIAL_THRESHOLD = "phoenix.limited.query.serial.threshold";
+
+ //currently BASE64 and ASCII is supported
+ public static final String UPLOAD_BINARY_DATA_TYPE_ENCODING = "phoenix.upload.binaryDataType.encoding";
public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index ef3ac39..0e8b9d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -79,6 +79,7 @@ import static org.apache.phoenix.query.QueryServices.STATS_USE_CURRENT_TIME_ATTR
import static org.apache.phoenix.query.QueryServices.THREAD_POOL_SIZE_ATTRIB;
import static org.apache.phoenix.query.QueryServices.THREAD_TIMEOUT_MS_ATTRIB;
import static org.apache.phoenix.query.QueryServices.TRANSACTIONS_ENABLED;
+import static org.apache.phoenix.query.QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING;
import static org.apache.phoenix.query.QueryServices.USE_BYTE_BASED_REGEX_ATTRIB;
import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
@@ -98,6 +99,7 @@ import org.apache.phoenix.util.DateUtil;
import org.apache.phoenix.util.ReadOnlyProps;
+
/**
* Options for {@link QueryServices}.
*
@@ -271,6 +273,10 @@ public class QueryServicesOptions {
}
};
public static final String DEFAULT_SCHEMA = null;
+ public static final String DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING = "BASE64"; // for backward compatibility, till
+ // 4.10, psql and CSVBulkLoad
+ // expects binary data to be base 64
+ // encoded
private final Configuration config;
@@ -338,7 +344,8 @@ public class QueryServicesOptions {
.setIfUnset(IS_NAMESPACE_MAPPING_ENABLED, DEFAULT_IS_NAMESPACE_MAPPING_ENABLED)
.setIfUnset(IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE)
.setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)
- .setIfUnset(AUTO_UPGRADE_ENABLED, DEFAULT_AUTO_UPGRADE_ENABLED);
+ .setIfUnset(AUTO_UPGRADE_ENABLED, DEFAULT_AUTO_UPGRADE_ENABLED)
+ .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
// HBase sets this to 1, so we reset it to something more appropriate.
// Hopefully HBase will change this, because we can't know if a user set
// it to 1, so we'll change it.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index 43906f0..3a9dcc7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -22,7 +22,6 @@ import java.text.Format;
import java.util.Arrays;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.phoenix.exception.DataExceedsCapacityException;
import org.apache.phoenix.query.QueryConstants;
@@ -182,10 +181,7 @@ public class PBinary extends PBinaryBase {
@Override
public Object toObject(String value) {
- if (value == null || value.length() == 0) {
- return null;
- }
- return Base64.decode(value);
+ return PVarbinary.INSTANCE.toObject(value);
}
@Override
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
index d96650d..b3ce57a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
@@ -131,7 +131,10 @@ public class PVarbinary extends PBinaryBase {
if (value == null || value.length() == 0) {
return null;
}
- return Base64.decode(value);
+ Object object = Base64.decode(value);
+ if (object == null) { throw newIllegalDataException(
+ "Input: [" + value + "] is not base64 encoded"); }
+ return object;
}
@Override
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index dbac76f..3c16d00 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -213,6 +213,9 @@ public class PhoenixRuntime {
if (execCmd.isLocalIndexUpgrade()) {
props.setProperty(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, "false");
}
+ if (execCmd.binaryEncoding != null) {
+ props.setProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING, execCmd.binaryEncoding);
+ }
conn = DriverManager.getConnection(jdbcUrl, props).unwrap(PhoenixConnection.class);
conn.setRunningUpgrade(true);
if (execCmd.isMapNamespace()) {
@@ -532,6 +535,7 @@ public class PhoenixRuntime {
private boolean mapNamespace;
private String srcTable;
private boolean localIndexUpgrade;
+ private String binaryEncoding;
/**
* Factory method to build up an {@code ExecutionCommand} based on supplied parameters.
@@ -539,6 +543,8 @@ public class PhoenixRuntime {
public static ExecutionCommand parseArgs(String[] args) {
Option tableOption = new Option("t", "table", true,
"Overrides the table into which the CSV data is loaded and is case sensitive");
+ Option binaryEncodingOption = new Option("b", "binaryEncoding", true,
+ "Specifies binary encoding");
Option headerOption = new Option("h", "header", true, "Overrides the column names to" +
" which the CSV data maps and is case sensitive. A special value of " +
"in-line indicating that the first line of the CSV file determines the " +
@@ -588,6 +594,7 @@ public class PhoenixRuntime {
options.addOption(bypassUpgradeOption);
options.addOption(mapNamespaceOption);
options.addOption(localIndexUpgradeOption);
+ options.addOption(binaryEncodingOption);
CommandLineParser parser = new PosixParser();
CommandLine cmdLine = null;
@@ -606,6 +613,10 @@ public class PhoenixRuntime {
if (cmdLine.hasOption(tableOption.getOpt())) {
execCmd.tableName = cmdLine.getOptionValue(tableOption.getOpt());
}
+
+ if (cmdLine.hasOption(binaryEncodingOption.getOpt())) {
+ execCmd.binaryEncoding = cmdLine.getOptionValue(binaryEncodingOption.getOpt());
+ }
if (cmdLine.hasOption(headerOption.getOpt())) {
String columnString = cmdLine.getOptionValue(headerOption.getOpt());
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
index 0d3e17d..cd40b44 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
@@ -27,12 +27,18 @@ import java.util.Properties;
import javax.annotation.Nullable;
import org.apache.commons.csv.CSVRecord;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.function.EncodeFormat;
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PBoolean;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PDataType.PDataCodec;
import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
import org.apache.phoenix.util.ColumnInfo;
import org.apache.phoenix.util.DateUtil;
import org.apache.phoenix.util.UpsertExecutor;
@@ -116,6 +122,7 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
private final PDataType dataType;
private final PDataCodec codec;
private final DateUtil.DateTimeParser dateTimeParser;
+ private final String binaryEncoding;
SimpleDatatypeConversionFunction(PDataType dataType, Connection conn) {
Properties props;
@@ -148,6 +155,8 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
this.dateTimeParser = null;
}
this.codec = codec;
+ this.binaryEncoding = props.getProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,
+ QueryServicesOptions.DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
}
@Nullable
@@ -175,6 +184,22 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
throw new RuntimeException("Invalid boolean value: '" + input
+ "', must be one of ['true','t','1','false','f','0']");
}
+ }else if (dataType == PVarbinary.INSTANCE || dataType == PBinary.INSTANCE){
+ EncodeFormat format = EncodeFormat.valueOf(binaryEncoding.toUpperCase());
+ Object object = null;
+ switch (format) {
+ case BASE64:
+ object = Base64.decode(input);
+ if (object == null) { throw new IllegalDataException(
+ "Input: [" + input + "] is not base64 encoded"); }
+ break;
+ case ASCII:
+ object = Bytes.toBytes(input);
+ break;
+ default:
+ throw new IllegalDataException("Unsupported encoding \"" + binaryEncoding + "\"");
+ }
+ return object;
}
return dataType.toObject(input);
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
index bbe0e30..ffa797d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
@@ -25,12 +25,20 @@ import java.sql.Types;
import java.util.List;
import java.util.Map;
import java.util.Properties;
+
import javax.annotation.Nullable;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.function.EncodeFormat;
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PBinary;
+import org.apache.phoenix.schema.types.PBoolean;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
import org.apache.phoenix.util.ColumnInfo;
import org.apache.phoenix.util.DateUtil;
import org.apache.phoenix.util.UpsertExecutor;
@@ -137,6 +145,7 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
private final PDataType dataType;
private final DateUtil.DateTimeParser dateTimeParser;
+ private final String binaryEncoding;
SimpleDatatypeConversionFunction(PDataType dataType, Connection conn) {
Properties props;
@@ -166,6 +175,8 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
} else {
this.dateTimeParser = null;
}
+ this.binaryEncoding = props.getProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,
+ QueryServicesOptions.DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
}
@Nullable
@@ -180,7 +191,40 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
byte[] byteValue = new byte[dataType.getByteSize()];
dataType.getCodec().encodeLong(epochTime, byteValue, 0);
return dataType.toObject(byteValue);
+ }else if (dataType == PBoolean.INSTANCE) {
+ switch (input.toString()) {
+ case "true":
+ case "t":
+ case "T":
+ case "1":
+ return Boolean.TRUE;
+ case "false":
+ case "f":
+ case "F":
+ case "0":
+ return Boolean.FALSE;
+ default:
+ throw new RuntimeException("Invalid boolean value: '" + input
+ + "', must be one of ['true','t','1','false','f','0']");
+ }
+ }else if (dataType == PVarbinary.INSTANCE || dataType == PBinary.INSTANCE){
+ EncodeFormat format = EncodeFormat.valueOf(binaryEncoding.toUpperCase());
+ Object object = null;
+ switch (format) {
+ case BASE64:
+ object = Base64.decode(input.toString());
+ if (object == null) { throw new IllegalDataException(
+ "Input: [" + input + "] is not base64 encoded"); }
+ break;
+ case ASCII:
+ object = Bytes.toBytes(input.toString());
+ break;
+ default:
+ throw new IllegalDataException("Unsupported encoding \"" + binaryEncoding + "\"");
}
+ return object;
+ }
+
return dataType.toObject(input, dataType);
}
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
index 3c6271a..33c72a8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
@@ -17,6 +17,9 @@
*/
package org.apache.phoenix.mapreduce;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
@@ -28,9 +31,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
public class CsvBulkImportUtilTest {
@Test
@@ -41,7 +41,7 @@ public class CsvBulkImportUtilTest {
char quote = '\002';
char escape = '!';
- CsvBulkImportUtil.initCsvImportJob(conf, delimiter, quote, escape, null);
+ CsvBulkImportUtil.initCsvImportJob(conf, delimiter, quote, escape, null, null);
// Serialize and deserialize the config to ensure that there aren't any issues
// with non-printable characters as delimiters
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
index 61b03fb..2b2544d 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
@@ -17,6 +17,12 @@
*/
package org.apache.phoenix.util;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
import java.io.IOException;
import java.sql.Connection;
import java.sql.DriverManager;
@@ -25,9 +31,14 @@ import java.sql.SQLException;
import java.sql.Types;
import java.util.Arrays;
import java.util.List;
+import java.util.Properties;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBinary;
import org.apache.phoenix.schema.types.PInteger;
import org.apache.phoenix.schema.types.PIntegerArray;
import org.junit.After;
@@ -36,12 +47,6 @@ import org.junit.Test;
import com.google.common.collect.ImmutableList;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionlessQueryTest {
protected Connection conn;
@@ -51,6 +56,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
protected abstract UpsertExecutor<R, F> getUpsertExecutor();
protected abstract R createRecord(Object... columnValues) throws IOException;
+ protected abstract UpsertExecutor<R, F> getUpsertExecutor(Connection conn);
@Before
public void setUp() throws SQLException {
@@ -59,7 +65,8 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
new ColumnInfo("NAME", Types.VARCHAR),
new ColumnInfo("AGE", Types.INTEGER),
new ColumnInfo("VALUES", PIntegerArray.INSTANCE.getSqlType()),
- new ColumnInfo("BEARD", Types.BOOLEAN));
+ new ColumnInfo("BEARD", Types.BOOLEAN),
+ new ColumnInfo("PIC", Types.BINARY));
preparedStatement = mock(PreparedStatement.class);
upsertListener = mock(UpsertExecutor.UpsertListener.class);
@@ -73,8 +80,10 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
@Test
public void testExecute() throws Exception {
+ byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+ String encodedBinaryData = Base64.encodeBytes(binaryData);
getUpsertExecutor().execute(createRecord(123L, "NameValue", 42,
- Arrays.asList(1, 2, 3), true));
+ Arrays.asList(1, 2, 3), true, encodedBinaryData));
verify(upsertListener).upsertDone(1L);
verifyNoMoreInteractions(upsertListener);
@@ -84,6 +93,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
verify(preparedStatement).setObject(3, Integer.valueOf(42));
verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
verify(preparedStatement).setObject(5, Boolean.TRUE);
+ verify(preparedStatement).setObject(6, binaryData);
verify(preparedStatement).execute();
verifyNoMoreInteractions(preparedStatement);
}
@@ -99,8 +109,10 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
@Test
public void testExecute_TooManyFields() throws Exception {
+ byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+ String encodedBinaryData = Base64.encodeBytes(binaryData);
R recordWithTooManyFields = createRecord(123L, "NameValue", 42, Arrays.asList(1, 2, 3),
- true, "Garbage");
+ true, encodedBinaryData, "garbage");
getUpsertExecutor().execute(recordWithTooManyFields);
verify(upsertListener).upsertDone(1L);
@@ -111,14 +123,17 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
verify(preparedStatement).setObject(3, Integer.valueOf(42));
verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
verify(preparedStatement).setObject(5, Boolean.TRUE);
+ verify(preparedStatement).setObject(6, binaryData);
verify(preparedStatement).execute();
verifyNoMoreInteractions(preparedStatement);
}
@Test
public void testExecute_NullField() throws Exception {
+ byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+ String encodedBinaryData = Base64.encodeBytes(binaryData);
getUpsertExecutor().execute(createRecord(123L, "NameValue", null,
- Arrays.asList(1, 2, 3), false));
+ Arrays.asList(1, 2, 3), false, encodedBinaryData));
verify(upsertListener).upsertDone(1L);
verifyNoMoreInteractions(upsertListener);
@@ -128,17 +143,62 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
verify(preparedStatement).setNull(3, columnInfoList.get(2).getSqlType());
verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
verify(preparedStatement).setObject(5, Boolean.FALSE);
+ verify(preparedStatement).setObject(6, binaryData);
verify(preparedStatement).execute();
verifyNoMoreInteractions(preparedStatement);
}
@Test
public void testExecute_InvalidType() throws Exception {
+ byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+ String encodedBinaryData = Base64.encodeBytes(binaryData);
R recordWithInvalidType = createRecord(123L, "NameValue", "ThisIsNotANumber",
- Arrays.asList(1, 2, 3), true);
+ Arrays.asList(1, 2, 3), true, encodedBinaryData);
getUpsertExecutor().execute(recordWithInvalidType);
verify(upsertListener).errorOnRecord(eq(recordWithInvalidType), any(Throwable.class));
verifyNoMoreInteractions(upsertListener);
}
+
+ @Test
+ public void testExecute_InvalidBoolean() throws Exception {
+ byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+ String encodedBinaryData = Base64.encodeBytes(binaryData);
+ R csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,NotABoolean,"+encodedBinaryData);
+ getUpsertExecutor().execute(csvRecordWithInvalidType);
+
+ verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
+ }
+
+ @Test
+ public void testExecute_InvalidBinary() throws Exception {
+ String notBase64Encoded="#@$df";
+ R csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,true,"+notBase64Encoded);
+ getUpsertExecutor().execute(csvRecordWithInvalidType);
+
+ verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
+ }
+
+ @Test
+ public void testExecute_AsciiEncoded() throws Exception {
+ String asciiValue="#@$df";
+ Properties info=new Properties();
+ info.setProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,"ASCII");
+ getUpsertExecutor(DriverManager.getConnection(getUrl(),info)).execute(createRecord(123L, "NameValue", 42,
+ Arrays.asList(1, 2, 3), true, asciiValue));
+
+ verify(upsertListener).upsertDone(1L);
+ verifyNoMoreInteractions(upsertListener);
+
+ verify(preparedStatement).setObject(1, Long.valueOf(123L));
+ verify(preparedStatement).setObject(2, "NameValue");
+ verify(preparedStatement).setObject(3, Integer.valueOf(42));
+ verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
+ verify(preparedStatement).setObject(5, Boolean.TRUE);
+ verify(preparedStatement).setObject(6, Bytes.toBytes(asciiValue));
+ verify(preparedStatement).execute();
+ verifyNoMoreInteractions(preparedStatement);
+ }
+
+
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
index c887ff7..a5ec4fa 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
@@ -18,23 +18,19 @@
package org.apache.phoenix.util.csv;
import java.io.IOException;
+import java.sql.Connection;
import java.sql.SQLException;
import java.util.List;
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
import org.apache.commons.csv.CSVFormat;
import org.apache.commons.csv.CSVParser;
import org.apache.commons.csv.CSVRecord;
import org.apache.phoenix.util.AbstractUpsertExecutorTest;
import org.apache.phoenix.util.UpsertExecutor;
import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.verify;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord, String> {
@@ -46,7 +42,13 @@ public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord,
public UpsertExecutor<CSVRecord, String> getUpsertExecutor() {
return upsertExecutor;
}
-
+
+ @Override
+ public UpsertExecutor<CSVRecord, String> getUpsertExecutor(Connection conn) {
+ return new CsvUpsertExecutor(conn, columnInfoList, preparedStatement,
+ upsertListener, ARRAY_SEP);
+ }
+
@Override
public CSVRecord createRecord(Object... columnValues) throws IOException {
for (int i = 0; i < columnValues.length; i++) {
@@ -69,11 +71,5 @@ public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord,
upsertListener, ARRAY_SEP);
}
- @Test
- public void testExecute_InvalidBoolean() throws Exception {
- CSVRecord csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,NotABoolean");
- upsertExecutor.execute(ImmutableList.of(csvRecordWithInvalidType));
-
- verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
- }
+
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
index c042dd4..6ac9cf9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
@@ -18,6 +18,7 @@
package org.apache.phoenix.util.json;
import java.io.IOException;
+import java.sql.Connection;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.Map;
@@ -50,4 +51,9 @@ public class JsonUpsertExecutorTest extends AbstractUpsertExecutorTest<Map<?, ?>
super.setUp();
upsertExecutor = new JsonUpsertExecutor(conn, columnInfoList, preparedStatement, upsertListener);
}
+
+ @Override
+ protected UpsertExecutor<Map<?, ?>, Object> getUpsertExecutor(Connection conn) {
+ return new JsonUpsertExecutor(conn, columnInfoList, preparedStatement, upsertListener);
+ }
}
[09/26] phoenix git commit: PHOENIX-3516 Performance Issues with
queries that have compound filters and specify
phoenix.query.force.rowkeyorder=true (addendum)
Posted by ma...@apache.org.
PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true (addendum)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/07f92732
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/07f92732
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/07f92732
Branch: refs/heads/calcite
Commit: 07f92732f9c6d2d9464012cebeb4cefc10da95d5
Parents: b326995
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Dec 22 17:29:35 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Dec 27 11:44:54 2016 -0800
----------------------------------------------------------------------
.../main/java/org/apache/phoenix/compile/WhereOptimizer.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/07f92732/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 0681925..5e15f61 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -439,7 +439,12 @@ public class WhereOptimizer {
if (l.size() == 1) {
return l.get(0);
}
- return new AndExpression(l);
+ try {
+ return AndExpression.create(l);
+ } catch (SQLException e) {
+ //shouldn't happen
+ throw new RuntimeException(e);
+ }
}
return node;
}
[24/26] phoenix git commit: PHOENIX-541 Make mutable batch size
bytes-based instead of row-based (Geoffrey Jacoby)
Posted by ma...@apache.org.
PHOENIX-541 Make mutable batch size bytes-based instead of row-based (Geoffrey Jacoby)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/a44d317e
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/a44d317e
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/a44d317e
Branch: refs/heads/calcite
Commit: a44d317e32d383c4964824448bf12dfb8ed8bfec
Parents: b9323e1
Author: Samarth <sa...@salesforce.com>
Authored: Thu Jan 26 15:31:43 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Jan 26 15:31:43 2017 -0800
----------------------------------------------------------------------
.../org/apache/phoenix/end2end/QueryMoreIT.java | 35 ++++++++++++++
.../end2end/UpsertSelectAutoCommitIT.java | 2 +-
.../apache/phoenix/end2end/UpsertSelectIT.java | 2 +-
.../phoenix/end2end/index/ImmutableIndexIT.java | 1 -
.../org/apache/phoenix/tx/TransactionIT.java | 2 +-
.../org/apache/phoenix/tx/TxCheckpointIT.java | 2 +-
.../UngroupedAggregateRegionObserver.java | 31 +++++++-----
.../apache/phoenix/execute/MutationState.java | 51 ++++++++++++++++++--
.../apache/phoenix/jdbc/PhoenixConnection.java | 10 +++-
.../index/PhoenixIndexImportDirectMapper.java | 19 +++++---
.../org/apache/phoenix/query/QueryServices.java | 3 ++
.../phoenix/query/QueryServicesOptions.java | 7 ++-
.../java/org/apache/phoenix/util/JDBCUtil.java | 6 +++
.../org/apache/phoenix/util/PhoenixRuntime.java | 12 ++++-
.../apache/phoenix/jdbc/PhoenixDriverTest.java | 9 ++++
.../org/apache/phoenix/util/JDBCUtilTest.java | 21 +++++++-
16 files changed, 181 insertions(+), 32 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 2b27f00..a2dab16 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
@@ -37,6 +37,8 @@ import java.util.Properties;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.util.PhoenixRuntime;
import org.apache.phoenix.util.TestUtil;
import org.junit.Test;
@@ -471,4 +473,37 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
assertFalse(rs.next());
}
}
+
+ @Test
+ public void testMutationBatch() throws Exception {
+ Properties connectionProperties = new Properties();
+ connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, "1024");
+ PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+ String fullTableName = generateUniqueName();
+ try (Statement stmt = connection.createStatement()) {
+ stmt.execute("CREATE TABLE " + fullTableName + "(\n" +
+ " ORGANIZATION_ID CHAR(15) NOT NULL,\n" +
+ " SCORE DOUBLE NOT NULL,\n" +
+ " ENTITY_ID CHAR(15) NOT NULL\n" +
+ " CONSTRAINT PAGE_SNAPSHOT_PK PRIMARY KEY (\n" +
+ " ORGANIZATION_ID,\n" +
+ " SCORE DESC,\n" +
+ " ENTITY_ID DESC\n" +
+ " )\n" +
+ ") MULTI_TENANT=TRUE");
+ }
+ PreparedStatement stmt = connection.prepareStatement("upsert into " + fullTableName +
+ " (organization_id, entity_id, score) values (?,?,?)");
+ try {
+ for (int i = 0; i < 4; i++) {
+ stmt.setString(1, "AAAA" + i);
+ stmt.setString(2, "BBBB" + i);
+ stmt.setInt(3, 1);
+ stmt.execute();
+ }
+ connection.commit();
+ } catch (IllegalArgumentException expected) {}
+
+ assertEquals(2L, connection.getMutationState().getBatchCount());
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index 37482de..6b781a0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -152,7 +152,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
@Test
public void testUpsertSelectDoesntSeeUpsertedData() throws Exception {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
- props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+ props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
Connection conn = DriverManager.getConnection(getUrl(), props);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 763f11b..f5905ee 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -1435,7 +1435,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
long ts = nextTimestamp();
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
- props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+ props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
Connection conn = DriverManager.getConnection(getUrl(), props);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 3ee9721..bc301fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -154,7 +154,6 @@ public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
return;
}
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
- props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(100));
String tableName = "TBL_" + generateUniqueName();
String indexName = "IND_" + generateUniqueName();
String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 83128f1..bde5cc8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -603,7 +603,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
@Test
public void testParallelUpsertSelect() throws Exception {
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
- props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+ props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
Connection conn = DriverManager.getConnection(getUrl(), props);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 14bcd70..246ecd4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -80,7 +80,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
String seqName = "SEQ_" + generateUniqueName();
String fullTableName = SchemaUtil.getTableName(tableName, tableName);
Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
- props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+ props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
Connection conn = getConnection(props);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 9ee0054..a888bb2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB;
import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT;
import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT;
@@ -70,6 +71,7 @@ import org.apache.hadoop.io.WritableUtils;
import org.apache.phoenix.cache.ServerCacheClient;
import org.apache.phoenix.coprocessor.generated.PTableProtos;
import org.apache.phoenix.exception.DataExceedsCapacityException;
+import org.apache.phoenix.execute.MutationState;
import org.apache.phoenix.execute.TupleProjector;
import org.apache.phoenix.expression.Expression;
import org.apache.phoenix.expression.ExpressionType;
@@ -343,6 +345,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
}
int batchSize = 0;
+ long batchSizeBytes = 0L;
List<Mutation> mutations = Collections.emptyList();
boolean needToWrite = false;
Configuration conf = c.getEnvironment().getConfiguration();
@@ -369,6 +372,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
// TODO: size better
mutations = Lists.newArrayListWithExpectedSize(1024);
batchSize = env.getConfiguration().getInt(MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE);
+ batchSizeBytes = env.getConfiguration().getLong(MUTATE_BATCH_SIZE_BYTES_ATTRIB,
+ QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE_BYTES);
}
Aggregators aggregators = ServerAggregators.deserialize(
scan.getAttribute(BaseScannerRegionObserver.AGGREGATORS), env.getConfiguration());
@@ -596,19 +601,23 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
mutations.add(put);
}
}
- // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
- if (!mutations.isEmpty() && batchSize > 0 &&
- mutations.size() % batchSize == 0) {
- commitBatch(region, mutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr,
- txState);
- mutations.clear();
+ // Commit in batches based on UPSERT_BATCH_SIZE_BYTES_ATTRIB in config
+ List<List<Mutation>> batchMutationList =
+ MutationState.getMutationBatchList(batchSize, batchSizeBytes, mutations);
+ for (List<Mutation> batchMutations : batchMutationList) {
+ commitBatch(region, batchMutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr,
+ txState);
+ batchMutations.clear();
}
- // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
- if (!indexMutations.isEmpty() && batchSize > 0 &&
- indexMutations.size() % batchSize == 0) {
- commitBatch(region, indexMutations, null, blockingMemStoreSize, null, txState);
- indexMutations.clear();
+ mutations.clear();
+ // Commit in batches based on UPSERT_BATCH_SIZE_BYTES_ATTRIB in config
+ List<List<Mutation>> batchIndexMutationList =
+ MutationState.getMutationBatchList(batchSize, batchSizeBytes, indexMutations);
+ for (List<Mutation> batchIndexMutations : batchIndexMutationList) {
+ commitBatch(region, batchIndexMutations, null, blockingMemStoreSize, null, txState);
+ batchIndexMutations.clear();
}
+ indexMutations.clear();
}
aggregators.aggregate(rowAggregators, result);
hasAny = true;
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 31ab7c9..4775d59 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -126,6 +126,8 @@ public class MutationState implements SQLCloseable {
private final PhoenixConnection connection;
private final long maxSize;
+ private final long maxSizeBytes;
+ private long batchCount = 0L;
private final Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations;
private final List<TransactionAware> txAwares;
private final TransactionContext txContext;
@@ -140,7 +142,7 @@ public class MutationState implements SQLCloseable {
private final MutationMetricQueue mutationMetricQueue;
private ReadMetricQueue readMetricQueue;
-
+
public MutationState(long maxSize, PhoenixConnection connection) {
this(maxSize,connection, null, null);
}
@@ -171,6 +173,7 @@ public class MutationState implements SQLCloseable {
Transaction tx, TransactionContext txContext) {
this.maxSize = maxSize;
this.connection = connection;
+ this.maxSizeBytes = connection.getMutateBatchSizeBytes();
this.mutations = mutations;
boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
@@ -743,12 +746,11 @@ public class MutationState implements SQLCloseable {
public static long getMutationTimestamp(final Long tableTimestamp, Long scn) {
return (tableTimestamp!=null && tableTimestamp!=QueryConstants.UNSET_TIMESTAMP) ? tableTimestamp : (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
}
-
+
/**
* Validates that the meta data is valid against the server meta data if we haven't yet done so.
* Otherwise, for every UPSERT VALUES call, we'd need to hit the server to see if the meta data
* has changed.
- * @param connection
* @return the server time to use for the upsert
* @throws SQLException if the table or any columns no longer exist
*/
@@ -842,6 +844,15 @@ public class MutationState implements SQLCloseable {
}
}
}
+
+ public long getMaxSizeBytes() {
+ return maxSizeBytes;
+ }
+
+ public long getBatchCount() {
+ return batchCount;
+ }
+
private class MetaDataAwareHTable extends DelegateHTable {
private final TableRef tableRef;
@@ -1068,7 +1079,11 @@ public class MutationState implements SQLCloseable {
long startTime = System.currentTimeMillis();
child.addTimelineAnnotation("Attempt " + retryCount);
- hTable.batch(mutationList);
+ List<List<Mutation>> mutationBatchList = getMutationBatchList(maxSize, maxSizeBytes, mutationList);
+ for (List<Mutation> mutationBatch : mutationBatchList) {
+ hTable.batch(mutationBatch);
+ batchCount++;
+ }
if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
child.stop();
child.stop();
@@ -1132,6 +1147,34 @@ public class MutationState implements SQLCloseable {
}
}
+ /**
+ * Split the list of mutations into multiple lists that don't exceed row and byte thresholds
+ * @param allMutationList List of HBase mutations
+ * @return List of lists of mutations
+ */
+ public static List<List<Mutation>> getMutationBatchList(long maxSize, long maxSizeBytes, List<Mutation> allMutationList) {
+ List<List<Mutation>> mutationBatchList = Lists.newArrayList();
+ List<Mutation> currentList = Lists.newArrayList();
+ long currentBatchSizeBytes = 0L;
+ for (Mutation mutation : allMutationList) {
+ long mutationSizeBytes = mutation.heapSize();
+ if (currentList.size() == maxSize || currentBatchSizeBytes + mutationSizeBytes > maxSizeBytes) {
+ if (currentList.size() > 0) {
+ mutationBatchList.add(currentList);
+ currentList = Lists.newArrayList();
+ currentBatchSizeBytes = 0L;
+ }
+ }
+ currentList.add(mutation);
+ currentBatchSizeBytes += mutationSizeBytes;
+ }
+ if (currentList.size() > 0) {
+ mutationBatchList.add(currentList);
+ }
+ return mutationBatchList;
+
+ }
+
public byte[] encodeTransaction() throws SQLException {
try {
return CODEC.encode(getTransaction());
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index c060164..cb2390e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -142,6 +142,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
private final Properties info;
private final Map<PDataType<?>, Format> formatters = new HashMap<>();
private final int mutateBatchSize;
+ private final long mutateBatchSizeBytes;
private final Long scn;
private MutationState mutationState;
private List<SQLCloseable> statements = new ArrayList<SQLCloseable>();
@@ -255,6 +256,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
this.services.getProps().get(QueryServices.SCHEMA_ATTRIB, QueryServicesOptions.DEFAULT_SCHEMA));
this.tenantId = tenantId;
this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info, this.services.getProps());
+ this.mutateBatchSizeBytes = JDBCUtil.getMutateBatchSizeBytes(url, this.info, this.services.getProps());
datePattern = this.services.getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
timePattern = this.services.getProps().get(QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
timestampPattern = this.services.getProps().get(QueryServices.TIMESTAMP_FORMAT_ATTRIB, DateUtil.DEFAULT_TIMESTAMP_FORMAT);
@@ -443,7 +445,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
public int getMutateBatchSize() {
return mutateBatchSize;
}
-
+
+ public long getMutateBatchSizeBytes(){
+ return mutateBatchSizeBytes;
+ }
+
public PMetaData getMetaDataCache() {
return metaData;
}
@@ -457,7 +463,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
}
protected MutationState newMutationState(int maxSize) {
- return new MutationState(maxSize, this);
+ return new MutationState(maxSize, this);
}
public MutationState getMutationState() {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index 15e55dd..c1db27c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -65,6 +65,7 @@ public class PhoenixIndexImportDirectMapper extends
private DirectHTableWriter writer;
private int batchSize;
+ private long batchSizeBytes;
private MutationState mutationState;
@@ -87,12 +88,16 @@ public class PhoenixIndexImportDirectMapper extends
}
connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
connection.setAutoCommit(false);
- // Get BatchSize
+ // Get BatchSize, which is in terms of rows
ConnectionQueryServices services = ((PhoenixConnection) connection).getQueryServices();
int maxSize =
services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,
QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
batchSize = Math.min(((PhoenixConnection) connection).getMutateBatchSize(), maxSize);
+
+ //Get batch size in terms of bytes
+ batchSizeBytes = ((PhoenixConnection) connection).getMutateBatchSizeBytes();
+
LOG.info("Mutation Batch Size = " + batchSize);
final String upsertQuery = PhoenixConfigurationUtil.getUpsertStatement(configuration);
@@ -107,8 +112,6 @@ public class PhoenixIndexImportDirectMapper extends
protected void map(NullWritable key, PhoenixIndexDBWritable record, Context context)
throws IOException, InterruptedException {
- context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
-
try {
final List<Object> values = record.getValues();
indxWritable.setValues(values);
@@ -119,7 +122,6 @@ public class PhoenixIndexImportDirectMapper extends
MutationState currentMutationState = pconn.getMutationState();
if (mutationState == null) {
mutationState = currentMutationState;
- return;
}
// Keep accumulating Mutations till batch size
mutationState.join(currentMutationState);
@@ -137,6 +139,7 @@ public class PhoenixIndexImportDirectMapper extends
context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
throw new RuntimeException(e);
}
+ context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
}
private void writeBatch(MutationState mutationState, Context context) throws IOException,
@@ -144,8 +147,12 @@ public class PhoenixIndexImportDirectMapper extends
final Iterator<Pair<byte[], List<Mutation>>> iterator = mutationState.toMutations(true, null);
while (iterator.hasNext()) {
Pair<byte[], List<Mutation>> mutationPair = iterator.next();
-
- writer.write(mutationPair.getSecond());
+ List<Mutation> batchMutations = mutationPair.getSecond();
+ List<List<Mutation>> batchOfBatchMutations =
+ MutationState.getMutationBatchList(batchSize, batchSizeBytes, batchMutations);
+ for (List<Mutation> mutationList : batchOfBatchMutations) {
+ writer.write(mutationList);
+ }
context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(
mutationPair.getSecond().size());
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 233007f..2035de8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -86,7 +86,10 @@ public interface QueryServices extends SQLCloseable {
public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin";
public static final String SCAN_CACHE_SIZE_ATTRIB = "hbase.client.scanner.caching";
public static final String MAX_MUTATION_SIZE_ATTRIB = "phoenix.mutate.maxSize";
+
+ @Deprecated //USE MUTATE_BATCH_SIZE_BYTES_ATTRIB instead
public static final String MUTATE_BATCH_SIZE_ATTRIB = "phoenix.mutate.batchSize";
+ public static final String MUTATE_BATCH_SIZE_BYTES_ATTRIB = "phoenix.mutate.batchSizeBytes";
public static final String MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxServerCacheTimeToLiveMs";
@Deprecated // Use FORCE_ROW_KEY_ORDER instead.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index df203b5..de0796f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -128,7 +128,10 @@ public class QueryServicesOptions {
public static final boolean DEFAULT_DROP_METADATA = true; // Drop meta data also.
public static final long DEFAULT_DRIVER_SHUTDOWN_TIMEOUT_MS = 5 * 1000; // Time to wait in ShutdownHook to exit gracefully.
- public final static int DEFAULT_MUTATE_BATCH_SIZE = 1000; // Batch size for UPSERT SELECT and DELETE
+ @Deprecated //use DEFAULT_MUTATE_BATCH_SIZE_BYTES
+ public final static int DEFAULT_MUTATE_BATCH_SIZE = 100; // Batch size for UPSERT SELECT and DELETE
+ //Batch size in bytes for UPSERT, SELECT and DELETE. By default, 10MB
+ public final static long DEFAULT_MUTATE_BATCH_SIZE_BYTES = 134217728;
// The only downside of it being out-of-sync is that the parallelization of the scan won't be as balanced as it could be.
public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; // 30 sec (with no activity)
public static final int DEFAULT_SCAN_CACHE_SIZE = 1000;
@@ -449,6 +452,7 @@ public class QueryServicesOptions {
return set(MAX_MUTATION_SIZE_ATTRIB, maxMutateSize);
}
+ @Deprecated
public QueryServicesOptions setMutateBatchSize(int mutateBatchSize) {
return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize);
}
@@ -513,6 +517,7 @@ public class QueryServicesOptions {
return config.getInt(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE);
}
+ @Deprecated
public int getMutateBatchSize() {
return config.getInt(MUTATE_BATCH_SIZE_ATTRIB, DEFAULT_MUTATE_BATCH_SIZE);
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
index f835a21..2cab6fb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
@@ -133,11 +133,17 @@ public class JDBCUtil {
return (scnStr == null ? null : Long.parseLong(scnStr));
}
+ @Deprecated // use getMutateBatchSizeBytes
public static int getMutateBatchSize(String url, Properties info, ReadOnlyProps props) throws SQLException {
String batchSizeStr = findProperty(url, info, PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB);
return (batchSizeStr == null ? props.getInt(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE) : Integer.parseInt(batchSizeStr));
}
+ public static long getMutateBatchSizeBytes(String url, Properties info, ReadOnlyProps props) throws SQLException {
+ String batchSizeStr = findProperty(url, info, PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB);
+ return (batchSizeStr == null ? props.getLong(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE_BYTES) : Long.parseLong(batchSizeStr));
+ }
+
public static @Nullable PName getTenantId(String url, Properties info) throws SQLException {
String tenantId = findProperty(url, info, PhoenixRuntime.TENANT_ID_ATTRIB);
return (tenantId == null ? null : PNameFactory.newName(tenantId));
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 3c16d00..5bfb55d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -147,6 +147,16 @@ public class PhoenixRuntime {
public final static String UPSERT_BATCH_SIZE_ATTRIB = "UpsertBatchSize";
/**
+ * Use this connection property to control the number of bytes that are
+ * batched together on an UPSERT INTO table1... SELECT ... FROM table2.
+ * It's only used when autoCommit is true and your source table is
+ * different than your target table or your SELECT statement has a
+ * GROUP BY clause. Overrides the value of UpsertBatchSize.
+ */
+ public final static String UPSERT_BATCH_SIZE_BYTES_ATTRIB = "UpsertBatchSizeBytes";
+
+
+ /**
* Use this connection property to explicitly enable or disable auto-commit on a new connection.
*/
public static final String AUTO_COMMIT_ATTRIB = "AutoCommit";
@@ -874,7 +884,7 @@ public class PhoenixRuntime {
* Column names and family names are enclosed in double quotes to allow for case sensitivity and for presence of
* special characters. Salting column and view index id column are not included. If the connection is tenant specific
* and the table used by the query plan is multi-tenant, then the tenant id column is not included as well.
- * @param datatypes - Initialized empty list to be filled with the corresponding data type for the columns in @param columns.
+ * @param dataTypes - Initialized empty list to be filled with the corresponding data type for the columns in @param columns.
* @param plan - query plan to get info for
* @param conn - phoenix connection used to generate the query plan. Caller should take care of closing the connection appropriately.
* @param forDataTable - if true, then column names and data types correspond to the data table even if the query plan uses
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
index 4ab75a9..c87c2db 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
@@ -77,6 +77,15 @@ public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
fail("Upsert should have failed since the number of upserts (200) is greater than the MAX_MUTATION_SIZE_ATTRIB (100)");
} catch (IllegalArgumentException expected) {}
}
+
+ @Test
+ public void testMaxMutationSizeInBytesSetCorrectly() throws Exception {
+ Properties connectionProperties = new Properties();
+ connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB,"100");
+ PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+ assertEquals(100L, connection.getMutateBatchSizeBytes());
+ assertEquals(100L, connection.getMutationState().getMaxSizeBytes());
+ }
@Test
public void testDisallowNegativeScn() {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
index 8c9a8a0..fae7633 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
@@ -27,7 +27,9 @@ import static org.junit.Assert.assertTrue;
import java.util.Map;
import java.util.Properties;
+import com.google.common.collect.Maps;
import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.phoenix.query.QueryServices;
import org.junit.Test;
public class JDBCUtilTest {
@@ -107,7 +109,7 @@ public class JDBCUtilTest {
@Test
public void testGetConsistency_TIMELINE_InUrl() {
assertTrue(JDBCUtil.getConsistencyLevel("localhost;Consistency=TIMELINE", new Properties(),
- Consistency.STRONG.toString()) == Consistency.TIMELINE);
+ Consistency.STRONG.toString()) == Consistency.TIMELINE);
}
@Test
@@ -122,6 +124,21 @@ public class JDBCUtilTest {
Properties props = new Properties();
props.setProperty(PhoenixRuntime.CONSISTENCY_ATTRIB, "TIMELINE");
assertTrue(JDBCUtil.getConsistencyLevel("localhost", props, Consistency.STRONG.toString())
- == Consistency.TIMELINE);
+ == Consistency.TIMELINE);
+ }
+
+ @Test
+ public void testGetMaxMutateBytes() throws Exception {
+ assertEquals(1000L, JDBCUtil.getMutateBatchSizeBytes("localhost;" + PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB +
+ "=1000", new Properties(), ReadOnlyProps.EMPTY_PROPS));
+
+ Properties props = new Properties();
+ props.setProperty(PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB, "2000");
+ assertEquals(2000L, JDBCUtil.getMutateBatchSizeBytes("localhost", props, ReadOnlyProps.EMPTY_PROPS));
+
+ Map<String, String> propMap = Maps.newHashMap();
+ propMap.put(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, "3000");
+ ReadOnlyProps readOnlyProps = new ReadOnlyProps(propMap);
+ assertEquals(3000L, JDBCUtil.getMutateBatchSizeBytes("localhost", new Properties(), readOnlyProps));
}
}
[26/26] phoenix git commit: Merge remote-tracking branch
'origin/master' into calcite
Posted by ma...@apache.org.
Merge remote-tracking branch 'origin/master' into calcite
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/40549904
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/40549904
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/40549904
Branch: refs/heads/calcite
Commit: 4054990479990f4f6551eb97fff8ffd91d15d526
Parents: 948d009 069c371
Author: maryannxue <ma...@gmail.com>
Authored: Thu Jan 26 17:14:57 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Thu Jan 26 17:14:57 2017 -0800
----------------------------------------------------------------------
bin/phoenix_utils.py | 5 +
bin/sqlline-thin.py | 17 +-
bin/sqlline.py | 52 +--
.../AlterMultiTenantTableWithViewsIT.java | 2 +-
.../apache/phoenix/end2end/AlterTableIT.java | 16 +-
.../phoenix/end2end/AlterTableWithViewsIT.java | 2 +-
.../phoenix/end2end/ArithmeticQueryIT.java | 2 +-
.../phoenix/end2end/ArrayFillFunctionIT.java | 28 +-
.../phoenix/end2end/AutomaticRebuildIT.java | 219 +++++++++
.../end2end/BaseTenantSpecificTablesIT.java | 4 +-
.../phoenix/end2end/CoalesceFunctionIT.java | 60 +--
.../end2end/ConvertTimezoneFunctionIT.java | 22 +-
.../org/apache/phoenix/end2end/DateTimeIT.java | 6 +-
.../org/apache/phoenix/end2end/DeleteIT.java | 26 +-
.../phoenix/end2end/End2EndTestDriver.java | 12 +-
.../phoenix/end2end/EvaluationOfORIT.java | 5 +-
.../apache/phoenix/end2end/GroupByCaseIT.java | 10 +-
.../org/apache/phoenix/end2end/GroupByIT.java | 6 +-
.../apache/phoenix/end2end/HashJoinMoreIT.java | 47 +-
.../end2end/IndexToolForPartialBuildIT.java | 298 ++++++++++++
...olForPartialBuildWithNamespaceEnabledIT.java | 70 +++
.../org/apache/phoenix/end2end/QueryMoreIT.java | 35 ++
.../phoenix/end2end/SortMergeJoinMoreIT.java | 37 +-
.../org/apache/phoenix/end2end/SortOrderIT.java | 8 +-
.../end2end/TenantSpecificTablesDDLIT.java | 12 +-
.../end2end/TenantSpecificTablesDMLIT.java | 66 +--
.../end2end/TenantSpecificViewIndexIT.java | 47 ++
.../end2end/UpsertSelectAutoCommitIT.java | 2 +-
.../apache/phoenix/end2end/UpsertSelectIT.java | 2 +-
.../apache/phoenix/end2end/UpsertValuesIT.java | 20 +-
.../phoenix/end2end/index/ImmutableIndexIT.java | 1 -
.../phoenix/end2end/index/IndexMetadataIT.java | 58 +++
.../end2end/index/MutableIndexFailureIT.java | 10 +-
.../phoenix/monitoring/PhoenixMetricsIT.java | 135 ++++++
.../org/apache/phoenix/tx/TransactionIT.java | 9 +-
.../org/apache/phoenix/tx/TxCheckpointIT.java | 2 +-
phoenix-core/src/main/antlr3/PhoenixSQL.g | 4 +-
.../apache/phoenix/compile/UpsertCompiler.java | 44 +-
.../apache/phoenix/compile/WhereCompiler.java | 3 +-
.../apache/phoenix/compile/WhereOptimizer.java | 7 +-
.../coprocessor/MetaDataEndpointImpl.java | 9 +-
.../coprocessor/MetaDataRegionObserver.java | 291 +++++++-----
.../UngroupedAggregateRegionObserver.java | 31 +-
.../phoenix/exception/SQLExceptionCode.java | 3 +-
.../apache/phoenix/execute/MutationState.java | 51 ++-
.../function/ConvertTimezoneFunction.java | 5 +-
.../expression/function/EncodeFormat.java | 4 +-
.../index/PhoenixIndexFailurePolicy.java | 52 +--
.../phoenix/iterate/OrderedResultIterator.java | 5 +-
.../apache/phoenix/jdbc/PhoenixConnection.java | 10 +-
.../phoenix/jdbc/PhoenixDatabaseMetaData.java | 2 +
.../org/apache/phoenix/jdbc/PhoenixDriver.java | 99 ++--
.../apache/phoenix/jdbc/PhoenixStatement.java | 8 +-
.../phoenix/mapreduce/CsvBulkImportUtil.java | 7 +-
.../phoenix/mapreduce/CsvBulkLoadTool.java | 12 +-
.../phoenix/mapreduce/index/IndexTool.java | 455 +++++++++++++------
.../phoenix/mapreduce/index/IndexToolUtil.java | 6 +-
.../index/PhoenixIndexImportDirectMapper.java | 21 +-
.../index/PhoenixIndexPartialBuildMapper.java | 182 ++++++++
.../util/PhoenixConfigurationUtil.java | 31 ++
.../phoenix/monitoring/GlobalClientMetrics.java | 6 +-
.../apache/phoenix/monitoring/MetricType.java | 4 +-
.../phoenix/parse/AlterIndexStatement.java | 8 +-
.../apache/phoenix/parse/ParseNodeFactory.java | 6 +-
.../query/ConnectionQueryServicesImpl.java | 47 +-
.../java/org/apache/phoenix/query/KeyRange.java | 5 +-
.../org/apache/phoenix/query/QueryServices.java | 13 +
.../phoenix/query/QueryServicesOptions.java | 18 +-
.../apache/phoenix/schema/MetaDataClient.java | 47 +-
.../stats/DefaultStatisticsCollector.java | 32 +-
.../apache/phoenix/schema/types/PBinary.java | 6 +-
.../apache/phoenix/schema/types/PVarbinary.java | 5 +-
.../org/apache/phoenix/util/ExpressionUtil.java | 10 +
.../java/org/apache/phoenix/util/IndexUtil.java | 61 ++-
.../java/org/apache/phoenix/util/JDBCUtil.java | 6 +
.../org/apache/phoenix/util/PhoenixRuntime.java | 23 +-
.../phoenix/util/csv/CsvUpsertExecutor.java | 25 +
.../phoenix/util/json/JsonUpsertExecutor.java | 44 ++
.../phoenix/compile/JoinQueryCompilerTest.java | 6 +-
.../phoenix/compile/QueryCompilerTest.java | 8 +-
.../iterate/OrderedResultIteratorTest.java | 41 ++
.../apache/phoenix/jdbc/PhoenixDriverTest.java | 9 +
.../mapreduce/CsvBulkImportUtilTest.java | 8 +-
.../phoenix/query/KeyRangeIntersectTest.java | 9 +-
.../util/AbstractUpsertExecutorTest.java | 82 +++-
.../org/apache/phoenix/util/JDBCUtilTest.java | 21 +-
.../phoenix/util/csv/CsvUpsertExecutorTest.java | 26 +-
.../util/json/JsonUpsertExecutorTest.java | 6 +
phoenix-spark/src/it/resources/globalSetup.sql | 7 +-
.../apache/phoenix/spark/PhoenixSparkIT.scala | 42 +-
.../phoenix/spark/DataFrameFunctions.scala | 6 +-
.../org/apache/phoenix/spark/PhoenixRDD.scala | 22 +-
.../phoenix/spark/ProductRDDFunctions.scala | 3 +-
.../tracingwebapp/http/EntityFactory.java | 19 +-
.../tracingwebapp/http/TraceServlet.java | 21 +-
pom.xml | 16 +-
96 files changed, 2657 insertions(+), 756 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 1a18ceb,d47a6b8..4005ee4
--- a/pom.xml
+++ b/pom.xml
@@@ -97,12 -97,11 +97,12 @@@
<jodatime.version>1.6</jodatime.version>
<joni.version>2.1.2</joni.version>
<avatica.version>1.9.0</avatica.version>
+ <calcite.version>1.12.0-SNAPSHOT</calcite.version>
<jettyVersion>8.1.7.v20120910</jettyVersion>
<tephra.version>0.9.0-incubating</tephra.version>
- <spark.version>1.6.1</spark.version>
- <scala.version>2.10.4</scala.version>
- <scala.binary.version>2.10</scala.binary.version>
+ <spark.version>2.0.2</spark.version>
+ <scala.version>2.11.8</scala.version>
+ <scala.binary.version>2.11</scala.binary.version>
<!-- Test Dependencies -->
<mockito-all.version>1.8.5</mockito-all.version>
[06/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow
incremental index rebuilds
Posted by ma...@apache.org.
PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/83827cd8
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/83827cd8
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/83827cd8
Branch: refs/heads/calcite
Commit: 83827cd8c2876c6b6dccf3a5678889b40a76261b
Parents: 70dc383
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 11:59:19 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 11:59:19 2016 +0530
----------------------------------------------------------------------
.../phoenix/end2end/AutomaticRebuildIT.java | 219 +++++++++
.../end2end/IndexToolForPartialBuildIT.java | 298 ++++++++++++
...olForPartialBuildWithNamespaceEnabledIT.java | 70 +++
.../phoenix/end2end/index/IndexMetadataIT.java | 58 +++
.../end2end/index/MutableIndexFailureIT.java | 10 +-
phoenix-core/src/main/antlr3/PhoenixSQL.g | 4 +-
.../coprocessor/MetaDataEndpointImpl.java | 9 +-
.../coprocessor/MetaDataRegionObserver.java | 291 +++++++-----
.../phoenix/exception/SQLExceptionCode.java | 3 +-
.../index/PhoenixIndexFailurePolicy.java | 52 +--
.../phoenix/jdbc/PhoenixDatabaseMetaData.java | 2 +
.../apache/phoenix/jdbc/PhoenixStatement.java | 8 +-
.../phoenix/mapreduce/index/IndexTool.java | 455 +++++++++++++------
.../phoenix/mapreduce/index/IndexToolUtil.java | 6 +-
.../index/PhoenixIndexImportDirectMapper.java | 2 +-
.../index/PhoenixIndexPartialBuildMapper.java | 182 ++++++++
.../util/PhoenixConfigurationUtil.java | 31 ++
.../phoenix/parse/AlterIndexStatement.java | 8 +-
.../apache/phoenix/parse/ParseNodeFactory.java | 6 +-
.../org/apache/phoenix/query/QueryServices.java | 4 +
.../apache/phoenix/schema/MetaDataClient.java | 47 +-
.../java/org/apache/phoenix/util/IndexUtil.java | 61 ++-
22 files changed, 1504 insertions(+), 322 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
new file mode 100644
index 0000000..cbb7745
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
@@ -0,0 +1,219 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link AutomaticRebuildIT}
+ */
+@RunWith(Parameterized.class)
+public class AutomaticRebuildIT extends BaseOwnClusterIT {
+
+ private final boolean localIndex;
+ protected boolean isNamespaceEnabled = false;
+ protected final String tableDDLOptions;
+
+ public AutomaticRebuildIT(boolean localIndex) {
+ this.localIndex = localIndex;
+ StringBuilder optionBuilder = new StringBuilder();
+ optionBuilder.append(" SPLIT ON(1,2)");
+ this.tableDDLOptions = optionBuilder.toString();
+ }
+
+ @BeforeClass
+ public static void doSetup() throws Exception {
+ Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+ serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+ serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+ serverProps.put(" yarn.scheduler.capacity.maximum-am-resource-percent", "1.0");
+ serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+ serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+ serverProps.put("hbase.client.pause", "5000");
+ serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, "1000");
+ serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE, "5");
+ Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+ setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+ new ReadOnlyProps(clientProps.entrySet().iterator()));
+ }
+
+ @Parameters(name = "localIndex = {0}")
+ public static Collection<Boolean[]> data() {
+ return Arrays.asList(new Boolean[][] { { false }, { true } });
+ }
+
+ @Test
+ public void testSecondaryAutomaticRebuildIndex() throws Exception {
+ String schemaName = generateUniqueName();
+ String dataTableName = generateUniqueName();
+ String fullTableName = SchemaUtil.getTableName(schemaName, dataTableName);
+ final String indxTable = String.format("%s_%s", dataTableName, FailingRegionObserver.INDEX_NAME);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
+ props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString());
+ props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ Statement stmt = conn.createStatement();
+ try {
+ if (isNamespaceEnabled) {
+ conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+ }
+ stmt.execute(String.format(
+ "CREATE TABLE %s (ID BIGINT NOT NULL, NAME VARCHAR, ZIP INTEGER CONSTRAINT PK PRIMARY KEY(ID ROW_TIMESTAMP)) %s",
+ fullTableName, tableDDLOptions));
+ String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", fullTableName);
+ PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+ FailingRegionObserver.FAIL_WRITE = false;
+ // insert two rows
+ upsertRow(stmt1, 1000);
+ upsertRow(stmt1, 2000);
+
+ conn.commit();
+ stmt.execute(String.format("CREATE %s INDEX %s ON %s (LPAD(UPPER(NAME),11,'x')||'_xyz') ",
+ (localIndex ? "LOCAL" : ""), indxTable, fullTableName));
+ FailingRegionObserver.FAIL_WRITE = true;
+ upsertRow(stmt1, 3000);
+ upsertRow(stmt1, 4000);
+ upsertRow(stmt1, 5000);
+ try {
+ conn.commit();
+ fail();
+ } catch (SQLException e) {
+ } catch (Exception e) {
+ }
+ FailingRegionObserver.FAIL_WRITE = false;
+ ResultSet rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+ new String[] { PTableType.INDEX.toString() });
+ assertTrue(rs.next());
+ assertEquals(indxTable, rs.getString(3));
+ String indexState = rs.getString("INDEX_STATE");
+ assertEquals(PIndexState.DISABLE.toString(), indexState);
+ assertFalse(rs.next());
+ upsertRow(stmt1, 6000);
+ upsertRow(stmt1, 7000);
+ conn.commit();
+ int maxTries = 4, nTries = 0;
+ boolean isInactive = false;
+ do {
+ rs = conn.createStatement()
+ .executeQuery(String.format("SELECT " + PhoenixDatabaseMetaData.INDEX_STATE + ","
+ + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " FROM "
+ + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " ("
+ + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " bigint) where "
+ + PhoenixDatabaseMetaData.TABLE_SCHEM + "='" + schemaName + "' and "
+ + PhoenixDatabaseMetaData.TABLE_NAME + "='" + indxTable + "'"));
+ rs.next();
+ if (PIndexState.INACTIVE.getSerializedValue().equals(rs.getString(1)) && rs.getLong(2) > 3000) {
+ isInactive = true;
+ break;
+ }
+ Thread.sleep(10 * 1000); // sleep 10 secs
+ } while (++nTries < maxTries);
+ assertTrue(isInactive);
+ nTries = 0;
+ boolean isActive = false;
+ do {
+ Thread.sleep(15 * 1000); // sleep 15 secs
+ rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+ new String[] { PTableType.INDEX.toString() });
+ assertTrue(rs.next());
+ if (PIndexState.ACTIVE.toString().equals(rs.getString("INDEX_STATE"))) {
+ isActive = true;
+ break;
+ }
+ } while (++nTries < maxTries);
+ assertTrue(isActive);
+
+ } finally {
+ conn.close();
+ }
+ }
+
+ public static void upsertRow(PreparedStatement stmt, int i) throws SQLException {
+ // insert row
+ stmt.setInt(1, i);
+ stmt.setString(2, "uname" + String.valueOf(i));
+ stmt.setInt(3, 95050 + i);
+ stmt.executeUpdate();
+ }
+
+ public static class FailingRegionObserver extends SimpleRegionObserver {
+ public static volatile boolean FAIL_WRITE = false;
+ public static final String INDEX_NAME = "IDX";
+
+ @Override
+ public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+ MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
+ if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
+ throw new DoNotRetryIOException();
+ }
+ Mutation operation = miniBatchOp.getOperation(0);
+ Set<byte[]> keySet = operation.getFamilyMap().keySet();
+ for (byte[] family : keySet) {
+ if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+ throw new DoNotRetryIOException();
+ }
+ }
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
new file mode 100644
index 0000000..116c47f
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
@@ -0,0 +1,298 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link IndexToolForPartialBuildIT}
+ */
+@RunWith(Parameterized.class)
+public class IndexToolForPartialBuildIT extends BaseOwnClusterIT {
+
+ private final boolean localIndex;
+ protected boolean isNamespaceEnabled = false;
+ protected final String tableDDLOptions;
+
+ public IndexToolForPartialBuildIT(boolean localIndex) {
+
+ this.localIndex = localIndex;
+ StringBuilder optionBuilder = new StringBuilder();
+ optionBuilder.append(" SPLIT ON(1,2)");
+ this.tableDDLOptions = optionBuilder.toString();
+ }
+
+ @BeforeClass
+ public static void doSetup() throws Exception {
+ Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+ serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+ serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+ serverProps.put(" yarn.scheduler.capacity.maximum-am-resource-percent", "1.0");
+ serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+ serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+ serverProps.put("hbase.client.pause", "5000");
+ serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_ATTRIB, Boolean.FALSE.toString());
+ Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+ setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+ }
+
+ @Parameters(name="localIndex = {0}")
+ public static Collection<Boolean[]> data() {
+ return Arrays.asList(new Boolean[][] {
+ { false},{ true }
+ });
+ }
+
+ @Test
+ public void testSecondaryIndex() throws Exception {
+ String schemaName = generateUniqueName();
+ String dataTableName = generateUniqueName();
+ String fullTableName = SchemaUtil.getTableName(schemaName, dataTableName);
+ final String indxTable = String.format("%s_%s", dataTableName, FailingRegionObserver.INDEX_NAME);
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
+ props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString());
+ props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+ final Connection conn = DriverManager.getConnection(getUrl(), props);
+ Statement stmt = conn.createStatement();
+ try {
+ if (isNamespaceEnabled) {
+ conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+ }
+ stmt.execute(
+ String.format("CREATE TABLE %s (ID BIGINT NOT NULL, NAME VARCHAR, ZIP INTEGER CONSTRAINT PK PRIMARY KEY(ID ROW_TIMESTAMP)) %s",
+ fullTableName, tableDDLOptions));
+ String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", fullTableName);
+ PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+ FailingRegionObserver.FAIL_WRITE = false;
+ // insert two rows
+ upsertRow(stmt1, 1000);
+ upsertRow(stmt1, 2000);
+
+ conn.commit();
+ stmt.execute(String.format("CREATE %s INDEX %s ON %s (LPAD(UPPER(NAME),11,'x')||'_xyz') ",
+ (localIndex ? "LOCAL" : ""), indxTable, fullTableName));
+ FailingRegionObserver.FAIL_WRITE = true;
+ upsertRow(stmt1, 3000);
+ upsertRow(stmt1, 4000);
+ upsertRow(stmt1, 5000);
+ try {
+ conn.commit();
+ fail();
+ } catch (SQLException e) {} catch (Exception e) {}
+ conn.createStatement()
+ .execute(String.format("ALTER INDEX %s on %s REBUILD ASYNC", indxTable, fullTableName));
+
+ FailingRegionObserver.FAIL_WRITE = false;
+ ResultSet rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+ new String[] { PTableType.INDEX.toString() });
+ assertTrue(rs.next());
+ assertEquals(indxTable, rs.getString(3));
+ String indexState = rs.getString("INDEX_STATE");
+ assertEquals(PIndexState.BUILDING.toString(), indexState);
+ assertFalse(rs.next());
+ upsertRow(stmt1, 6000);
+ upsertRow(stmt1, 7000);
+ conn.commit();
+
+ rs = conn.createStatement()
+ .executeQuery(String.format("SELECT " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + ","
+ + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " FROM "
+ + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " ("
+ + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " bigint) where "
+ + PhoenixDatabaseMetaData.TABLE_SCHEM + "='" + schemaName + "' and "
+ + PhoenixDatabaseMetaData.TABLE_NAME + "='" + indxTable + "'"));
+ rs.next();
+ PTable pindexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(schemaName, indxTable));
+ assertEquals(PIndexState.BUILDING, pindexTable.getIndexState());
+ assertEquals(rs.getLong(1), pindexTable.getTimeStamp());
+ //assert disabled timestamp
+ assertEquals(rs.getLong(2), 3000);
+
+ String selectSql = String.format("SELECT LPAD(UPPER(NAME),11,'x')||'_xyz',ID FROM %s", fullTableName);
+ rs = conn.createStatement().executeQuery("EXPLAIN " + selectSql);
+ String actualExplainPlan = QueryUtil.getExplainPlan(rs);
+
+ // assert we are pulling from data table.
+ assertExplainPlan(actualExplainPlan, schemaName, dataTableName, null, false, isNamespaceEnabled);
+
+ rs = stmt1.executeQuery(selectSql);
+ for (int i = 1; i <= 7; i++) {
+ assertTrue(rs.next());
+ assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+ }
+
+ // Validate Index table data till disabled timestamp
+ rs = stmt1.executeQuery(String.format("SELECT * FROM %s", SchemaUtil.getTableName(schemaName, indxTable)));
+ for (int i = 1; i <= 2; i++) {
+ assertTrue(rs.next());
+ assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+ }
+ assertFalse(rs.next());
+ // run the index MR job.
+ final IndexTool indexingTool = new IndexTool();
+ Configuration conf = new Configuration(getUtility().getConfiguration());
+ conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+ indexingTool.setConf(conf);
+
+ final String[] cmdArgs = getArgValues(schemaName, dataTableName);
+ int status = indexingTool.run(cmdArgs);
+ assertEquals(0, status);
+
+ // insert two more rows
+ upsertRow(stmt1, 8000);
+ upsertRow(stmt1, 9000);
+ conn.commit();
+
+ // assert we are pulling from index table.
+ rs = conn.createStatement().executeQuery("EXPLAIN " + selectSql);
+ actualExplainPlan = QueryUtil.getExplainPlan(rs);
+ assertExplainPlan(actualExplainPlan, schemaName, dataTableName, indxTable, localIndex, isNamespaceEnabled);
+
+ rs = stmt.executeQuery(selectSql);
+
+ for (int i = 1; i <= 9; i++) {
+ assertTrue(rs.next());
+ assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+ }
+
+ assertFalse(rs.next());
+
+ // conn.createStatement().execute(String.format("DROP INDEX %s ON %s", indxTable, fullTableName));
+ } finally {
+ conn.close();
+ }
+ }
+
+ public static void assertExplainPlan(final String actualExplainPlan, String schemaName, String dataTable,
+ String indxTable, boolean isLocal, boolean isNamespaceMapped) {
+
+ String expectedExplainPlan = "";
+ if (indxTable != null) {
+ if (isLocal) {
+ final String localIndexName = SchemaUtil
+ .getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, dataTable), isNamespaceMapped,
+ PTableType.INDEX)
+ .getString();
+ expectedExplainPlan = String.format("CLIENT PARALLEL 3-WAY RANGE SCAN OVER %s [1]", localIndexName);
+ } else {
+ expectedExplainPlan = String.format("CLIENT PARALLEL 1-WAY FULL SCAN OVER %s",
+ SchemaUtil.getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, indxTable),
+ isNamespaceMapped, PTableType.INDEX));
+ }
+ } else {
+ expectedExplainPlan = String.format("CLIENT PARALLEL 1-WAY FULL SCAN OVER %s",
+ SchemaUtil.getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, dataTable),
+ isNamespaceMapped, PTableType.TABLE));
+ }
+ assertTrue(actualExplainPlan.contains(expectedExplainPlan));
+ }
+
+ public String[] getArgValues(String schemaName, String dataTable) {
+ final List<String> args = Lists.newArrayList();
+ if (schemaName!=null) {
+ args.add("-s");
+ args.add(schemaName);
+ }
+ args.add("-dt");
+ args.add(dataTable);
+ args.add("-pr");
+ args.add("-op");
+ args.add("/tmp/output/partialTable_"+localIndex);
+ return args.toArray(new String[0]);
+ }
+
+ public static void upsertRow(PreparedStatement stmt, int i) throws SQLException {
+ // insert row
+ stmt.setInt(1, i);
+ stmt.setString(2, "uname" + String.valueOf(i));
+ stmt.setInt(3, 95050 + i);
+ stmt.executeUpdate();
+ }
+
+
+ public static class FailingRegionObserver extends SimpleRegionObserver {
+ public static volatile boolean FAIL_WRITE = false;
+ public static final String INDEX_NAME = "IDX";
+ @Override
+ public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
+ if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
+ throw new DoNotRetryIOException();
+ }
+ Mutation operation = miniBatchOp.getOperation(0);
+ Set<byte[]> keySet = operation.getFamilyMap().keySet();
+ for(byte[] family: keySet) {
+ if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+ throw new DoNotRetryIOException();
+ }
+ }
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
new file mode 100644
index 0000000..5e16b05
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
@@ -0,0 +1,70 @@
+/*
+ * 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.phoenix.end2end;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link IndexToolForPartialBuildWithNamespaceEnabled}
+ */
+@RunWith(Parameterized.class)
+public class IndexToolForPartialBuildWithNamespaceEnabled extends IndexToolForPartialBuildIT {
+
+
+ public IndexToolForPartialBuildWithNamespaceEnabled(boolean localIndex, boolean isNamespaceEnabled) {
+ super(localIndex);
+ this.isNamespaceEnabled=isNamespaceEnabled;
+ }
+
+ @BeforeClass
+ public static void doSetup() throws Exception {
+ Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+ serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+ serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+ serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+ serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+ serverProps.put("hbase.client.pause", "5000");
+ serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, "2000");
+ serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB, "1000");
+ serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+ Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+ clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+ setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+ }
+
+ @Parameters(name="localIndex = {0} , isNamespaceEnabled = {1}")
+ public static Collection<Boolean[]> data() {
+ return Arrays.asList(new Boolean[][] {
+ { false, true},{ true, false }
+ });
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index f0c670b..63a6bd6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -31,6 +31,7 @@ import java.sql.DriverManager;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
+import java.sql.Statement;
import java.sql.Types;
import java.util.Properties;
@@ -43,10 +44,13 @@ import org.apache.phoenix.query.QueryConstants;
import org.apache.phoenix.schema.AmbiguousColumnException;
import org.apache.phoenix.schema.ColumnAlreadyExistsException;
import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.PTableKey;
import org.apache.phoenix.schema.PTableType;
import org.apache.phoenix.schema.TableNotFoundException;
import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.PhoenixRuntime;
import org.apache.phoenix.util.PropertiesUtil;
import org.apache.phoenix.util.SchemaUtil;
import org.apache.phoenix.util.StringUtil;
@@ -216,6 +220,15 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
assertFalse(rs.next());
assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
+
+ ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " REBUILD ASYNC";
+ conn.createStatement().execute(ddl);
+ // Verify the metadata for index is correct.
+ rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
+ assertTrue(rs.next());
+ assertEquals(indexName , rs.getString(3));
+ assertEquals(PIndexState.BUILDING.toString(), rs.getString("INDEX_STATE"));
+ assertFalse(rs.next());
ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
stmt = conn.prepareStatement(ddl);
@@ -568,4 +581,49 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
assertTrue(d2.after(d1));
assertFalse(rs.next());
}
+
+ @Test
+ public void testAsyncRebuildTimestamp() throws Exception {
+ long startTimestamp = System.currentTimeMillis();
+ Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+ Connection conn = DriverManager.getConnection(getUrl(), props);
+ conn.setAutoCommit(false);
+ String testTable = generateUniqueName();
+
+
+ String ddl = "create table " + testTable + " (k varchar primary key, v1 varchar, v2 varchar, v3 varchar)";
+ Statement stmt = conn.createStatement();
+ stmt.execute(ddl);
+ String indexName = "R_ASYNCIND_" + generateUniqueName();
+
+ ddl = "CREATE INDEX " + indexName + "1 ON " + testTable + " (v1) ";
+ stmt.execute(ddl);
+ ddl = "CREATE INDEX " + indexName + "2 ON " + testTable + " (v2) ";
+ stmt.execute(ddl);
+ ddl = "CREATE INDEX " + indexName + "3 ON " + testTable + " (v3)";
+ stmt.execute(ddl);
+ conn.createStatement().execute("ALTER INDEX "+indexName+"1 ON " + testTable +" DISABLE ");
+ conn.createStatement().execute("ALTER INDEX "+indexName+"2 ON " + testTable +" REBUILD ");
+ conn.createStatement().execute("ALTER INDEX "+indexName+"3 ON " + testTable +" REBUILD ASYNC");
+
+ ResultSet rs = conn.createStatement().executeQuery(
+ "select table_name, " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " +
+ "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() + ") " +
+ "where " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " !=0 and table_name like 'R_ASYNCIND_%' " +
+ "order by table_name");
+ assertTrue(rs.next());
+ assertEquals(indexName + "3", rs.getString(1));
+ long asyncTimestamp = rs.getLong(2);
+ assertTrue("Async timestamp is recent timestamp", asyncTimestamp > startTimestamp);
+ PTable table = PhoenixRuntime.getTable(conn, indexName+"3");
+ assertEquals(table.getTimeStamp(), asyncTimestamp);
+ assertFalse(rs.next());
+ conn.createStatement().execute("ALTER INDEX "+indexName+"3 ON " + testTable +" DISABLE");
+ rs = conn.createStatement().executeQuery(
+ "select table_name, " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " +
+ "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() + ") " +
+ "where " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " !=0 and table_name like 'ASYNCIND_%' " +
+ "order by table_name" );
+ assertFalse(rs.next());
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 687b2c2..e9205c9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -102,7 +102,7 @@ public class MutableIndexFailureIT extends BaseTest {
this.tableDDLOptions = " SALT_BUCKETS=2 " + (transactional ? ", TRANSACTIONAL=true " : "");
this.tableName = (localIndex ? "L_" : "") + TestUtil.DEFAULT_DATA_TABLE_NAME + (transactional ? "_TXN" : "")
+ (isNamespaceMapped ? "_NM" : "");
- this.indexName = INDEX_NAME;
+ this.indexName = FailingRegionObserver.INDEX_NAME;
fullTableName = SchemaUtil.getTableName(schema, tableName);
this.fullIndexName = SchemaUtil.getTableName(schema, indexName);
this.isNamespaceMapped = isNamespaceMapped;
@@ -155,7 +155,7 @@ public class MutableIndexFailureIT extends BaseTest {
rs = conn.createStatement().executeQuery(query);
assertFalse(rs.next());
- FAIL_WRITE = false;
+ FailingRegionObserver.FAIL_WRITE = false;
conn.createStatement().execute(
"CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
// Create other index which should be local/global if the other index is global/local to
@@ -202,7 +202,7 @@ public class MutableIndexFailureIT extends BaseTest {
assertEquals("z", rs.getString(2));
assertFalse(rs.next());
- FAIL_WRITE = true;
+ FailingRegionObserver.FAIL_WRITE = true;
updateTable(conn, fullTableName);
updateTable(conn, secondTableName);
// Verify the metadata for index is correct.
@@ -259,7 +259,7 @@ public class MutableIndexFailureIT extends BaseTest {
}
// re-enable index table
- FAIL_WRITE = false;
+ FailingRegionObserver.FAIL_WRITE = false;
waitForIndexToBeActive(conn,indexName);
waitForIndexToBeActive(conn,indexName+"_2");
waitForIndexToBeActive(conn,secondIndexName);
@@ -391,6 +391,8 @@ public class MutableIndexFailureIT extends BaseTest {
}
public static class FailingRegionObserver extends SimpleRegionObserver {
+ public static volatile boolean FAIL_WRITE = false;
+ public static final String INDEX_NAME = "IDX";
@Override
public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 3e09766..07a51ce 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -567,8 +567,8 @@ drop_index_node returns [DropIndexStatement ret]
// Parse a alter index statement
alter_index_node returns [AlterIndexStatement ret]
- : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE)
- {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText()))); }
+ : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE) (async=ASYNC)?
+ {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())), async!=null); }
;
// Parse a trace statement.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 38dc494..a0681fb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -3296,11 +3296,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
List<Cell> newKVs = tableMetadata.get(0).getFamilyCellMap().get(TABLE_FAMILY_BYTES);
Cell newKV = null;
int disableTimeStampKVIndex = -1;
+ int indexStateKVIndex = 0;
int index = 0;
for(Cell cell : newKVs){
if(Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
INDEX_STATE_BYTES, 0, INDEX_STATE_BYTES.length) == 0){
newKV = cell;
+ indexStateKVIndex = index;
} else if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
INDEX_DISABLE_TIMESTAMP_BYTES, 0, INDEX_DISABLE_TIMESTAMP_BYTES.length) == 0){
disableTimeStampKVIndex = index;
@@ -3378,11 +3380,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
if ((currentState == PIndexState.UNUSABLE && newState == PIndexState.ACTIVE)
|| (currentState == PIndexState.ACTIVE && newState == PIndexState.UNUSABLE)) {
newState = PIndexState.INACTIVE;
- newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
+ newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
} else if (currentState == PIndexState.INACTIVE && newState == PIndexState.USABLE) {
newState = PIndexState.ACTIVE;
- newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
+ newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
}
@@ -3414,7 +3416,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
if(dataTableKey != null) {
metaDataCache.invalidate(new ImmutableBytesPtr(dataTableKey));
}
- if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1) {
+ if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1
+ || currentState == PIndexState.DISABLE || newState == PIndexState.BUILDING) {
returnTable = doGetTable(key, HConstants.LATEST_TIMESTAMP, rowLock);
}
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index e790b59..a60de03 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -17,12 +17,19 @@
*/
package org.apache.phoenix.coprocessor;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
import java.io.IOException;
+import java.sql.PreparedStatement;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
@@ -45,6 +52,7 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -69,6 +77,7 @@ import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesOptions;
import org.apache.phoenix.schema.MetaDataClient;
import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PName;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.SortOrder;
import org.apache.phoenix.schema.TableNotFoundException;
@@ -97,6 +106,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
private boolean enableRebuildIndex = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD;
private long rebuildIndexTimeInterval = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL;
private boolean blockWriteRebuildIndex = false;
+ private static Map<PName, Long> batchExecutedPerTableMap = new HashMap<PName, Long>();
@Override
public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
@@ -125,6 +135,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
+
}
@Override
@@ -195,9 +206,15 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
// running
private final static AtomicInteger inProgress = new AtomicInteger(0);
RegionCoprocessorEnvironment env;
+ private long rebuildIndexBatchSize = HConstants.LATEST_TIMESTAMP;
+ private long configuredBatches = 10;
public BuildIndexScheduleTask(RegionCoprocessorEnvironment env) {
this.env = env;
+ this.rebuildIndexBatchSize = env.getConfiguration().getLong(
+ QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, HConstants.LATEST_TIMESTAMP);
+ this.configuredBatches = env.getConfiguration().getLong(
+ QueryServices.INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE, configuredBatches);
}
@Override
@@ -228,6 +245,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
scan.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES);
+ PreparedStatement updateDisabledTimeStampSmt = null;
Map<PTable, List<PTable>> dataTableToIndexesMap = null;
MetaDataClient client = null;
@@ -243,8 +261,13 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
Result r = Result.create(results);
byte[] disabledTimeStamp = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES);
+ byte[] indexState = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+ PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
+
+ if (disabledTimeStamp == null || disabledTimeStamp.length == 0 || (indexState != null
+ && PIndexState.BUILDING == PIndexState.fromSerializedValue(Bytes.toString(indexState)))) {
- if (disabledTimeStamp == null || disabledTimeStamp.length == 0) {
+ // Don't rebuild the building index , because they are marked for aysnc
continue;
}
@@ -255,8 +278,6 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
}
byte[] dataTable = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES);
- byte[] indexState = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
- PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
if ((dataTable == null || dataTable.length == 0) || (indexState == null || indexState.length == 0)) {
// data table name can't be empty
continue;
@@ -317,109 +338,169 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
indexesToPartiallyRebuild.add(indexPTable);
} while (hasMore);
- if (dataTableToIndexesMap != null) {
- long overlapTime = env.getConfiguration().getLong(
- QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB,
- QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME);
- for (Map.Entry<PTable, List<PTable>> entry : dataTableToIndexesMap.entrySet()) {
- PTable dataPTable = entry.getKey();
- List<PTable> indexesToPartiallyRebuild = entry.getValue();
- try {
- long earliestDisableTimestamp = Long.MAX_VALUE;
- List<IndexMaintainer> maintainers = Lists
- .newArrayListWithExpectedSize(indexesToPartiallyRebuild.size());
- for (PTable index : indexesToPartiallyRebuild) {
- long disabledTimeStampVal = index.getIndexDisableTimestamp();
- if (disabledTimeStampVal > 0) {
- if (disabledTimeStampVal < earliestDisableTimestamp) {
- earliestDisableTimestamp = disabledTimeStampVal;
- }
-
- maintainers.add(index.getIndexMaintainer(dataPTable, conn));
- }
- }
- // No indexes are disabled, so skip this table
- if (earliestDisableTimestamp == Long.MAX_VALUE) {
- continue;
- }
-
- long timeStamp = Math.max(0, earliestDisableTimestamp - overlapTime);
- LOG.info("Starting to build " + dataPTable + " indexes " + indexesToPartiallyRebuild
- + " from timestamp=" + timeStamp);
- TableRef tableRef = new TableRef(null, dataPTable, HConstants.LATEST_TIMESTAMP, false);
- // TODO Need to set high timeout
- PostDDLCompiler compiler = new PostDDLCompiler(conn);
- MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), null, null, null,
- HConstants.LATEST_TIMESTAMP);
- Scan dataTableScan = IndexManagementUtil.newLocalStateScan(plan.getContext().getScan(),
- maintainers);
- dataTableScan.setTimeRange(timeStamp, HConstants.LATEST_TIMESTAMP);
- dataTableScan.setCacheBlocks(false);
- dataTableScan.setAttribute(BaseScannerRegionObserver.REBUILD_INDEXES, TRUE_BYTES);
-
- ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(
- ByteUtil.EMPTY_BYTE_ARRAY);
- IndexMaintainer.serializeAdditional(dataPTable, indexMetaDataPtr, indexesToPartiallyRebuild,
- conn);
- byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
-
- dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
- MutationState mutationState = plan.execute();
- long rowCount = mutationState.getUpdateCount();
- LOG.info(rowCount + " rows of index which are rebuild");
- for (PTable indexPTable : indexesToPartiallyRebuild) {
- String indexTableFullName = SchemaUtil.getTableName(indexPTable.getSchemaName()
- .getString(), indexPTable.getTableName().getString());
- updateIndexState(conn, indexTableFullName, env, PIndexState.INACTIVE, PIndexState.ACTIVE);
- }
- } catch (Exception e) { // Log, but try next table's indexes
- LOG.warn("Unable to rebuild " + dataPTable + " indexes " + indexesToPartiallyRebuild
- + ". Will try again next on next scheduled invocation.", e);
- }
- }
- }
- } catch (Throwable t) {
- LOG.warn("ScheduledBuildIndexTask failed!", t);
- } finally {
- inProgress.decrementAndGet();
- if (scanner != null) {
- try {
- scanner.close();
- } catch (IOException ignored) {
- LOG.debug("ScheduledBuildIndexTask can't close scanner.", ignored);
- }
- }
- if (conn != null) {
- try {
- conn.close();
- } catch (SQLException ignored) {
- LOG.debug("ScheduledBuildIndexTask can't close connection", ignored);
- }
- }
- }
+ if (dataTableToIndexesMap != null) {
+ long overlapTime = env.getConfiguration().getLong(
+ QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB,
+ QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME);
+ for (Map.Entry<PTable, List<PTable>> entry : dataTableToIndexesMap.entrySet()) {
+ PTable dataPTable = entry.getKey();
+ List<PTable> indexesToPartiallyRebuild = entry.getValue();
+ ReadOnlyProps props = new ReadOnlyProps(env.getConfiguration().iterator());
+ try (HTableInterface metaTable = env.getTable(
+ SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props))) {
+ long earliestDisableTimestamp = Long.MAX_VALUE;
+ List<IndexMaintainer> maintainers = Lists
+ .newArrayListWithExpectedSize(indexesToPartiallyRebuild.size());
+ for (PTable index : indexesToPartiallyRebuild) {
+ long disabledTimeStampVal = index.getIndexDisableTimestamp();
+ if (disabledTimeStampVal > 0) {
+ if (disabledTimeStampVal < earliestDisableTimestamp) {
+ earliestDisableTimestamp = disabledTimeStampVal;
+ }
+
+ maintainers.add(index.getIndexMaintainer(dataPTable, conn));
+ }
+ }
+ // No indexes are disabled, so skip this table
+ if (earliestDisableTimestamp == Long.MAX_VALUE) {
+ continue;
+ }
+ long timeStamp = Math.max(0, earliestDisableTimestamp - overlapTime);
+ LOG.info("Starting to build " + dataPTable + " indexes " + indexesToPartiallyRebuild
+ + " from timestamp=" + timeStamp);
+
+ TableRef tableRef = new TableRef(null, dataPTable, HConstants.LATEST_TIMESTAMP, false);
+ // TODO Need to set high timeout
+ PostDDLCompiler compiler = new PostDDLCompiler(conn);
+ MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), null, null, null,
+ HConstants.LATEST_TIMESTAMP);
+ Scan dataTableScan = IndexManagementUtil.newLocalStateScan(plan.getContext().getScan(),
+ maintainers);
+
+ long scanEndTime = getTimestampForBatch(timeStamp,
+ batchExecutedPerTableMap.get(dataPTable.getName()));
+ dataTableScan.setTimeRange(timeStamp, scanEndTime);
+ dataTableScan.setCacheBlocks(false);
+ dataTableScan.setAttribute(BaseScannerRegionObserver.REBUILD_INDEXES, TRUE_BYTES);
+
+ ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(
+ ByteUtil.EMPTY_BYTE_ARRAY);
+ IndexMaintainer.serializeAdditional(dataPTable, indexMetaDataPtr, indexesToPartiallyRebuild,
+ conn);
+ byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
+ dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+ MutationState mutationState = plan.execute();
+ long rowCount = mutationState.getUpdateCount();
+ LOG.info(rowCount + " rows of index which are rebuild");
+ for (PTable indexPTable : indexesToPartiallyRebuild) {
+ String indexTableFullName = SchemaUtil.getTableName(
+ indexPTable.getSchemaName().getString(),
+ indexPTable.getTableName().getString());
+ if (scanEndTime == HConstants.LATEST_TIMESTAMP) {
+ updateIndexState(conn, indexTableFullName, env, PIndexState.INACTIVE,
+ PIndexState.ACTIVE);
+ batchExecutedPerTableMap.remove(dataPTable.getName());
+ } else {
+
+ updateDisableTimestamp(conn, indexTableFullName, env, scanEndTime, metaTable);
+ Long noOfBatches = batchExecutedPerTableMap.get(dataPTable.getName());
+ if (noOfBatches == null) {
+ noOfBatches = 0l;
+ }
+ batchExecutedPerTableMap.put(dataPTable.getName(), ++noOfBatches);
+ // clearing cache to get the updated
+ // disabled timestamp
+ new MetaDataClient(conn).updateCache(dataPTable.getSchemaName().getString(),
+ dataPTable.getTableName().getString());
+ new MetaDataClient(conn).updateCache(indexPTable.getSchemaName().getString(),
+ indexPTable.getTableName().getString());
+ LOG.info(
+ "During Round-robin build: Successfully updated index disabled timestamp for "
+ + indexTableFullName + " to " + scanEndTime);
+ }
+
+ }
+ } catch (Exception e) { // Log, but try next table's
+ // indexes
+ LOG.warn("Unable to rebuild " + dataPTable + " indexes " + indexesToPartiallyRebuild
+ + ". Will try again next on next scheduled invocation.", e);
+ }
+ }
+ }
+ } catch (Throwable t) {
+ LOG.warn("ScheduledBuildIndexTask failed!", t);
+ } finally {
+ inProgress.decrementAndGet();
+ if (scanner != null) {
+ try {
+ scanner.close();
+ } catch (IOException ignored) {
+ LOG.debug("ScheduledBuildIndexTask can't close scanner.", ignored);
+ }
+ }
+ if (conn != null) {
+ try {
+ conn.close();
+ } catch (SQLException ignored) {
+ LOG.debug("ScheduledBuildIndexTask can't close connection", ignored);
+ }
+ }
+ }
}
- }
-
- private static void updateIndexState(PhoenixConnection conn, String indexTableName, RegionCoprocessorEnvironment env, PIndexState oldState,
- PIndexState newState) throws ServiceException, Throwable {
- byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
- String schemaName = SchemaUtil.getSchemaNameFromFullName(indexTableName);
- String indexName = SchemaUtil.getTableNameFromFullName(indexTableName);
- // Mimic the Put that gets generated by the client on an update of the index state
- Put put = new Put(indexTableKey);
- put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
- newState.getSerializedBytes());
- if (newState == PIndexState.ACTIVE) {
- put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
- PLong.INSTANCE.toBytes(0));
+
+ private long getTimestampForBatch(long disabledTimeStamp, Long noOfBatches) {
+ if (disabledTimeStamp < 0 || rebuildIndexBatchSize > (HConstants.LATEST_TIMESTAMP
+ - disabledTimeStamp)) { return HConstants.LATEST_TIMESTAMP; }
+ long timestampForNextBatch = disabledTimeStamp + rebuildIndexBatchSize;
+ if (timestampForNextBatch < 0 || timestampForNextBatch > System.currentTimeMillis()
+ || (noOfBatches != null && noOfBatches > configuredBatches)) {
+ // if timestampForNextBatch cross current time , then we should
+ // build the complete index
+ timestampForNextBatch = HConstants.LATEST_TIMESTAMP;
+ }
+ return timestampForNextBatch;
}
- final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
- MetaDataMutationResult result = conn.getQueryServices().updateIndexState(tableMetadata, null);
- MutationCode code = result.getMutationCode();
- if (code == MutationCode.TABLE_NOT_FOUND) { throw new TableNotFoundException(schemaName, indexName); }
- if (code == MutationCode.UNALLOWED_TABLE_MUTATION) { throw new SQLExceptionInfo.Builder(
- SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION)
- .setMessage(" currentState=" + oldState + ". requestedState=" + newState).setSchemaName(schemaName)
- .setTableName(indexName).build().buildException(); }
}
+
+ private static void updateIndexState(PhoenixConnection conn, String indexTableName,
+ RegionCoprocessorEnvironment env, PIndexState oldState, PIndexState newState)
+ throws ServiceException, Throwable {
+ byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+ String schemaName = SchemaUtil.getSchemaNameFromFullName(indexTableName);
+ String indexName = SchemaUtil.getTableNameFromFullName(indexTableName);
+ // Mimic the Put that gets generated by the client on an update of the
+ // index state
+ Put put = new Put(indexTableKey);
+ put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+ newState.getSerializedBytes());
+ if (newState == PIndexState.ACTIVE) {
+ put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+ PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES, PLong.INSTANCE.toBytes(0));
+ put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+ PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP_BYTES, PLong.INSTANCE.toBytes(0));
+ }
+ final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
+ MetaDataMutationResult result = conn.getQueryServices().updateIndexState(tableMetadata, null);
+ MutationCode code = result.getMutationCode();
+ if (code == MutationCode.TABLE_NOT_FOUND) {
+ throw new TableNotFoundException(schemaName, indexName);
+ }
+ if (code == MutationCode.UNALLOWED_TABLE_MUTATION) {
+ throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION)
+ .setMessage(" currentState=" + oldState + ". requestedState=" + newState).setSchemaName(schemaName)
+ .setTableName(indexName).build().buildException();
+ }
+ }
+
+ private static void updateDisableTimestamp(PhoenixConnection conn, String indexTableName,
+ RegionCoprocessorEnvironment env, long disabledTimestamp, HTableInterface metaTable) throws IOException {
+ byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+ Put put = new Put(indexTableKey);
+ put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
+ PLong.INSTANCE.toBytes(disabledTimestamp));
+ metaTable.checkAndPut(indexTableKey, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+ PhoenixDatabaseMetaData.INDEX_STATE_BYTES, CompareOp.EQUAL, PIndexState.INACTIVE.getSerializedBytes(),
+ put);
+
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index fb4e3c3..fde403c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -421,7 +421,8 @@ public enum SQLExceptionCode {
724, "43M07", "Schema name not allowed!!"), CREATE_SCHEMA_NOT_ALLOWED(725, "43M08",
"Cannot create schema because config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+ " for enabling name space mapping isn't enabled."), INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES(
- 726, "43M10", " Inconsistent namespace mapping properites..");
+ 726, "43M10", " Inconsistent namespace mapping properites.."), ASYNC_NOT_ALLOWED(
+ 727, "43M11", " ASYNC option is not allowed.. ");
private final int errorCode;
private final String sqlState;
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index eb73d6b..e515dbb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -35,30 +35,21 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
-import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
import org.apache.phoenix.hbase.index.write.DelegateIndexFailurePolicy;
import org.apache.phoenix.hbase.index.write.KillServerOnFailurePolicy;
import org.apache.phoenix.jdbc.PhoenixConnection;
import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.protobuf.ProtobufUtil;
import org.apache.phoenix.query.QueryServices;
import org.apache.phoenix.query.QueryServicesOptions;
import org.apache.phoenix.schema.PIndexState;
import org.apache.phoenix.schema.PTable;
import org.apache.phoenix.schema.PTable.IndexType;
-import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.IndexUtil;
import org.apache.phoenix.util.MetaDataUtil;
import org.apache.phoenix.util.PhoenixRuntime;
import org.apache.phoenix.util.QueryUtil;
@@ -153,47 +144,10 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
String indexTableName = tableTimeElement.getKey();
long minTimeStamp = tableTimeElement.getValue();
// Disable the index by using the updateIndexState method of MetaDataProtocol end point coprocessor.
- byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
HTableInterface systemTable = env.getTable(SchemaUtil
.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
- // Mimic the Put that gets generated by the client on an update of the index state
- Put put = new Put(indexTableKey);
- if (blockWriteRebuildIndex)
- put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
- PIndexState.ACTIVE.getSerializedBytes());
- else
- put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
- PIndexState.DISABLE.getSerializedBytes());
- put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
- PLong.INSTANCE.toBytes(minTimeStamp));
- final List<Mutation> tableMetadata = Collections.<Mutation>singletonList(put);
-
- final Map<byte[], MetaDataResponse> results =
- systemTable.coprocessorService(MetaDataService.class, indexTableKey, indexTableKey,
- new Batch.Call<MetaDataService, MetaDataResponse>() {
- @Override
- public MetaDataResponse call(MetaDataService instance) throws IOException {
- ServerRpcController controller = new ServerRpcController();
- BlockingRpcCallback<MetaDataResponse> rpcCallback =
- new BlockingRpcCallback<MetaDataResponse>();
- UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
- for (Mutation m : tableMetadata) {
- MutationProto mp = ProtobufUtil.toProto(m);
- builder.addTableMetadataMutations(mp.toByteString());
- }
- instance.updateIndexState(controller, builder.build(), rpcCallback);
- if (controller.getFailedOn() != null) {
- throw controller.getFailedOn();
- }
- return rpcCallback.get();
- }
- });
- if (results.isEmpty()) {
- throw new IOException("Didn't get expected result size");
- }
- MetaDataResponse tmpResponse = results.values().iterator().next();
- MetaDataMutationResult result = MetaDataMutationResult.constructFromProto(tmpResponse);
-
+ MetaDataMutationResult result = IndexUtil.disableIndexWithTimestamp(indexTableName, minTimeStamp,
+ systemTable, blockWriteRebuildIndex);
if (result.getMutationCode() == MutationCode.TABLE_NOT_FOUND) {
LOG.info("Index " + indexTableName + " has been dropped. Ignore uncommitted mutations");
continue;
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 54080d1..5142b57 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -279,6 +279,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
public static final String LAST_STATS_UPDATE_TIME = "LAST_STATS_UPDATE_TIME";
public static final byte[] LAST_STATS_UPDATE_TIME_BYTES = Bytes.toBytes(LAST_STATS_UPDATE_TIME);
public static final String GUIDE_POST_KEY = "GUIDE_POST_KEY";
+ public static final String ASYNC_REBUILD_TIMESTAMP = "ASYNC_REBUILD_TIMESTAMP";
+ public static final byte[] ASYNC_REBUILD_TIMESTAMP_BYTES = Bytes.toBytes(ASYNC_REBUILD_TIMESTAMP);
public static final String PARENT_TENANT_ID = "PARENT_TENANT_ID";
public static final byte[] PARENT_TENANT_ID_BYTES = Bytes.toBytes(PARENT_TENANT_ID);
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index d57c250..f3c6d30 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -989,8 +989,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
private static class ExecutableAlterIndexStatement extends AlterIndexStatement implements CompilableStatement {
- public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
- super(indexTableNode, dataTableName, ifExists, state);
+ public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+ super(indexTableNode, dataTableName, ifExists, state, async);
}
@SuppressWarnings("unchecked")
@@ -1302,8 +1302,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
}
@Override
- public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
- return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+ public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+ return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
}
@Override
[21/26] phoenix git commit: PHOENIX-3613 Avoid possible SQL Injection
with proper input validations(Rajeshbabu)
Posted by ma...@apache.org.
PHOENIX-3613 Avoid possible SQL Injection with proper input validations(Rajeshbabu)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/4e4f7ddb
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/4e4f7ddb
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/4e4f7ddb
Branch: refs/heads/calcite
Commit: 4e4f7ddba9bddec94793e92698346dedfb520c9e
Parents: e7ef25e
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 23:11:43 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 23:11:43 2017 +0530
----------------------------------------------------------------------
.../tracingwebapp/http/EntityFactory.java | 19 +-----------------
.../tracingwebapp/http/TraceServlet.java | 21 ++++++++++++++++++--
2 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e4f7ddb/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
index afb6312..a17630d 100644
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
@@ -39,29 +39,12 @@ public class EntityFactory {
this.connection = connection;
}
- public Map<String, Object> findSingle(Object[] params) throws SQLException {
- List<Map<String, Object>> objects = this.findMultiple(params);
-
- if (objects.size() != 1) {
- throw new SQLException("Query did not produce one object it produced: "
- + objects.size() + " objects.");
- }
-
- Map<String, Object> object = objects.get(0); // get first record;
-
- return object;
- }
-
- public List<Map<String, Object>> findMultiple(Object[] params)
+ public List<Map<String, Object>> findMultiple()
throws SQLException {
ResultSet rs = null;
PreparedStatement ps = null;
try {
ps = this.connection.prepareStatement(this.queryString);
- for (int i = 0; i < params.length; ++i) {
- ps.setObject(1, params[i]);
- }
-
rs = ps.executeQuery();
return getEntitiesFromResultSet(rs);
} catch (SQLException e) {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e4f7ddb/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
index de047ba..c20b20d 100755
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
@@ -25,7 +25,7 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.codehaus.jackson.map.ObjectMapper;
-
+import org.apache.phoenix.metrics.MetricInfo;
import java.sql.Connection;
import java.sql.SQLException;
import java.util.List;
@@ -82,6 +82,11 @@ public class TraceServlet extends HttpServlet {
if(limit == null) {
limit = DEFAULT_LIMIT;
}
+ try{
+ Long.parseLong(limit);
+ } catch (NumberFormatException e) {
+ throw new RuntimeException("The LIMIT passed to the query is not a number.", e);
+ }
String sqlQuery = "SELECT * FROM " + TRACING_TABLE + " LIMIT "+limit;
json = getResults(sqlQuery);
return getJson(json);
@@ -93,6 +98,8 @@ public class TraceServlet extends HttpServlet {
if(countby == null) {
countby = DEFAULT_COUNTBY;
}
+ // Throws exception if the column not present in the trace table.
+ MetricInfo.getColumnName(countby.toLowerCase());
String sqlQuery = "SELECT "+countby+", COUNT(*) AS count FROM " + TRACING_TABLE + " GROUP BY "+countby+" HAVING COUNT(*) > 1 ";
json = getResults(sqlQuery);
return json;
@@ -102,6 +109,16 @@ public class TraceServlet extends HttpServlet {
protected String searchTrace(String parentId, String traceId,String logic) {
String json = null;
String query = null;
+ // Check the parent Id, trace id type or long or not.
+ try {
+ Long.parseLong(parentId);
+ Long.parseLong(traceId);
+ } catch (NumberFormatException e) {
+ throw new RuntimeException("The passed parentId/traceId is not a number.", e);
+ }
+ if(!logic.equals(LOGIC_AND) || !logic.equals(LOGIC_OR)) {
+ throw new RuntimeException("Wrong logical operator passed to the query. Only "+ LOGIC_AND+","+LOGIC_OR+" are allowed.") ;
+ }
if(parentId != null && traceId != null) {
query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id="+parentId+" "+logic+" trace_id="+traceId;
}else if (parentId != null && traceId == null) {
@@ -132,7 +149,7 @@ public class TraceServlet extends HttpServlet {
con = ConnectionFactory.getConnection();
EntityFactory nutrientEntityFactory = new EntityFactory(con,sqlQuery);
List<Map<String, Object>> nutrients = nutrientEntityFactory
- .findMultiple(new Object[] {});
+ .findMultiple();
ObjectMapper mapper = new ObjectMapper();
json = mapper.writeValueAsString(nutrients);
} catch (Exception e) {
[15/26] phoenix git commit: PHOENIX-3553 Zookeeper connection should
be closed immediately after DefaultStatisticsCollector's collecting stats
done (Yeonseop Kim)
Posted by ma...@apache.org.
PHOENIX-3553 Zookeeper connection should be closed immediately after DefaultStatisticsCollector's collecting stats done (Yeonseop Kim)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/eb72fa89
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/eb72fa89
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/eb72fa89
Branch: refs/heads/calcite
Commit: eb72fa892ac105251e21355eb8b9fefb35e92819
Parents: f5de28b
Author: James Taylor <ja...@apache.org>
Authored: Fri Jan 13 11:12:48 2017 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Fri Jan 13 11:12:48 2017 -0800
----------------------------------------------------------------------
.../stats/DefaultStatisticsCollector.java | 32 ++++++++++++++------
1 file changed, 23 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb72fa89/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
index 17010ac..47ee39b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
@@ -23,6 +23,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
@@ -70,6 +72,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
private long guidePostDepth;
private long maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
+ private static final Log LOG = LogFactory.getLog(DefaultStatisticsCollector.class);
DefaultStatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp, byte[] family,
byte[] gp_width_bytes, byte[] gp_per_region_bytes) throws IOException {
@@ -119,16 +122,27 @@ class DefaultStatisticsCollector implements StatisticsCollector {
this.guidePostDepth = StatisticsUtil.getGuidePostDepth(guidepostPerRegion, guidepostWidth,
env.getRegion().getTableDesc());
} else {
- // Next check for GUIDE_POST_WIDTH on table
- HTableInterface htable = env.getTable(
- SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
- Get get = new Get(ptableKey);
- get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
- Result result = htable.get(get);
long guidepostWidth = -1;
- if (!result.isEmpty()) {
- Cell cell = result.listCells().get(0);
- guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+ HTableInterface htable = null;
+ try {
+ // Next check for GUIDE_POST_WIDTH on table
+ htable = env.getTable(
+ SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
+ Get get = new Get(ptableKey);
+ get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
+ Result result = htable.get(get);
+ if (!result.isEmpty()) {
+ Cell cell = result.listCells().get(0);
+ guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+ }
+ } finally {
+ if (htable != null) {
+ try {
+ htable.close();
+ } catch (IOException e) {
+ LOG.warn("Failed to close " + htable.getName(), e);
+ }
+ }
}
if (guidepostWidth >= 0) {
this.guidePostDepth = guidepostWidth;
[16/26] phoenix git commit: PHOENIX-3567 Use argparse for sqlline
Posted by ma...@apache.org.
PHOENIX-3567 Use argparse for sqlline
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/7332fc30
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/7332fc30
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/7332fc30
Branch: refs/heads/calcite
Commit: 7332fc30c66b67d36cee7474afb04c73d01e43c3
Parents: eb72fa8
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 12:34:48 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jan 16 16:18:08 2017 -0500
----------------------------------------------------------------------
bin/phoenix_utils.py | 5 +++++
bin/sqlline-thin.py | 17 ++++++++--------
bin/sqlline.py | 52 ++++++++++++++++++++---------------------------
3 files changed, 35 insertions(+), 39 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index 2da094f..580a78b 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -198,6 +198,11 @@ def shell_quote(args):
import pipes
return " ".join([pipes.quote(v) for v in args])
+def common_sqlline_args(parser):
+ parser.add_argument('-v', '--verbose', help='Verbosity on sqlline.', default='true')
+ parser.add_argument('-c', '--color', help='Color setting for sqlline.', default='true')
+ parser.add_argument('-fc', '--fastconnect', help='Fetch all schemas on initial connection', default='false')
+
if __name__ == "__main__":
setPath()
print "phoenix_class_path:", phoenix_class_path
http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/sqlline-thin.py
----------------------------------------------------------------------
diff --git a/bin/sqlline-thin.py b/bin/sqlline-thin.py
index e4cb540..47384d8 100755
--- a/bin/sqlline-thin.py
+++ b/bin/sqlline-thin.py
@@ -42,14 +42,16 @@ parser = argparse.ArgumentParser(description='Launches the Apache Phoenix Thin C
parser.add_argument('url', nargs='?', help='The URL to the Phoenix Query Server.', default='http://localhost:8765')
# Positional argument "sqlfile" is optional
parser.add_argument('sqlfile', nargs='?', help='A file of SQL commands to execute.', default='')
-parser.add_argument('-u', '--user', help='Username for database authentication (unsupported).', default='none')
-parser.add_argument('-p', '--password', help='Password for database authentication (unsupported).', default='none')
+# Avatica wire authentication
parser.add_argument('-a', '--authentication', help='Mechanism for HTTP authentication.', choices=('SPNEGO', 'BASIC', 'DIGEST', 'NONE'), default='')
+# Avatica wire serialization
parser.add_argument('-s', '--serialization', help='Serialization type for HTTP API.', choices=('PROTOBUF', 'JSON'), default=None)
+# Avatica authentication
parser.add_argument('-au', '--auth-user', help='Username for HTTP authentication.')
parser.add_argument('-ap', '--auth-password', help='Password for HTTP authentication.')
-parser.add_argument('-v', '--verbose', help='Verbosity on sqlline.', default='true')
-parser.add_argument('-c', '--color', help='Color setting for sqlline.', default='true')
+# Common arguments across sqlline.py and sqlline-thin.py
+phoenix_utils.common_sqlline_args(parser)
+# Parse the args
args=parser.parse_args()
phoenix_utils.setPath()
@@ -58,9 +60,6 @@ url = args.url
sqlfile = args.sqlfile
serialization_key = 'phoenix.queryserver.serialization'
-def usage_and_exit():
- sys.exit("usage: sqlline-thin.py [host[:port]] [sql_file]")
-
def cleanup_url(url):
parsed = urlparse.urlparse(url)
if parsed.scheme == "":
@@ -161,8 +160,8 @@ java_cmd = java + ' $PHOENIX_OPTS ' + \
os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
" org.apache.phoenix.queryserver.client.SqllineWrapper -d org.apache.phoenix.queryserver.client.Driver " + \
- ' -u "' + jdbc_url + '"' + " -n " + args.user + " -p " + args.password + \
- " --color=" + colorSetting + " --fastConnect=false --verbose=" + args.verbose + \
+ ' -u "' + jdbc_url + '"' + " -n none -p none " + \
+ " --color=" + colorSetting + " --fastConnect=" + args.fastconnect + " --verbose=" + args.verbose + \
" --incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
exitcode = subprocess.call(java_cmd, shell=True)
http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/sqlline.py
----------------------------------------------------------------------
diff --git a/bin/sqlline.py b/bin/sqlline.py
index 474968f..7a724de 100755
--- a/bin/sqlline.py
+++ b/bin/sqlline.py
@@ -24,6 +24,7 @@ import subprocess
import sys
import phoenix_utils
import atexit
+import argparse
global childProc
childProc = None
@@ -37,36 +38,26 @@ atexit.register(kill_child)
phoenix_utils.setPath()
-def printUsage():
- print "\nUsage: sqlline.py [zookeeper] \
-[optional_sql_file] \nExample: \n 1. sqlline.py \n \
-2. sqlline.py localhost:2181:/hbase \n 3. sqlline.py \
-localhost:2181:/hbase ../examples/stock_symbol.sql \n \
-4. sqlline.py ../examples/stock_symbol.sql"
- sys.exit(-1)
-
-if len(sys.argv) > 3:
- printUsage()
+parser = argparse.ArgumentParser(description='Launches the Apache Phoenix Client.')
+# Positional argument 'zookeepers' is optional
+parser.add_argument('zookeepers', nargs='?', help='The ZooKeeper quorum string', default='localhost:2181:/hbase')
+# Positional argument 'sqlfile' is optional
+parser.add_argument('sqlfile', nargs='?', help='A file of SQL commands to execute', default='')
+# Common arguments across sqlline.py and sqlline-thin.py
+phoenix_utils.common_sqlline_args(parser)
+# Parse the args
+args=parser.parse_args()
-sqlfile = ""
-zookeeper = ""
+zookeeper = args.zookeepers
+sqlfile = args.sqlfile
# HBase configuration folder path (where hbase-site.xml reside) for
# HBase/Phoenix client side property override
hbase_config_path = os.getenv('HBASE_CONF_DIR', phoenix_utils.current_dir)
-if len(sys.argv) == 2:
- if os.path.isfile(sys.argv[1]):
- sqlfile = sys.argv[1]
- else:
- zookeeper = sys.argv[1]
-
-if len(sys.argv) == 3:
- if os.path.isfile(sys.argv[1]):
- printUsage()
- else:
- zookeeper = sys.argv[1]
- sqlfile = sys.argv[2]
+if sqlfile and not os.path.isfile(sqlfile):
+ parser.print_help()
+ sys.exit(-1)
if sqlfile:
sqlfile = "--run=" + phoenix_utils.shell_quote([sqlfile])
@@ -101,19 +92,20 @@ if java_home:
else:
java = 'java'
-colorSetting = "true"
+colorSetting = args.color
# disable color setting for windows OS
if os.name == 'nt':
colorSetting = "false"
java_cmd = java + ' $PHOENIX_OPTS ' + \
- ' -cp "' + hbase_config_path + os.pathsep + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
+ ' -cp "' + hbase_config_path + os.pathsep + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + \
+ os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
- " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver \
--u jdbc:phoenix:" + phoenix_utils.shell_quote([zookeeper]) + \
- " -n none -p none --color=" + colorSetting + " --fastConnect=false --verbose=true \
---incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
+ " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver" + \
+ " -u jdbc:phoenix:" + phoenix_utils.shell_quote([zookeeper]) + \
+ " -n none -p none --color=" + colorSetting + " --fastConnect=" + args.fastconnect + \
+ " --verbose=" + args.verbose + " --incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
childProc = subprocess.Popen(java_cmd, shell=True)
#Wait for child process exit
[04/26] phoenix git commit: PHOENIX-32451 Support DATE and TIMESTAMP
in CONVERT_TZ()
Posted by ma...@apache.org.
PHOENIX-32451 Support DATE and TIMESTAMP in CONVERT_TZ()
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/70dc3836
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/70dc3836
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/70dc3836
Branch: refs/heads/calcite
Commit: 70dc3836ac4d472bb55e79a723a1d2a1d30d75de
Parents: e23634a
Author: Josh Elser <el...@apache.org>
Authored: Wed Nov 9 13:30:15 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Sun Dec 25 21:44:03 2016 -0500
----------------------------------------------------------------------
.../end2end/ConvertTimezoneFunctionIT.java | 22 +++++++++++++++++++-
.../function/ConvertTimezoneFunction.java | 5 +++--
2 files changed, 24 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/70dc3836/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
index 229e705..a51b6c9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
@@ -39,7 +39,7 @@ import org.junit.Test;
public class ConvertTimezoneFunctionIT extends ParallelStatsDisabledIT {
@Test
- public void testConvertTimezoneEurope() throws Exception {
+ public void testDateConvertTimezoneEurope() throws Exception {
Connection conn = DriverManager.getConnection(getUrl());
String timezone_offset_test = generateUniqueName();
String ddl = "CREATE TABLE IF NOT EXISTS " + timezone_offset_test
@@ -59,6 +59,26 @@ public class ConvertTimezoneFunctionIT extends ParallelStatsDisabledIT {
}
@Test
+ public void testTimestampConvertTimezoneEurope() throws Exception {
+ Connection conn = DriverManager.getConnection(getUrl());
+ String timezone_offset_test = generateUniqueName();
+ String ddl = "CREATE TABLE IF NOT EXISTS " + timezone_offset_test
+ + " (k1 INTEGER NOT NULL, timestamps TIMESTAMP CONSTRAINT pk PRIMARY KEY (k1))";
+ conn.createStatement().execute(ddl);
+ String dml = "UPSERT INTO " + timezone_offset_test
+ + " (k1, timestamps) VALUES (1, TO_TIMESTAMP('2014-03-01 00:00:00'))";
+ conn.createStatement().execute(dml);
+ conn.commit();
+
+ ResultSet rs = conn.createStatement().executeQuery(
+ "SELECT k1, timestamps, CONVERT_TZ(timestamps, 'UTC', 'Europe/Prague') FROM "
+ + timezone_offset_test);
+
+ assertTrue(rs.next());
+ assertEquals(1393635600000L, rs.getDate(3).getTime()); //Sat, 01 Mar 2014 01:00:00
+ }
+
+ @Test
public void testConvertTimezoneAmerica() throws Exception {
Connection conn = DriverManager.getConnection(getUrl());
String timezone_offset_test = generateUniqueName();
http://git-wip-us.apache.org/repos/asf/phoenix/blob/70dc3836/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
index f06ddbc..8d13ab6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
@@ -23,10 +23,11 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.phoenix.cache.JodaTimezoneCache;
import org.apache.phoenix.expression.Expression;
import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PTimestamp;
import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.tuple.Tuple;
import org.joda.time.DateTimeZone;
/**
@@ -35,7 +36,7 @@ import org.joda.time.DateTimeZone;
*
*/
@FunctionParseNode.BuiltInFunction(name = ConvertTimezoneFunction.NAME, args = {
- @FunctionParseNode.Argument(allowedTypes = { PDate.class }),
+ @FunctionParseNode.Argument(allowedTypes = { PTimestamp.class }),
@FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
@FunctionParseNode.Argument(allowedTypes = { PVarchar.class })})
public class ConvertTimezoneFunction extends ScalarFunction {
[02/26] phoenix git commit: PHOENIX-3516 Performance Issues with
queries that have compound filters and specify
phoenix.query.force.rowkeyorder=true
Posted by ma...@apache.org.
PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/c5046047
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/c5046047
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/c5046047
Branch: refs/heads/calcite
Commit: c5046047a78e0365d75bc696dff4870304c2b5b2
Parents: bd2acd5
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Tue Dec 20 17:56:37 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Dec 22 10:54:24 2016 -0800
----------------------------------------------------------------------
.../end2end/TenantSpecificViewIndexIT.java | 47 ++++++++++++++++++++
.../apache/phoenix/compile/WhereCompiler.java | 3 +-
.../org/apache/phoenix/util/ExpressionUtil.java | 10 +++++
.../phoenix/query/KeyRangeIntersectTest.java | 9 +++-
4 files changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index b7b8902..6ae1445 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -28,6 +29,7 @@ import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
+import java.sql.SQLException;
import java.util.Properties;
import org.apache.hadoop.hbase.util.Bytes;
@@ -284,4 +286,49 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
assertEquals("value1", rs.getString(1));
assertFalse("No other rows should have been returned for the tenant", rs.next()); // should have just returned one record since for org1 we have only one row.
}
+
+ @Test
+ public void testOverlappingDatesFilter() throws SQLException {
+ String tenantUrl = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1" + ";" + QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB + "=true";
+ String tableName = generateUniqueName();
+ String viewName = generateUniqueName();
+ String ddl = "CREATE TABLE " + tableName
+ + "(ORGANIZATION_ID CHAR(15) NOT NULL, "
+ + "PARENT_TYPE CHAR(3) NOT NULL, "
+ + "PARENT_ID CHAR(15) NOT NULL,"
+ + "CREATED_DATE DATE NOT NULL "
+ + "CONSTRAINT PK PRIMARY KEY (ORGANIZATION_ID, PARENT_TYPE, PARENT_ID, CREATED_DATE DESC)"
+ + ") VERSIONS=1,MULTI_TENANT=true,REPLICATION_SCOPE=1";
+
+ try (Connection conn = DriverManager.getConnection(getUrl());
+ Connection viewConn = DriverManager.getConnection(tenantUrl) ) {
+ // create table
+ conn.createStatement().execute(ddl);
+ // create index
+ conn.createStatement().execute("CREATE INDEX IF NOT EXISTS IDX ON " + tableName + "(PARENT_TYPE, CREATED_DATE, PARENT_ID)");
+ // create view
+ viewConn.createStatement().execute("CREATE VIEW IF NOT EXISTS " + viewName + " AS SELECT * FROM "+ tableName );
+
+ String query ="EXPLAIN SELECT PARENT_ID FROM " + viewName
+ + " WHERE PARENT_TYPE='001' "
+ + "AND (CREATED_DATE > to_date('2011-01-01') AND CREATED_DATE < to_date('2016-10-31'))"
+ + "ORDER BY PARENT_TYPE,CREATED_DATE LIMIT 501";
+
+ ResultSet rs = viewConn.createStatement().executeQuery(query);
+ String expectedPlanFormat = "CLIENT SERIAL 1-WAY RANGE SCAN OVER IDX ['tenant1 ','001','%s 00:00:00.001'] - ['tenant1 ','001','%s 00:00:00.000']" + "\n" +
+ " SERVER FILTER BY FIRST KEY ONLY" + "\n" +
+ " SERVER 501 ROW LIMIT" + "\n" +
+ "CLIENT 501 ROW LIMIT";
+ assertEquals(String.format(expectedPlanFormat, "2011-01-01", "2016-10-31"), QueryUtil.getExplainPlan(rs));
+
+ query ="EXPLAIN SELECT PARENT_ID FROM " + viewName
+ + " WHERE PARENT_TYPE='001' "
+ + " AND (CREATED_DATE >= to_date('2011-01-01') AND CREATED_DATE <= to_date('2016-01-01'))"
+ + " AND (CREATED_DATE > to_date('2012-10-21') AND CREATED_DATE < to_date('2016-10-31')) "
+ + "ORDER BY PARENT_TYPE,CREATED_DATE LIMIT 501";
+
+ rs = viewConn.createStatement().executeQuery(query);
+ assertEquals(String.format(expectedPlanFormat, "2012-10-21", "2016-01-01"), QueryUtil.getExplainPlan(rs));
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 13963d7..39451b8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -56,6 +56,7 @@ import org.apache.phoenix.schema.TableRef;
import org.apache.phoenix.schema.TypeMismatchException;
import org.apache.phoenix.schema.types.PBoolean;
import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ExpressionUtil;
import org.apache.phoenix.util.ScanUtil;
import org.apache.phoenix.util.SchemaUtil;
@@ -237,7 +238,7 @@ public class WhereCompiler {
if (LiteralExpression.isBooleanFalseOrNull(whereClause)) {
context.setScanRanges(ScanRanges.NOTHING);
- } else if (whereClause != null && !LiteralExpression.isTrue(whereClause) && !hashJoinOptimization) {
+ } else if (whereClause != null && !ExpressionUtil.evaluatesToTrue(whereClause) && !hashJoinOptimization) {
Filter filter = null;
final Counter counter = new Counter();
whereClause.accept(new KeyValueExpressionVisitor() {
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
index 65cf075..1fbb534 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
@@ -15,6 +15,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.phoenix.expression.Determinism;
import org.apache.phoenix.expression.Expression;
import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.types.PBoolean;
import org.apache.phoenix.schema.types.PDataType;
public class ExpressionUtil {
@@ -43,5 +44,14 @@ public class ExpressionUtil {
public static LiteralExpression getNullExpression(Expression expression) throws SQLException {
return LiteralExpression.newConstant(null, expression.getDataType(), expression.getDeterminism());
}
+
+ public static boolean evaluatesToTrue(Expression expression) {
+ if (isConstant(expression)) {
+ ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+ expression.evaluate(null, ptr);
+ return Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(ptr));
+ }
+ return false;
+ }
}
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
index 98c10a8..9e21b69 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
@@ -18,9 +18,9 @@
package org.apache.phoenix.query;
+import static org.apache.hadoop.hbase.util.Bytes.toBytes;
import static org.apache.phoenix.query.KeyRange.EMPTY_RANGE;
import static org.apache.phoenix.query.KeyRange.EVERYTHING_RANGE;
-import static org.apache.hadoop.hbase.util.Bytes.toBytes;
import java.util.Arrays;
import java.util.Collection;
@@ -28,6 +28,8 @@ import java.util.Collection;
import junit.framework.TestCase;
import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.DateUtil;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@@ -86,6 +88,11 @@ public class KeyRangeIntersectTest extends TestCase {
PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
EMPTY_RANGE
},
+ {
+ PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2011-01-01")), true, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true),
+ PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-10-31")), false),
+ PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true)
+ }
});
}
@Test
[12/26] phoenix git commit: PHOENIX-3565 Set failIfNoTests=false on
surefire and failsafe
Posted by ma...@apache.org.
PHOENIX-3565 Set failIfNoTests=false on surefire and failsafe
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/a164f032
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/a164f032
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/a164f032
Branch: refs/heads/calcite
Commit: a164f0327e568a515ce9da2f5760c57a4d4dd188
Parents: e4c6f5c
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 12:48:21 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 5 22:19:35 2017 -0500
----------------------------------------------------------------------
pom.xml | 2 ++
1 file changed, 2 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a164f032/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4a0292d..d47a6b8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,6 +119,8 @@
<!-- Plugin options -->
<numForkedUT>8</numForkedUT>
<numForkedIT>8</numForkedIT>
+ <it.failIfNoSpecifiedTests>false</it.failIfNoSpecifiedTests>
+ <surefire.failIfNoSpecifiedTests>false</surefire.failIfNoSpecifiedTests>
<!-- Set default encoding so multi-byte tests work correctly on the Mac -->
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
[23/26] phoenix git commit: PHOENIX-3351 Implement TODOs in
PhoenixTableModify#upsert to allow writes to tenant specific
tables(Rajeshbabu)
Posted by ma...@apache.org.
PHOENIX-3351 Implement TODOs in PhoenixTableModify#upsert to allow writes to tenant specific tables(Rajeshbabu)
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/b9323e1d
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/b9323e1d
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/b9323e1d
Branch: refs/heads/calcite
Commit: b9323e1d30ba6b449f059b86ae7b8157de16b13d
Parents: 9b7f3ca
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon Jan 23 15:05:37 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon Jan 23 15:05:37 2017 +0530
----------------------------------------------------------------------
.../apache/phoenix/compile/UpsertCompiler.java | 44 ++++++++++++++------
1 file changed, 32 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9323e1d/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 8837445..32ce6ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -98,6 +98,7 @@ import org.apache.phoenix.schema.TypeMismatchException;
import org.apache.phoenix.schema.tuple.Tuple;
import org.apache.phoenix.schema.types.PDataType;
import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PSmallint;
import org.apache.phoenix.schema.types.PTimestamp;
import org.apache.phoenix.schema.types.PUnsignedLong;
import org.apache.phoenix.schema.types.PVarbinary;
@@ -116,7 +117,7 @@ public class UpsertCompiler {
private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes,
PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
- byte[][] viewConstants, byte[] onDupKeyBytes) throws SQLException {
+ byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
byte[][] pkValues = new byte[table.getPKColumns().size()][];
// If the table uses salting, the first byte is the salting byte, set to an empty array
@@ -124,10 +125,13 @@ public class UpsertCompiler {
if (table.getBucketNum() != null) {
pkValues[0] = new byte[] {0};
}
+ for(int i = 0; i < numSplColumns; i++) {
+ pkValues[i] = values[i];
+ }
Long rowTimestamp = null; // case when the table doesn't have a row timestamp column
RowTimestampColInfo rowTsColInfo = new RowTimestampColInfo(useServerTimestamp, rowTimestamp);
- for (int i = 0; i < values.length; i++) {
- byte[] value = values[i];
+ for (int i = 0, j = numSplColumns; j < values.length; j++, i++) {
+ byte[] value = values[j];
PColumn column = table.getColumns().get(columnIndexes[i]);
if (SchemaUtil.isPKColumn(column)) {
pkValues[pkSlotIndex[i]] = value;
@@ -163,8 +167,8 @@ public class UpsertCompiler {
mutation.put(ptr, new RowMutationState(columnValues, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));
}
- private static MutationState upsertSelect(StatementContext childContext, TableRef tableRef, RowProjector projector,
- ResultIterator iterator, int[] columnIndexes, int[] pkSlotIndexes, boolean useServerTimestamp) throws SQLException {
+ public static MutationState upsertSelect(StatementContext childContext, TableRef tableRef, RowProjector projector,
+ ResultIterator iterator, int[] columnIndexes, int[] pkSlotIndexes, boolean useServerTimestamp, boolean prefixSysColValues) throws SQLException {
PhoenixStatement statement = childContext.getStatement();
PhoenixConnection connection = statement.getConnection();
ConnectionQueryServices services = connection.getQueryServices();
@@ -172,7 +176,23 @@ public class UpsertCompiler {
QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
boolean isAutoCommit = connection.getAutoCommit();
- byte[][] values = new byte[columnIndexes.length][];
+ int numSplColumns =
+ (tableRef.getTable().isMultiTenant() ? 1 : 0)
+ + (tableRef.getTable().getViewIndexId() != null ? 1 : 0);
+ byte[][] values = new byte[columnIndexes.length + numSplColumns][];
+ if(prefixSysColValues) {
+ int i = 0;
+ if(tableRef.getTable().isMultiTenant()) {
+ values[i++] = connection.getTenantId().getBytes();
+ }
+ if(tableRef.getTable().getViewIndexId() != null) {
+ values[i++] = PSmallint.INSTANCE.toBytes(tableRef.getTable().getViewIndexId());
+ }
+
+ for(int k = 0; k < pkSlotIndexes.length; k++) {
+ pkSlotIndexes[k] += (i + (tableRef.getTable().getBucketNum() != null ? 1 : 0));
+ }
+ }
int rowCount = 0;
Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
PTable table = tableRef.getTable();
@@ -192,7 +212,7 @@ public class UpsertCompiler {
try (ResultSet rs = new PhoenixResultSet(iterator, projector, childContext)) {
ImmutableBytesWritable ptr = new ImmutableBytesWritable();
while (rs.next()) {
- for (int i = 0; i < values.length; i++) {
+ for (int i = 0, j = numSplColumns; j < values.length; j++, i++) {
PColumn column = table.getColumns().get(columnIndexes[i]);
byte[] bytes = rs.getBytes(i + 1);
ptr.set(bytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : bytes);
@@ -212,9 +232,9 @@ public class UpsertCompiler {
precision, scale, SortOrder.getDefault(),
column.getMaxLength(), column.getScale(), column.getSortOrder(),
table.rowKeyOrderOptimizable());
- values[i] = ByteUtil.copyKeyBytesIfNecessary(ptr);
+ values[j] = ByteUtil.copyKeyBytesIfNecessary(ptr);
}
- setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, null);
+ setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, null, numSplColumns);
rowCount++;
// Commit a batch if auto commit is true and we're at our batch size
if (isAutoCommit && rowCount % batchSize == 0) {
@@ -256,7 +276,7 @@ public class UpsertCompiler {
StatementContext childContext = new StatementContext(statement, false);
// Clone the row projector as it's not thread safe and would be used simultaneously by
// multiple threads otherwise.
- MutationState state = upsertSelect(childContext, tableRef, projector.cloneIfNecessary(), iterator, columnIndexes, pkSlotIndexes, useSeverTimestamp);
+ MutationState state = upsertSelect(childContext, tableRef, projector.cloneIfNecessary(), iterator, columnIndexes, pkSlotIndexes, useSeverTimestamp, false);
return state;
}
@@ -798,7 +818,7 @@ public class UpsertCompiler {
public MutationState execute() throws SQLException {
ResultIterator iterator = queryPlan.iterator();
if (parallelIteratorFactory == null) {
- return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp);
+ return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
}
try {
parallelIteratorFactory.setRowProjector(projector);
@@ -1043,7 +1063,7 @@ public class UpsertCompiler {
indexMaintainer = table.getIndexMaintainer(parentTable, connection);
viewConstants = IndexUtil.getViewConstants(parentTable);
}
- setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes);
+ setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes, 0);
return new MutationState(tableRef, mutation, 0, maxSize, connection);
}