You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2020/01/10 00:51:13 UTC

[phoenix] branch 4.x-HBase-1.4 updated: PHOENIX-5654: String values (ALWAYS and NEVER) don't work for connection level config phoenix.default.update.cache.frequency.

This is an automated email from the ASF dual-hosted git repository.

chinmayskulkarni pushed a commit to branch 4.x-HBase-1.4
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.4 by this push:
     new 3f930cd  PHOENIX-5654: String values (ALWAYS and NEVER) don't work for connection level config phoenix.default.update.cache.frequency.
3f930cd is described below

commit 3f930cded99bd578c15f50c369f3205388d88081
Author: Nitesh Maheshwari <nm...@salesforce.com>
AuthorDate: Wed Jan 8 19:08:14 2020 -0800

    PHOENIX-5654: String values (ALWAYS and NEVER) don't work for connection level config phoenix.default.update.cache.frequency.
    
    Signed-off-by: Chinmay Kulkarni <ch...@apache.org>
---
 .../org/apache/phoenix/end2end/CreateTableIT.java  | 132 ++++++++++++++-------
 .../java/org/apache/phoenix/rpc/UpdateCacheIT.java |  25 ++++
 .../phoenix/query/ConnectionQueryServicesImpl.java |  14 ++-
 .../apache/phoenix/query/QueryServicesOptions.java |   2 +-
 .../apache/phoenix/schema/ConnectionProperty.java  |  55 +++++++++
 .../org/apache/phoenix/schema/MetaDataClient.java  |  11 +-
 .../org/apache/phoenix/schema/TableProperty.java   |  23 +++-
 7 files changed, 208 insertions(+), 54 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 1b2b8bd..4f6ccab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -32,6 +32,8 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Properties;
 
@@ -620,56 +622,100 @@ public class CreateTableIT extends ParallelStatsDisabledIT {
         }
     }
 
+    private void verifyUCFValueInSysCat(String tableName, String createTableString,
+            Properties props, long expectedUCFInSysCat) throws SQLException {
+        String readSysCatQuery = "SELECT TABLE_NAME, UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                + "WHERE TABLE_NAME = '" + tableName + "'  AND TABLE_TYPE='u'";
+
+        try (Connection connection = DriverManager.getConnection(getUrl(), props);
+                Statement stmt = connection.createStatement()) {
+            stmt.execute(createTableString);
+            try (ResultSet rs = stmt.executeQuery(readSysCatQuery)) {
+                assertTrue(rs.next());
+                assertEquals(expectedUCFInSysCat, rs.getLong(2));
+            }
+            stmt.execute("drop table " + tableName);
+        }
+    }
+
     @Test
