You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2017/07/17 17:01:57 UTC

phoenix git commit: PHOENIX-4027 Addendum - move testRebuildIndexConnectionProperties to its own class. Fix typo for number of rpc retries

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 b72a2bb1f -> 5512e7ae8


PHOENIX-4027 Addendum - move testRebuildIndexConnectionProperties to its own class. Fix typo for number of rpc retries


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 5512e7ae8bc49603e72b46a4d76d960003ec67b5
Parents: b72a2bb
Author: Samarth Jain <sa...@apache.org>
Authored: Mon Jul 17 10:01:33 2017 -0700
Committer: Samarth Jain <sa...@apache.org>
Committed: Mon Jul 17 10:01:45 2017 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/PhoenixRuntimeIT.java       |  65 ---------
 .../end2end/RebuildIndexConnectionPropsIT.java  | 132 +++++++++++++++++++
 .../phoenix/query/QueryServicesOptions.java     |   2 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |   6 +-
 4 files changed, 135 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5512e7ae/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index 95ab906..72ff21e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -18,8 +18,6 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
@@ -31,9 +29,6 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -45,13 +40,9 @@ import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -60,7 +51,6 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
 
 import com.google.common.collect.Sets;
 
@@ -157,59 +147,4 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         HTableInterface htable7 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
         assertTenantIds(e7, htable7, new FirstKeyOnlyFilter(), new String[] {t1, t2} );
     }
