You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by GitBox <gi...@apache.org> on 2019/12/18 22:54:30 UTC

[GitHub] [phoenix] ntshmah opened a new pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

ntshmah opened a new pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659
 
 
   …ction properties at query time
   
   The effective update cache frequency value for a table should be evaluated based on the following precedence order:
   Table level > Connection level > Default.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366513113
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] yanxinyi commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
yanxinyi commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361736575
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
 
 Review comment:
   nit: apache open-source license 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#issuecomment-575804793
 
 
   Thank you, @yanxinyi and @ChinmaySKulkarni for the reviews and for committing the patches. I'll close this PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361564201
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for table-level UCF set, connection-level UCF not set
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF not set
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF set
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Helper method that sets up the connections and creates the table
+    private static void setUpTableAndConnections(String fullTableName, long tableUpdateCacheFrequency, long connUpdateCacheFrequency) throws SQLException {
+        // Create two connections - a connection that we'll use to create the table and the second one that we will
+        // spy on and will have 'phoenix.default.update.cache.frequency' set
+        Properties props = new Properties();
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        conn1.setAutoCommit(true);
+
+        Properties propsWithUCF = new Properties();
+        //if (connUpdateCacheFrequency != -1) {
 
 Review comment:
   Right, I've updated this. Added some comments to this method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366512414
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+
+            return (table.getRowTimestampColPos() == -1 &&
+                    connection.getMetaDataCache().getAge(tableRef) <
+                            effectiveUpdateCacheFrequency);
+        }
+        return false;
 
 Review comment:
   I've modified the tests to also check for the UPSERT path. UPSERT follows the same behavior as SELECT.
   
   > I guess for DDL operations, we force the client to get the latest metadata
   That's not true. We have logic in PhoenixStatement#executeMutation() to [force update cache if we run into meta not found exceptions](https://github.com/apache/phoenix/blob/master/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java#L427-L437).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361758494
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
 
 Review comment:
   nit: Make `tableUpdateCacheFrequency` and `connUpdateCacheFrequency` final

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361759420
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+
+            return (table.getRowTimestampColPos() == -1 &&
+                    connection.getMetaDataCache().getAge(tableRef) <
+                            effectiveUpdateCacheFrequency);
+        }
+        return false;
 
 Review comment:
   I think it should be covered under this JIRA since the understanding is that once you set the connection-level property, that would be honored for any actions taken with that connection. Good to go through various code paths where we `getTable` and see that we only get the latest metadata based on the UCF value 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361758521
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at table-level only, and not at
+     * connection-level.
+     */
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
 
 Review comment:
   nit: Make `tableUpdateCacheFrequency` and `connUpdateCacheFrequency` final

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361564862
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for table-level UCF set, connection-level UCF not set
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF not set
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF set
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Helper method that sets up the connections and creates the table
+    private static void setUpTableAndConnections(String fullTableName, long tableUpdateCacheFrequency, long connUpdateCacheFrequency) throws SQLException {
+        // Create two connections - a connection that we'll use to create the table and the second one that we will
+        // spy on and will have 'phoenix.default.update.cache.frequency' set
+        Properties props = new Properties();
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        conn1.setAutoCommit(true);
+
+        Properties propsWithUCF = new Properties();
+        //if (connUpdateCacheFrequency != -1) {
+            propsWithUCF.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, "" + connUpdateCacheFrequency);
+
+        // use a spied ConnectionQueryServices so we can verify calls to getTable()
+        spyForConn2 = Mockito.spy(driver.getConnectionQueryServices(getUrl(), connUpdateCacheFrequency != -1 ? propsWithUCF : props));
+        conn2 = spyForConn2.connect(getUrl(), propsWithUCF);
 
 Review comment:
   I can do with a single `props` variable instead, that makes it clearer.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366516939
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at table-level only, and not at
+     * connection-level.
+     */
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
 
 Review comment:
   I've not made any changes related to this behavior here, and also the initial commits for UPDATE_CACHE_FREQUENCY feature were added a while ago so I'm assuming that works as expected. I could add new tests under a separate JIRA if you would like that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366512414
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+
+            return (table.getRowTimestampColPos() == -1 &&
+                    connection.getMetaDataCache().getAge(tableRef) <
+                            effectiveUpdateCacheFrequency);
+        }
+        return false;
 
 Review comment:
   I've modified the tests to also check for the UPSERT path. UPSERT follows the same behavior as SELECT.
   
   > I guess for DDL operations, we force the client to get the latest metadata
   
   That's not true. We have logic in PhoenixStatement#executeMutation() to [force update cache if we run into meta not found exceptions](https://github.com/apache/phoenix/blob/master/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java#L427-L437).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361758957
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at table-level only, and not at
+     * connection-level.
+     */
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
 
 Review comment:
   Is it also worth testing that when we set UCF on table1, queries for table2 are not affected?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361606430
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+
+            return (table.getRowTimestampColPos() == -1 &&
+                    connection.getMetaDataCache().getAge(tableRef) <
+                            effectiveUpdateCacheFrequency);
+        }
+        return false;
 
 Review comment:
   Would you prefer this investigation and adding of tests be done under a separate JIRA? Or this one itself?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah edited a comment on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah edited a comment on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#issuecomment-574323990
 
 
   Hi @ChinmaySKulkarni 
   I will be squashing my commits together and force pushing here since quite a few things changed from the time this was last reviewed. I hope that is fine.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366513179
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at table-level only, and not at
+     * connection-level.
+     */
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361564899
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for table-level UCF set, connection-level UCF not set
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF not set
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF set
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Helper method that sets up the connections and creates the table
+    private static void setUpTableAndConnections(String fullTableName, long tableUpdateCacheFrequency, long connUpdateCacheFrequency) throws SQLException {
+        // Create two connections - a connection that we'll use to create the table and the second one that we will
+        // spy on and will have 'phoenix.default.update.cache.frequency' set
+        Properties props = new Properties();
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        conn1.setAutoCommit(true);
+
+        Properties propsWithUCF = new Properties();
+        //if (connUpdateCacheFrequency != -1) {
+            propsWithUCF.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, "" + connUpdateCacheFrequency);
+
+        // use a spied ConnectionQueryServices so we can verify calls to getTable()
+        spyForConn2 = Mockito.spy(driver.getConnectionQueryServices(getUrl(), connUpdateCacheFrequency != -1 ? propsWithUCF : props));
+        conn2 = spyForConn2.connect(getUrl(), propsWithUCF);
+        conn2.setAutoCommit(true);
+
+        String createTableQuery =
+                "CREATE TABLE " + fullTableName + " (k UNSIGNED_DOUBLE NOT NULL PRIMARY KEY, "
+                        + "v1 UNSIGNED_DOUBLE, v2 UNSIGNED_DOUBLE, v3 UNSIGNED_DOUBLE)";
+
+        if (tableUpdateCacheFrequency != -1) {
+            createTableQuery += " UPDATE_CACHE_FREQUENCY = " + tableUpdateCacheFrequency;
+        }
+
+        // Create the table over first connection
+        conn1.createStatement().execute(createTableQuery);
+        conn1.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES (1, 2, 3, 4)");
+        conn1.commit();
+    }
+
+    // Helper method that executes a select query on the table for numSelectExecutions times and verifies that
+    // numExpectedGetTableCalls were made to getTable for the table
 
 Review comment:
   Noted, will update the headers.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361573223
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
 
 Review comment:
   Done. Created [PHOENIX-5653](https://issues.apache.org/jira/browse/PHOENIX-5653)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361564009
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for table-level UCF set, connection-level UCF not set
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF not set
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF set
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Helper method that sets up the connections and creates the table
 
 Review comment:
   Noted, will update the headers.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361759047
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.mockito.Mockito;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for connection level 'Update Cache Frequency' property.
+ *
+ * These tests verify that the 'Update Cache Frequency' is honored in the following precedence order:
+ * Table-level property > Connection-level property > Default value
+ */
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at connection-level only, and not at
+     * table-level.
+     */
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at table-level only, and not at
+     * connection-level.
+     */
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is not set at both table-level and
+     * connection-level.
+     */
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Test 'Update Cache Frequency' property when it is set at both table-level and
+     * connection-level.
+     */
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    /**
+     * Helper method that sets up the connections and creates the table to be tested.
+     * @param fullTableName The table's full name
+     * @param tableUpdateCacheFrequency If not '-1', the table-level value to be set for 'Update
+     *                                  Cache Frequency'
+     * @param connUpdateCacheFrequency If not '-1', the connection-level value to be set for 'Update
+     *                                 Cache Frequency'
+     */
+    private static void setUpTableAndConnections(String fullTableName, long tableUpdateCacheFrequency, long connUpdateCacheFrequency) throws SQLException {
+        // Create two connections - a connection that we'll use to create the table and the second
+        // one that we will spy on and use to query the table.
+        Properties props = new Properties();
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        conn1.setAutoCommit(true);
+
+        if (connUpdateCacheFrequency != -1) {
+            props.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB,
+                    String.valueOf(connUpdateCacheFrequency));
+        }
+
+        // use a spied ConnectionQueryServices so we can verify calls to getTable()
+        spyForConn2 = Mockito.spy(driver.getConnectionQueryServices(getUrl(), props));
+        conn2 = spyForConn2.connect(getUrl(), props);
+        conn2.setAutoCommit(true);
+
+        String createTableQuery =
+                "CREATE TABLE " + fullTableName + " (k UNSIGNED_DOUBLE NOT NULL PRIMARY KEY, "
+                        + "v1 UNSIGNED_DOUBLE, v2 UNSIGNED_DOUBLE, v3 UNSIGNED_DOUBLE)";
+
+        if (tableUpdateCacheFrequency != -1) {
+            createTableQuery += " UPDATE_CACHE_FREQUENCY = " + tableUpdateCacheFrequency;
+        }
+
+        // Create the table over first connection
+        conn1.createStatement().execute(createTableQuery);
+        conn1.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES (1, 2, 3, 4)");
+        conn1.commit();
+    }
+
+    /**
+     * Helper method that executes a select query on the table for \p numSelectExecutions times and
+     * verifies that \p numExpectedGetTableCalls were made to getTable() for the table.
+     *
+     * Also resets the spy object for conn2 before returning.
+     *
+     * @param fullTableName The table's full name
+     * @param numSelectExecutions Number of times the select query should be executed on the table
+     * @param numExpectedGetTableCalls Number of expected calls to getTable()
+     */
+    private static void verifyExpectedGetTableCalls(String fullTableName, int numSelectExecutions, int numExpectedGetTableCalls) throws SQLException {
+        String tableName = SchemaUtil.getTableNameFromFullName(fullTableName);
+        String schemaName = SchemaUtil.getSchemaNameFromFullName(fullTableName);
+        String selectFromTableQuery = "SELECT k, v1, v2, v3 FROM " + fullTableName;
+
+        for (int i = 0; i < numSelectExecutions; i++) {
+            // Query the table over the spied connection that has update cache frequency set
+            ResultSet rs = conn2.createStatement().executeQuery(selectFromTableQuery);
+            assertTrue(rs.next());
+        }
+
+        // Ensure that getTable() was called only once for our table
+        verify(spyForConn2, times(numExpectedGetTableCalls)).getTable((PName) isNull(),
+                eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)),
+                anyLong(), anyLong());
+        reset(spyForConn2);
+    }
+}
 
 Review comment:
   Nice test coverage overall!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361564501
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for table-level UCF set, connection-level UCF not set
+    @Test
+    public void testWithTableLevelUCFNoConnLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = 1000;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'tableUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF not set
+    @Test
+    public void testWithNoConnAndTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = -1;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // This is the default behavior (i.e. always fetch the latest metadata of the table) when both connection and
+        // table level properties are not set
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 2;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Test for both connection-level and table-level UCF set
+    @Test
+    public void testWithBothConnAndTableLevelUCF() throws Exception {
+        // Set table level property to a much higher value than the connection level property
+        long tableUpdateCacheFrequency = 5000;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there were no new calls to getTable() as the
+        // table level UCF should come in to effect
+        Thread.sleep(connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 0;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Extend the wait to a period of 'tableUpdateCacheFrequency' and verify that there was one new call to
+        // getTable() for fetching the table's metadata
+        Thread.sleep(tableUpdateCacheFrequency - connUpdateCacheFrequency);
+        numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+    }
+
+    // Helper method that sets up the connections and creates the table
+    private static void setUpTableAndConnections(String fullTableName, long tableUpdateCacheFrequency, long connUpdateCacheFrequency) throws SQLException {
+        // Create two connections - a connection that we'll use to create the table and the second one that we will
+        // spy on and will have 'phoenix.default.update.cache.frequency' set
+        Properties props = new Properties();
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        conn1.setAutoCommit(true);
+
+        Properties propsWithUCF = new Properties();
+        //if (connUpdateCacheFrequency != -1) {
+            propsWithUCF.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, "" + connUpdateCacheFrequency);
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361565280
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
+
+    private static Connection conn1;
+    private static Connection conn2;
+    private static ConnectionQueryServices spyForConn2;
+
+    @AfterClass
+    public static synchronized void freeResources() {
+        try {
+            conn1.close();
+            conn2.close();
+        } catch (Exception e) {
+            /* ignored */
+        }
+    }
+
+    // Test for connection-level UCF set, table-level UCF not set
+    @Test
+    public void testWithConnLevelUCFNoTableLevelUCF() throws Exception {
+        long tableUpdateCacheFrequency = -1;
+        long connUpdateCacheFrequency = 1000;
+        String fullTableName = DEFAULT_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateUniqueName();
+
+        setUpTableAndConnections(fullTableName, tableUpdateCacheFrequency, connUpdateCacheFrequency);
+
+        // There should only be a single call to getTable() for fetching the table's metadata
+        int numSelectExecutions = 2;
+        int numExpectedGetTableCalls = 1;
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
+
+        // Wait for a period of 'connUpdateCacheFrequency' and verify that there was one new call to getTable() for
+        // fetching the table's metadata
+        Thread.sleep(connUpdateCacheFrequency);
+        verifyExpectedGetTableCalls(fullTableName, numSelectExecutions, numExpectedGetTableCalls);
 
 Review comment:
   The call to `verifyExpectedGetTableCalls()` before this block of code will also reset the spy object. This code will verify the number of expected `getTable()` calls after the reset. I'll add this information in the method header for `verifyExpectedGetTableCalls()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r366606457
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+
+            return (table.getRowTimestampColPos() == -1 &&
+                    connection.getMetaDataCache().getAge(tableRef) <
+                            effectiveUpdateCacheFrequency);
+        }
+        return false;
 
 Review comment:
   Cool. Thanks for checking.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361606151
 
 

 ##########
 File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
 ##########
 @@ -722,14 +722,27 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName,
     // 1. table is a system table that does not have a ROW_TIMESTAMP column OR
     // 2. table was already resolved as of that timestamp OR
     // 3. table does not have a ROW_TIMESTAMP column and age is less then UPDATE_CACHE_FREQUENCY
+    // 3a. Get the effective UPDATE_CACHE_FREQUENCY for checking the age in the following precedence order:
+    // Table-level property > Connection-level property > Default value.
     private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimestamp,
             boolean systemTable, PTable table, PTableRef tableRef, long tableResolvedTimestamp) {
-        return table != null && !alwaysHitServer &&
-                (systemTable && table.getRowTimestampColPos() == -1 ||
-                        resolvedTimestamp == tableResolvedTimestamp ||
-                        (table.getRowTimestampColPos() == -1 &&
-                                connection.getMetaDataCache().getAge(tableRef) <
-                                        table.getUpdateCacheFrequency()));
+        if (table != null && !alwaysHitServer) {
+            if (systemTable && table.getRowTimestampColPos() == -1 || resolvedTimestamp == tableResolvedTimestamp) {
+                return true;
+            }
+
+            // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS (= 0)?
+            // We should ideally be checking something like hasUpdateCacheFrequency()
+            long effectiveUpdateCacheFrequency = table.getUpdateCacheFrequency() != 0L ?
+                    table.getUpdateCacheFrequency() :
+                    connection.getQueryServices().getProps().getLong(
+                            QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
 
 Review comment:
   Done, added a debug log line here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah closed pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah closed pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on issue #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#issuecomment-574323990
 
 
   Hi @ckulkarni
   I will be squashing my commits together and force pushing here since quite a few things changed from the time this was last reviewed. I hope that is fine.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361563494
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361563804
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
 
 Review comment:
   I could, but only if you have a strong preference for that. I would prefer to keep these tests in a separate class as I have some private methods and members which won't have a purpose for the methods in `UpdateCacheIT` class.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ChinmaySKulkarni commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361758271
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,173 @@
+package org.apache.phoenix.rpc;
+
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT {
 
 Review comment:
   Ok sure, let's keep it separate in that case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [phoenix] ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…

Posted by GitBox <gi...@apache.org>.
ntshmah commented on a change in pull request #659: PHOENIX-5634: Use 'phoenix.default.update.cache.frequency' from conne…
URL: https://github.com/apache/phoenix/pull/659#discussion_r361751798
 
 

 ##########
 File path: phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java
 ##########
 @@ -0,0 +1,211 @@
+package org.apache.phoenix.rpc;
 
 Review comment:
   Noted, will add that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services