You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/03/30 08:00:42 UTC

[1/4] phoenix git commit: PHOENIX-4675 Better parsing around the allowed UDF jar directory configuration [Forced Update!]

Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.12 e6bda5f67 -> cab7d474f (forced update)


PHOENIX-4675 Better parsing around the allowed UDF jar directory configuration

The parsing/validation logic on the allowed path for UDF jar loading was lacking
in that it didn't correctly handle a trailing slash.


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

Branch: refs/heads/4.x-cdh5.12
Commit: bd4d15bbd8c562a0a24b72989477cf22cc481ef5
Parents: 881d7aa
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 28 00:06:45 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Mar 30 08:58:23 2018 +0100

----------------------------------------------------------------------
 .../phoenix/end2end/UserDefinedFunctionsIT.java | 42 ++++++++++----------
 .../expression/function/UDFExpression.java      | 21 ++++++----
 2 files changed, 35 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd4d15bb/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 943119d..ebb2462 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -201,7 +201,6 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     public void cleanUpAfterTest() throws Exception {
         Connection conn = driver.connect(url, EMPTY_PROPS);
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("list jars");
         stmt.execute("delete jar '"+ util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar'");
         stmt.execute("delete jar '"+ util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar2.jar'");
         stmt.execute("delete jar '"+ util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar3.jar'");
@@ -280,7 +279,8 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         util.startMiniDFSCluster(1);
         util.startMiniZKCluster(1);
         String string = util.getConfiguration().get("fs.defaultFS");
-        conf.set(DYNAMIC_JARS_DIR_KEY, string+"/hbase/tmpjars");
+        // PHOENIX-4675 setting the trailing slash implicitly tests that we're doing some path normalization
+        conf.set(DYNAMIC_JARS_DIR_KEY, string+"/hbase/tmpjars/");
         util.startMiniHBaseCluster(1, 1);
         UDFExpression.setConfig(conf);
 
@@ -297,20 +297,21 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     @Test
     public void testListJars() throws Exception {
         Connection conn = driver.connect(url, EMPTY_PROPS);
+        Path jarPath = new Path(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY));
         Statement stmt = conn.createStatement();
         ResultSet rs = stmt.executeQuery("list jars");
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar1.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar2.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar2.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar3.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar3.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar4.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar4.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar5.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar5.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar6.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar6.jar").toString(), rs.getString("jar_location"));
         assertFalse(rs.next());
     }
 
@@ -320,30 +321,31 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         Statement stmt = conn.createStatement();
         ResultSet rs = stmt.executeQuery("list jars");
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
+        Path jarPath = new Path(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY));
+        assertEquals(new Path(jarPath, "myjar1.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar2.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar2.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar3.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar3.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar4.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar4.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar5.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar5.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar6.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar6.jar").toString(), rs.getString("jar_location"));
         assertFalse(rs.next());
-        stmt.execute("delete jar '"+ util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar4.jar'");
+        stmt.execute("delete jar '"+ new Path(jarPath, "myjar4.jar").toString() + "'");
         rs = stmt.executeQuery("list jars");
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar1.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar1.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar2.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar2.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar3.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar3.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar5.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar5.jar").toString(), rs.getString("jar_location"));
         assertTrue(rs.next());
-        assertEquals(util.getConfiguration().get(QueryServices.DYNAMIC_JARS_DIR_KEY)+"/"+"myjar6.jar", rs.getString("jar_location"));
+        assertEquals(new Path(jarPath, "myjar6.jar").toString(), rs.getString("jar_location"));
         assertFalse(rs.next());
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd4d15bb/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
index c8636fd..80a569a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/UDFExpression.java
@@ -173,21 +173,18 @@ public class UDFExpression extends ScalarFunction {
 
     public static DynamicClassLoader getClassLoader(final PName tenantId, final String jarPath) {
         DynamicClassLoader cl = tenantIdSpecificCls.get(tenantId);
-        String parent = null;
+        Path parent = null;
         if (cl != null) return cl;
         if(jarPath != null && !jarPath.isEmpty()) {
             cl = pathSpecificCls.get(jarPath);
             if (cl != null) return cl;
-            Path path = new Path(jarPath);
-            if(jarPath.endsWith(".jar")) {
-                parent = path.getParent().toString();
-            } else {
-                parent = path.toString();
-            }
+            parent = getPathForParent(jarPath);
         }
+        // Parse the DYNAMIC_JARS_DIR_KEY value as a Path if it's present in the configuration
+        Path allowedDynamicJarsPath = config.get(DYNAMIC_JARS_DIR_KEY) != null ? new Path(config.get(DYNAMIC_JARS_DIR_KEY)) : null;
         // The case jarPath is not provided, or it is provided and the jar is inside hbase.dynamic.jars.dir
         if (jarPath == null || jarPath.isEmpty()
-                || config.get(DYNAMIC_JARS_DIR_KEY) != null && (parent != null && parent.equals(config.get(DYNAMIC_JARS_DIR_KEY)))) {
+                || (allowedDynamicJarsPath != null && parent != null && parent.equals(allowedDynamicJarsPath))) {
             cl = tenantIdSpecificCls.get(tenantId);
             if (cl == null) {
                 cl = new DynamicClassLoader(config, UDFExpression.class.getClassLoader());
@@ -204,6 +201,14 @@ public class UDFExpression extends ScalarFunction {
             throw new SecurityException("Loading jars from " + jarPath + " is not allowed. The only location that is allowed is "+ config.get(DYNAMIC_JARS_DIR_KEY));
         }
     }
+
+    public static Path getPathForParent(String jarPath) {
+        Path path = new Path(jarPath);
+        if (jarPath.endsWith(".jar")) {
+            return path.getParent();
+        }
+        return path;
+    }
     
     @VisibleForTesting
     public static void setConfig(Configuration conf) {


[3/4] phoenix git commit: PHOENIX-4671 Fix minor size accounting bug for MutationSize.

Posted by pb...@apache.org.
PHOENIX-4671 Fix minor size accounting bug for MutationSize.


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

Branch: refs/heads/4.x-cdh5.12
Commit: 0fa6d947ebfc237e854b206f35af3a558e28a8ee
Parents: 235def6
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Mar 29 22:42:53 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Mar 30 08:58:50 2018 +0100

----------------------------------------------------------------------
 .../end2end/UpsertSelectAutoCommitIT.java       | 28 ++++++++++++++++++++
 .../apache/phoenix/execute/MutationState.java   |  1 +
 2 files changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0fa6d947/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 6b781a0..38d48d6 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
@@ -23,15 +23,19 @@ 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.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Properties;
 
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -173,4 +177,28 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
         conn.close();
     }
 
+    @Test
+    public void testMaxMutationSize() throws Exception {
+        Properties connectionProperties = new Properties();
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "3");
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "50000");
+        PhoenixConnection connection =
+                (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        connection.setAutoCommit(true);
+        String fullTableName = generateUniqueName();
+        try (Statement stmt = connection.createStatement()) {
+            stmt.execute(
+                    "CREATE TABLE " + fullTableName + " (pk INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER)");
+            stmt.execute(
+                    "CREATE SEQUENCE " + fullTableName + "_seq cache 1000");
+            stmt.execute("UPSERT INTO " + fullTableName + " VALUES (NEXT VALUE FOR " + fullTableName + "_seq, rand(), rand())");
+        }
+        try (Statement stmt = connection.createStatement()) {
+            for (int i=0; i<16; i++) {
+                stmt.execute("UPSERT INTO " + fullTableName + " SELECT NEXT VALUE FOR " + fullTableName + "_seq, rand(), rand() FROM " + fullTableName);
+            }
+        }
+        connection.close();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0fa6d947/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 727b424..f6d11a0 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
@@ -1563,6 +1563,7 @@ public class MutationState implements SQLCloseable {
         
         public void clear(){
             rowKeyToRowMutationState.clear();
+            estimatedSize = 0;
         }
         
         public Collection<RowMutationState> values() {


[4/4] phoenix git commit: Changes for CDH 5.12.x

Posted by pb...@apache.org.
Changes for CDH 5.12.x


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

Branch: refs/heads/4.x-cdh5.12
Commit: cab7d474f81b6fb4a3ee9073a5a4075bca288c3e
Parents: 0fa6d94
Author: Pedro Boado <pb...@apache.org>
Authored: Sat Mar 10 17:54:04 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Mar 30 08:59:51 2018 +0100

----------------------------------------------------------------------
 phoenix-assembly/pom.xml                        |  2 +-
 phoenix-client/pom.xml                          |  2 +-
 phoenix-core/pom.xml                            |  2 +-
 .../hadoop/hbase/ipc/PhoenixRpcScheduler.java   | 34 ++++++++++++++++++--
 phoenix-flume/pom.xml                           |  2 +-
 phoenix-hive/pom.xml                            |  2 +-
 phoenix-kafka/pom.xml                           |  2 +-
 phoenix-load-balancer/pom.xml                   |  2 +-
 phoenix-parcel/pom.xml                          |  2 +-
 phoenix-pherf/pom.xml                           |  2 +-
 phoenix-pig/pom.xml                             |  2 +-
 phoenix-queryserver-client/pom.xml              |  2 +-
 phoenix-queryserver/pom.xml                     |  2 +-
 phoenix-server/pom.xml                          |  2 +-
 phoenix-spark/pom.xml                           |  2 +-
 phoenix-tracing-webapp/pom.xml                  |  2 +-
 pom.xml                                         |  4 +--
 17 files changed, 49 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 55a9a6e..14225ee 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index 2454de6..e211008 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index b07cbbb..61ceeb3 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
index 4fdddf5..d1f05f8 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java
@@ -124,6 +124,36 @@ public class PhoenixRpcScheduler extends RpcScheduler {
     public void setMetadataExecutorForTesting(RpcExecutor executor) {
         this.metadataCallExecutor = executor;
     }
-    
-    
+
+    @Override
+    public int getReadQueueLength() {
+        return delegate.getReadQueueLength();
+    }
+
+    @Override
+    public int getWriteQueueLength() {
+        return delegate.getWriteQueueLength();
+    }
+
+    @Override
+    public int getScanQueueLength() {
+        return delegate.getScanQueueLength();
+    }
+
+    @Override
+    public int getActiveReadRpcHandlerCount() {
+        return delegate.getActiveReadRpcHandlerCount();
+    }
+
+    @Override
+    public int getActiveWriteRpcHandlerCount() {
+        return delegate.getActiveWriteRpcHandlerCount();
+    }
+
+    @Override
+    public int getActiveScanRpcHandlerCount() {
+        return delegate.getActiveScanRpcHandlerCount();
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index 0883e5e..e1bde63 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-hive/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index 809fbea..804ba5f 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index 8718f5f..817c2a3 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -26,7 +26,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+		<version>4.14.0-cdh5.12.2-SNAPSHOT</version>
 	</parent>
 	<artifactId>phoenix-kafka</artifactId>
 	<name>Phoenix - Kafka</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-load-balancer/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-load-balancer/pom.xml b/phoenix-load-balancer/pom.xml
index 81e124a..cdb4c1b 100644
--- a/phoenix-load-balancer/pom.xml
+++ b/phoenix-load-balancer/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-load-balancer</artifactId>
   <name>Phoenix Load Balancer</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-parcel/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-parcel/pom.xml b/phoenix-parcel/pom.xml
index 5e6fccc..8e7b096 100644
--- a/phoenix-parcel/pom.xml
+++ b/phoenix-parcel/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-parcel</artifactId>
   <name>Phoenix Parcels for CDH</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 7831f35..0634a01 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+		<version>4.14.0-cdh5.12.2-SNAPSHOT</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index e5d0d52..cd4f6bc 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index 83cfde6..86c56b9 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index 7180b18..e6c32cb 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 01a7bc3..d33bdaa 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index a45d4b5..b3b6e1c 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+    <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 93edc43..5226f84 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+      <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cab7d474/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 73964c0..a0a26be 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.14.0-cdh5.11.2-SNAPSHOT</version>
+  <version>4.14.0-cdh5.12.2-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>
@@ -86,7 +86,7 @@
   <parent>
     <groupId>com.cloudera.cdh</groupId>
     <artifactId>cdh-root</artifactId>
-    <version>5.11.2</version>
+    <version>5.12.2</version>
   </parent>
 
   <scm>


[2/4] phoenix git commit: PHOENIX-4677 Add explicit commons-cli dependency

Posted by pb...@apache.org.
PHOENIX-4677 Add explicit commons-cli dependency


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

Branch: refs/heads/4.x-cdh5.12
Commit: 235def6aa8a096f2f2ddbfc466143e75aca29676
Parents: bd4d15b
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 28 20:21:07 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Mar 30 08:58:35 2018 +0100

----------------------------------------------------------------------
 phoenix-core/pom.xml  | 4 ++++
 phoenix-kafka/pom.xml | 4 ++++
 phoenix-pherf/pom.xml | 4 ++++
 phoenix-pig/pom.xml   | 4 ++++
 pom.xml               | 6 +++++-
 5 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/235def6a/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 2cb4c81..b07cbbb 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -355,6 +355,10 @@
       <artifactId>htrace-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/235def6a/phoenix-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index c2cb7db..8718f5f 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -153,6 +153,10 @@
 			<artifactId>htrace-core</artifactId>
 		</dependency>
 		<dependency>
+			<groupId>commons-cli</groupId>
+			<artifactId>commons-cli</artifactId>
+		</dependency>
+		<dependency>
 			<groupId>commons-codec</groupId>
 			<artifactId>commons-codec</artifactId>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/235def6a/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 6741bf3..7831f35 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -60,6 +60,10 @@
 			<artifactId>commons-math3</artifactId>
 			<version>3.3</version>
 		</dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
 
 		<!-- Test Dependencies -->
 		<dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/235def6a/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 31f38b8..e5d0d52 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -172,6 +172,10 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/235def6a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1479b75..73964c0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -122,7 +122,6 @@
     <log4j.version>1.2.17</log4j.version>
     <slf4j.version>${cdh.slf4j.version}</slf4j.version>
     <protobuf-java.version>2.5.0</protobuf-java.version>
-    <commons-configuration.version>1.6</commons-configuration.version>
     <commons-io.version>2.1</commons-io.version>
     <commons-lang.version>2.5</commons-lang.version>
     <commons-logging.version>1.2</commons-logging.version>
@@ -936,6 +935,11 @@
         <version>${slf4j.version}</version>
       </dependency>
       <dependency>
+        <groupId>commons-cli</groupId>
+        <artifactId>commons-cli</artifactId>
+        <version>${commons-cli.version}</version>
+      </dependency>
+      <dependency>
         <groupId>commons-logging</groupId>
         <artifactId>commons-logging</artifactId>
         <version>${commons-logging.version}</version>