-    public void testCreateTableWithUpdateCacheFrequencyAttrib() throws Exception {
-        Connection connection = null;
+    public void testCreateTableNoUpdateCacheFreq() throws Exception {
         String tableName = generateUniqueName();
-        try {
-            Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-            connection = DriverManager.getConnection(getUrl(), props);
-
-            // Assert update cache frequency to default value zero
-            connection.createStatement().execute(
-                "create table " + tableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-            String readSysCatQuery =
-                    "select TABLE_NAME,UPDATE_CACHE_FREQUENCY from SYSTEM.CATALOG where "
-                            + "TABLE_NAME = '" + tableName + "'  AND TABLE_TYPE='u'";
-            ResultSet rs = connection.createStatement().executeQuery(readSysCatQuery);
-            Assert.assertTrue(rs.next());
-            Assert.assertEquals(0, rs.getLong(2));
-            connection.createStatement().execute("drop table " + tableName);
-            connection.close();
-
-            // Assert update cache frequency to configured default value 10sec
-            int defaultUpdateCacheFrequency = 10000;
-            props.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB,
-                "" + defaultUpdateCacheFrequency);
-            connection = DriverManager.getConnection(getUrl(), props);
-            connection.createStatement().execute(
-                "create table " + tableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-            rs = connection.createStatement().executeQuery(readSysCatQuery);
-            Assert.assertTrue(rs.next());
-            Assert.assertEquals(defaultUpdateCacheFrequency, rs.getLong(2));
-            connection.createStatement().execute("drop table " + tableName);
-
-            // Assert update cache frequency to table specific value 30sec
-            int tableSpecificUpdateCacheFrequency = 30000;
-            connection.createStatement()
-                    .execute("create table " + tableName
-                            + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
-                            + "UPDATE_CACHE_FREQUENCY=" + tableSpecificUpdateCacheFrequency);
-            rs = connection.createStatement().executeQuery(readSysCatQuery);
-            Assert.assertTrue(rs.next());
-            Assert.assertEquals(tableSpecificUpdateCacheFrequency, rs.getLong(2));
-        } finally {
-            if (connection != null) {
-                connection.createStatement().execute("drop table if exists " + tableName);
-                connection.close();
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        String createTableString = "CREATE TABLE " + tableName + " (k VARCHAR PRIMARY KEY, "
+                + "v1 VARCHAR, v2 VARCHAR)";
+        verifyUCFValueInSysCat(tableName, createTableString, props, 0L);
+    }
+
+    @Test
+    public void testCreateTableWithTableLevelUpdateCacheFreq() throws Exception {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+
+        HashMap<String, Long> expectedUCF = new HashMap<>();
+        expectedUCF.put("10", new Long(10L));
+        expectedUCF.put("0", new Long(0L));
+        expectedUCF.put("10000", new Long(10000L));
+        expectedUCF.put("ALWAYS", new Long(0L));
+        expectedUCF.put("NEVER", new Long(Long.MAX_VALUE));
+
+        for (HashMap.Entry<String, Long> entry : expectedUCF.entrySet()) {
+            String tableLevelUCF = entry.getKey();
+            long expectedUCFInSysCat = entry.getValue();
+
+            String createTableString = "CREATE TABLE " + tableName + " (k VARCHAR PRIMARY KEY,"
+                    + "v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY = " + tableLevelUCF;
+            verifyUCFValueInSysCat(tableName, createTableString, props, expectedUCFInSysCat);
+        }
+    }
+
+    @Test
+    public void testCreateTableWithInvalidTableUpdateCacheFreqShouldThrow() throws Exception {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+
+        ArrayList<String> invalidUCF = new ArrayList<>();
+        invalidUCF.add("GIBBERISH");
+        invalidUCF.add("10000.6");
+
+        for (String tableLevelUCF : invalidUCF) {
+            String createTableString = "CREATE TABLE " + tableName + " (k VARCHAR PRIMARY KEY,"
+                    + "v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY = " + tableLevelUCF;
+            try {
+                verifyUCFValueInSysCat(tableName, createTableString, props, -1L);
+                fail();
+            } catch (IllegalArgumentException e) {
+                // expected
+                assertTrue(e.getMessage().contains("Table's " +
+                        PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY));
             }
         }
     }
 
     @Test
+    public void testCreateTableWithConnLevelUpdateCacheFreq() throws Exception {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+
+        HashMap<String, Long> expectedUCF = new HashMap<>();
+        expectedUCF.put("10", new Long(10L));
+        expectedUCF.put("0", new Long(0L));
+        expectedUCF.put("10000", new Long(10000L));
+        expectedUCF.put("ALWAYS", new Long(0L));
+        expectedUCF.put("NEVER", new Long(Long.MAX_VALUE));
+
+        for (HashMap.Entry<String, Long> entry : expectedUCF.entrySet()) {
+            String connLevelUCF = entry.getKey();
+            long expectedUCFInSysCat = entry.getValue();
+
+            String createTableString = "CREATE TABLE " + tableName + " (k VARCHAR PRIMARY KEY,"
+                    + "v1 VARCHAR, v2 VARCHAR)";
+            props.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, connLevelUCF);
+            verifyUCFValueInSysCat(tableName, createTableString, props, expectedUCFInSysCat);
+        }
+    }
+
+    @Test
     public void testCreateTableWithNamespaceMappingEnabled() throws Exception {
         final String NS = "NS_" + generateUniqueName();
         final String TBL = "TBL_" + generateUniqueName();
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index e21b933..302089d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -19,8 +19,10 @@ package org.apache.phoenix.rpc;
 
 import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.isNull;
@@ -33,12 +35,14 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
@@ -244,4 +248,25 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
         }
         assertFalse(rs.next());
     }
+
+    @Test
+    public void testInvalidConnUpdateCacheFrequencyShouldThrow() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+
+        ArrayList<String> invalidUCF = new ArrayList<>();
+        invalidUCF.add("GIBBERISH");
+        invalidUCF.add("10000.6");
+
+        for (String connLevelUCF : invalidUCF) {
+            props.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, connLevelUCF);
+            try {
+                DriverManager.getConnection(getUrl(), props);
+                fail();
+            } catch (IllegalArgumentException e) {
+                // expected
+                assertTrue(e.getMessage().contains("Connection's " +
+                        QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB));
+            }
+        }
+    }
 }
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 959be3f..6e158eb 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
@@ -212,6 +212,7 @@ import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ConnectionProperty;
 import org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
@@ -748,17 +749,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         });
     }
 