-    
-    @Test
-    public void testRebuildIndexConnectionProperties() throws Exception {
-        try (PhoenixConnection rebuildIndexConnection =
-                MetaDataRegionObserver.getRebuildIndexConnection(config)) {
-            try (PhoenixConnection regularConnection =
-                    DriverManager.getConnection(url).unwrap(PhoenixConnection.class)) {
-                String rebuildUrl = rebuildIndexConnection.getURL();
-                // assert that the url ends with expected string
-                assertTrue(
-                    rebuildUrl.contains(MetaDataRegionObserver.REBUILD_INDEX_APPEND_TO_URL_STRING));
-                // assert that the url for regular connection vs the rebuild connection is different
-                assertFalse(rebuildUrl.equals(regularConnection.getURL()));
-                Configuration rebuildQueryServicesConfig =
-                        rebuildIndexConnection.getQueryServices().getConfiguration();
-                // assert that the properties are part of the query services config
-                assertEquals(Long.toString(Long.MAX_VALUE),
-                    rebuildQueryServicesConfig.get(PhoenixRuntime.CURRENT_SCN_ATTRIB));
-                assertEquals(
-                    Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_QUERY_TIMEOUT),
-                    rebuildQueryServicesConfig.get(QueryServices.THREAD_TIMEOUT_MS_ATTRIB));
-                assertEquals(
-                    Long.toString(
-                        QueryServicesOptions.DEFAULT_INDEX_REBUILD_CLIENT_SCANNER_TIMEOUT),
-                    rebuildQueryServicesConfig.get(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
-                assertEquals(Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_TIMEOUT),
-                    rebuildQueryServicesConfig.get(HConstants.HBASE_RPC_TIMEOUT_KEY));
-                assertEquals(
-                    Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_RETRIES_COUNTER),
-                    rebuildQueryServicesConfig.get(HConstants.HBASE_CLIENT_RETRIES_NUMBER));
-                ConnectionQueryServices rebuildQueryServices = rebuildIndexConnection.getQueryServices();
-                HConnection rebuildIndexHConnection =
-                        (HConnection) Whitebox.getInternalState(rebuildQueryServices,
-                            "connection");
-                HConnection regularHConnection =
-                        (HConnection) Whitebox.getInternalState(
-                            regularConnection.getQueryServices(), "connection");
-                // assert that a new HConnection was spawned
-                assertFalse(
-                    regularHConnection.toString().equals(rebuildIndexHConnection.toString()));
-                Configuration rebuildHConnectionConfig = rebuildIndexHConnection.getConfiguration();
-                // assert that the HConnection has the desired properties needed for rebuilding
-                // indices
-                assertEquals(
-                    Long.toString(
-                        QueryServicesOptions.DEFAULT_INDEX_REBUILD_CLIENT_SCANNER_TIMEOUT),
-                    rebuildHConnectionConfig.get(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
-                assertEquals(Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_TIMEOUT),
-                    rebuildHConnectionConfig.get(HConstants.HBASE_RPC_TIMEOUT_KEY));
-                assertEquals(
-                    Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_RETRIES_COUNTER),
-                    rebuildHConnectionConfig.get(HConstants.HBASE_CLIENT_RETRIES_NUMBER));
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5512e7ae/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
new file mode 100644
index 0000000..56ae0f7
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
@@ -0,0 +1,132 @@
+/*
+ * 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 java.sql.DriverManager;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+public class RebuildIndexConnectionPropsIT extends BaseUniqueNamesOwnClusterIT {
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    private static String url;
+    private static int NUM_RPC_RETRIES = 1;
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        hbaseTestUtil = new HBaseTestingUtility(conf);
+        Map<String, String> serverProps = new HashMap<>();
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        // need at least one retry otherwise test fails
+        serverProps.put(QueryServices.INDEX_REBUILD_RPC_RETRIES_COUNTER, Long.toString(NUM_RPC_RETRIES));
+        setUpConfigForMiniCluster(conf, new ReadOnlyProps(serverProps.entrySet().iterator()));
+        hbaseTestUtil.startMiniCluster();
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        Properties driverProps = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+        try (PhoenixConnection phxConn =
+                DriverManager.getConnection(url, driverProps).unwrap(PhoenixConnection.class)) {
+        }
+    }
+
+    @Test
+    public void testRebuildIndexConnectionProperties() throws Exception {
+        try (PhoenixConnection rebuildIndexConnection =
+                MetaDataRegionObserver.getRebuildIndexConnection(hbaseTestUtil.getMiniHBaseCluster().getConfiguration())) {
+            try (PhoenixConnection regularConnection =
+                    DriverManager.getConnection(url).unwrap(PhoenixConnection.class)) {
+                String rebuildUrl = rebuildIndexConnection.getURL();
+                // assert that we are working with non-test urls
+                assertFalse(PhoenixEmbeddedDriver.isTestUrl(url));
+                assertFalse(PhoenixEmbeddedDriver.isTestUrl(rebuildUrl));
+                // assert that the url ends with expected string
+                assertTrue(
+                    rebuildUrl.contains(MetaDataRegionObserver.REBUILD_INDEX_APPEND_TO_URL_STRING));
+                // assert that the url for regular connection vs the rebuild connection is different
+                assertFalse(rebuildUrl.equals(regularConnection.getURL()));
+                Configuration rebuildQueryServicesConfig =
+                        rebuildIndexConnection.getQueryServices().getConfiguration();
+                // assert that the properties are part of the query services config
+                assertEquals(Long.toString(Long.MAX_VALUE),
+                    rebuildQueryServicesConfig.get(PhoenixRuntime.CURRENT_SCN_ATTRIB));
+                assertEquals(
+                    Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_QUERY_TIMEOUT),
+                    rebuildQueryServicesConfig.get(QueryServices.THREAD_TIMEOUT_MS_ATTRIB));
+                assertEquals(
+                    Long.toString(
+                        QueryServicesOptions.DEFAULT_INDEX_REBUILD_CLIENT_SCANNER_TIMEOUT),
+                    rebuildQueryServicesConfig.get(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
+                assertEquals(Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_TIMEOUT),
+                    rebuildQueryServicesConfig.get(HConstants.HBASE_RPC_TIMEOUT_KEY));
+                assertEquals(
+                    Long.toString(NUM_RPC_RETRIES),
+                    rebuildQueryServicesConfig.get(HConstants.HBASE_CLIENT_RETRIES_NUMBER));
+                ConnectionQueryServices rebuildQueryServices = rebuildIndexConnection.getQueryServices();
+                HConnection rebuildIndexHConnection =
+                        (HConnection) Whitebox.getInternalState(rebuildQueryServices,
+                            "connection");
+                HConnection regularHConnection =
+                        (HConnection) Whitebox.getInternalState(
+                            regularConnection.getQueryServices(), "connection");
+                // assert that a new HConnection was created
+                assertFalse(
+                    regularHConnection.toString().equals(rebuildIndexHConnection.toString()));
+                Configuration rebuildHConnectionConfig = rebuildIndexHConnection.getConfiguration();
+                // assert that the HConnection has the desired properties needed for rebuilding
+                // indices
+                assertEquals(
+                    Long.toString(
+                        QueryServicesOptions.DEFAULT_INDEX_REBUILD_CLIENT_SCANNER_TIMEOUT),
+                    rebuildHConnectionConfig.get(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD));
+                assertEquals(Long.toString(QueryServicesOptions.DEFAULT_INDEX_REBUILD_RPC_TIMEOUT),
+                    rebuildHConnectionConfig.get(HConstants.HBASE_RPC_TIMEOUT_KEY));
+                assertEquals(
+                    Long.toString(NUM_RPC_RETRIES),
+                    rebuildHConnectionConfig.get(HConstants.HBASE_CLIENT_RETRIES_NUMBER));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5512e7ae/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 ee9e0b8..553d1e9 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
@@ -175,7 +175,7 @@ public class QueryServicesOptions {
     public static final long DEFAULT_INDEX_REBUILD_QUERY_TIMEOUT = 30000 * 60; // 30 mins
     public static final long DEFAULT_INDEX_REBUILD_RPC_TIMEOUT = 30000 * 60; // 30 mins
     public static final long DEFAULT_INDEX_REBUILD_CLIENT_SCANNER_TIMEOUT = 30000 * 60; // 30 mins
-    public static final int DEFAULT_INDEX_REBUILD_RPC_RETRIES_COUNTER = 1; // no retries at rpc level
+    public static final int DEFAULT_INDEX_REBUILD_RPC_RETRIES_COUNTER = 0; // no retries at rpc level
     public static final int DEFAULT_INDEX_REBUILD_DISABLE_TIMESTAMP_THRESHOLD = 30000 * 60; // 30 mins
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5512e7ae/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index b8406b4..02cbb6c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -351,9 +351,7 @@ public final class QueryUtil {
         return getConnectionUrl(props, conf, null);
     }
     /**
-     * @return connection url using the various properties set in props and conf. This method is an
-     *         alternative to {@link #getConnectionUrlUsingProps(Properties, String)} when all the
-     *         relevant connection properties are passed in both {@link Properties} and {@link Configuration}
+     * @return connection url using the various properties set in props and conf.
      */
     public static String getConnectionUrl(Properties props, Configuration conf, String principal)
             throws ClassNotFoundException, SQLException {
@@ -377,7 +375,7 @@ public final class QueryUtil {
                 props.getProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, defaultExtraArgs);
         if (extraArgs.length() > 0) {
             url +=
-                    PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + extraArgs
+                    PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR + extraArgs
                             + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
         } else {
             url += PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;