+    /**
+     * Check that the supplied connection properties are set to valid values.
+     * @param info The properties to be validated.
+     * @throws IllegalArgumentException when a property is not set to a valid value.
+     */
+    private void validateConnectionProperties(Properties info) {
+        if (info.get(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB) != null) {
+            ConnectionProperty.UPDATE_CACHE_FREQUENCY.getValue(
+                    info.getProperty(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB));
+        }
+    }
 
     @Override
     public PhoenixConnection connect(String url, Properties info) throws SQLException {
         checkClosed();
         PMetaData metadata = latestMetaData;
         throwConnectionClosedIfNullMetaData();
+        validateConnectionProperties(info);
         metadata = metadata.clone();
         return new PhoenixConnection(this, url, info, metadata);
     }
 
-
     private HColumnDescriptor generateColumnFamilyDescriptor(Pair<byte[],Map<String,Object>> family, PTableType tableType) throws SQLException {
         HColumnDescriptor columnDesc = new HColumnDescriptor(family.getFirst());
         if (tableType != PTableType.VIEW) {
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 c7a293a..3dab605 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
@@ -348,7 +348,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_PHOENIX_ACLS_ENABLED = false;
 
     //default update cache frequency
-    public static final int DEFAULT_UPDATE_CACHE_FREQUENCY = 0;
+    public static final long DEFAULT_UPDATE_CACHE_FREQUENCY = 0;
     public static final int DEFAULT_SMALL_SCAN_THRESHOLD = 100;
 
     // default system task handling interval in milliseconds
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ConnectionProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConnectionProperty.java
new file mode 100644
index 0000000..c44490e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ConnectionProperty.java
@@ -0,0 +1,55 @@
+/*
+ * 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.schema;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+
+public enum ConnectionProperty {
+    /**
+     * Connection level property phoenix.default.update.cache.frequency
+     */
+    UPDATE_CACHE_FREQUENCY() {
+        @Override
+        public Object getValue(String value) {
+            if (value == null) {
+                return QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY;
+            }
+
+            if ("ALWAYS".equalsIgnoreCase(value)) {
+                return 0L;
+            }
+
+            if ("NEVER".equalsIgnoreCase(value)) {
+                return Long.MAX_VALUE;
+            }
+
+            try {
+                return Long.parseLong(value);
+            } catch (NumberFormatException e) {
+                throw new IllegalArgumentException("Connection's " +
+                        QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB +
+                        " can only be set to 'ALWAYS', 'NEVER' or a millisecond numeric value.");
+            }
+        }
+    };
+
+    public Object getValue(String value) {
+        return value;
+    }
+}
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 9c446d5..75b86c3 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
@@ -78,7 +78,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYNC_INDEX_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
@@ -998,8 +997,9 @@ public class MetaDataClient {
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps, statement.getTableType());
 
         boolean isAppendOnlySchema = false;
-        long updateCacheFrequency = connection.getQueryServices().getProps().getLong(
-            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+        long updateCacheFrequency = (Long) ConnectionProperty.UPDATE_CACHE_FREQUENCY.getValue(
+                connection.getQueryServices().getProps().get(
+                        QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB));
         Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps);
         if (parent==null) {
 	        Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
@@ -2098,8 +2098,9 @@ public class MetaDataClient {
             if (disableWALProp != null) {
                 disableWAL = disableWALProp;
             }
-            long updateCacheFrequency = connection.getQueryServices().getProps().getLong(
-                QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+            long updateCacheFrequency = (Long) ConnectionProperty.UPDATE_CACHE_FREQUENCY.getValue(
+                    connection.getQueryServices().getProps().get(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB));
             if (tableType == PTableType.INDEX && parent != null) {
                 updateCacheFrequency = parent.getUpdateCacheFrequency();
             }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
index ab667c2..5e8324e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
@@ -115,17 +115,32 @@ public enum TableProperty {
     UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true, true) {
         @Override
         public Object getValue(Object value) {
+            if (value == null) {
+                return null;
+            }
+
             if (value instanceof String) {
                 String strValue = (String) value;
                 if ("ALWAYS".equalsIgnoreCase(strValue)) {
                     return 0L;
-                } else if ("NEVER".equalsIgnoreCase(strValue)) {
+                }
+
+                if ("NEVER".equalsIgnoreCase(strValue)) {
                     return Long.MAX_VALUE;
                 }
-            } else {
-                return value == null ? null : ((Number) value).longValue();
+
+                throw new IllegalArgumentException("Table's " +
+                        PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY +
+                        " can only be set to 'ALWAYS', 'NEVER' or a millisecond numeric value.");
             }
-            return value;
+
+            if (value instanceof Integer || value instanceof Long) {
+                return ((Number) value).longValue();
+            }
+
+            throw new IllegalArgumentException("Table's " +
+                    PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY +
+                    " can only be set to 'ALWAYS', 'NEVER' or a millisecond numeric value.");
         }
 
         @Override