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 2020/10/21 18:15:29 UTC

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #928: PHOENIX-6192 : Use tenant connection to resolve tenant views in syncUpdateCacheFreqAllIndexes()

ChinmaySKulkarni commented on a change in pull request #928:
URL: https://github.com/apache/phoenix/pull/928#discussion_r509507594



##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);

Review comment:
       nit: Not sure we need to assign `rs.get..()` to a variable each time for `tenantId`, `ucf`, etc. Up to you if you think it's good for clarity (I don't feel strongly about it).

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {

Review comment:
       Just wondering, can these tests can't be added to `GlobalConnectionTenantTableIT` itself? 

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
##########
@@ -1400,6 +1393,43 @@ public static void syncUpdateCacheFreqAllIndexes(PhoenixConnection conn, PTable
         }
     }
 
+    private static void iterateOverChildViewAndSyncCacheFreq(
+            final PhoenixConnection newConn,
+            final PreparedStatement stmt, final TableInfo tableInfo)
+            throws SQLException {
+        String viewName = SchemaUtil.getTableName(tableInfo.getSchemaName(),
+            tableInfo.getTableName());
+        String viewTenantId = Bytes.toString(tableInfo.getTenantId());

Review comment:
       It will be better to resolve the PTable for the view (either using tenanted connection if required or a global connection if a global view) and then call `syncUpdateCacheFreqForIndexesOfTable()` just once. Basically, combine this method and `getViewAndSyncCacheFreqForIndexes()`.

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
##########
@@ -1400,6 +1393,43 @@ public static void syncUpdateCacheFreqAllIndexes(PhoenixConnection conn, PTable
         }
     }
 
+    private static void iterateOverChildViewAndSyncCacheFreq(
+            final PhoenixConnection newConn,
+            final PreparedStatement stmt, final TableInfo tableInfo)
+            throws SQLException {
+        String viewName = SchemaUtil.getTableName(tableInfo.getSchemaName(),
+            tableInfo.getTableName());
+        String viewTenantId = Bytes.toString(tableInfo.getTenantId());
+        if (StringUtils.isNotEmpty(viewTenantId)) {
+            Properties props = new Properties(newConn.getClientInfo());
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, viewTenantId);
+            // use tenant connection to resolve tenant views
+            try (PhoenixConnection tenantConn =
+                    new PhoenixConnection(newConn, props)) {
+                getViewAndSyncCacheFreqForIndexes(stmt, viewName,
+                    viewTenantId, tenantConn);
+            }
+        } else {
+            getViewAndSyncCacheFreqForIndexes(stmt, viewName,
+                viewTenantId, newConn);
+        }
+    }
+
+    private static void getViewAndSyncCacheFreqForIndexes(
+            final PreparedStatement stmt, final String viewName,
+            final String viewTenantId, final PhoenixConnection conn)
+            throws SQLException {
+        final PTable view;
+        try {
+            view = PhoenixRuntime.getTable(conn, viewName);
+        } catch (TableNotFoundException e) {
+            // Ignore
+            LOGGER.warn("Error getting PTable for view: {}", viewName);

Review comment:
       This should be `ERROR` level

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {

Review comment:
       In case the answer is no for some reason, can we rename this test class to be more representative of the actual test i.e. all tests are related to syncUpdateCacheFreq so maybe rename it to show that

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, null, TABLE_NAME, 500);
+            updateCacheFreq(conn, null, TABLE_INDEX, 400);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_INDEX + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(400, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_NAME, 999);
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_INDEX_NAME, 888);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(888, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            // clear the server-side cache to get the latest built PTables
+            conn.unwrap(PhoenixConnection.class).getQueryServices().clearCache();
+            PhoenixConnection pcon = conn.unwrap(PhoenixConnection.class);
+            pcon.setRunningUpgrade(true);
+
+            UpgradeUtil.syncUpdateCacheFreqAllIndexes(pcon,
+                PhoenixRuntime.getTableNoCache(conn, SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME)));
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM"
+                    + " SYSTEM.CATALOG WHERE TABLE_NAME='" + TABLE_NAME + "'");
+            rs.next();
+            String tenantId = rs.getString(1);
+            String schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"

Review comment:
       Extract this query into a `private final String` member variable to avoid duplication

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, null, TABLE_NAME, 500);
+            updateCacheFreq(conn, null, TABLE_INDEX, 400);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_INDEX + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(400, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_NAME, 999);
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_INDEX_NAME, 888);
+
+            rs = conn.createStatement().executeQuery(

Review comment:
       nit: these 4-5 steps are repeated multiple times so maybe worth extracting to a small helper method..something like `assertUpdateCacheFreqValue()`)

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, null, TABLE_NAME, 500);
+            updateCacheFreq(conn, null, TABLE_INDEX, 400);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_INDEX + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(400, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_NAME, 999);
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_INDEX_NAME, 888);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(888, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            // clear the server-side cache to get the latest built PTables
+            conn.unwrap(PhoenixConnection.class).getQueryServices().clearCache();
+            PhoenixConnection pcon = conn.unwrap(PhoenixConnection.class);
+            pcon.setRunningUpgrade(true);
+
+            UpgradeUtil.syncUpdateCacheFreqAllIndexes(pcon,
+                PhoenixRuntime.getTableNoCache(conn, SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME)));
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM"
+                    + " SYSTEM.CATALOG WHERE TABLE_NAME='" + TABLE_NAME + "'");
+            rs.next();
+            String tenantId = rs.getString(1);
+            String schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + TABLE_INDEX + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME, tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW2_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+        }
+    }
+
+    private void updateCacheFreq(Connection conn, String tenantId,

Review comment:
       nit: Maybe this method should be called `updateUpdateCacheFreq()` ;)

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, null, TABLE_NAME, 500);
+            updateCacheFreq(conn, null, TABLE_INDEX, 400);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_INDEX + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(400, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_NAME, 999);
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_INDEX_NAME, 888);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(888, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            // clear the server-side cache to get the latest built PTables
+            conn.unwrap(PhoenixConnection.class).getQueryServices().clearCache();
+            PhoenixConnection pcon = conn.unwrap(PhoenixConnection.class);
+            pcon.setRunningUpgrade(true);
+
+            UpgradeUtil.syncUpdateCacheFreqAllIndexes(pcon,
+                PhoenixRuntime.getTableNoCache(conn, SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME)));
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM"
+                    + " SYSTEM.CATALOG WHERE TABLE_NAME='" + TABLE_NAME + "'");
+            rs.next();
+            String tenantId = rs.getString(1);
+            String schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + TABLE_INDEX + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME, tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);

Review comment:
       Shouldn't this be 999 since the view it is an index of has 999 set?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);

Review comment:
       Would be better to try out a couple of different TENANT_IDs to make sure that `syncUpdateCacheFreqAllIndexes()` changes the tenant_id to use as per each view that is found

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
##########
@@ -1400,6 +1393,43 @@ public static void syncUpdateCacheFreqAllIndexes(PhoenixConnection conn, PTable
         }
     }
 
+    private static void iterateOverChildViewAndSyncCacheFreq(

Review comment:
       Rename this method since we aren't iterating over anything. Maybe after combining this with `getViewAndSyncCacheFreqForIndexes()` we can just call it the latter.

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/GlobalConnectionTenantTable2IT.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createBaseTable;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createView;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.createViewIndex;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getConnection;
+import static org.apache.phoenix.end2end.GlobalConnectionTenantTableIT.getTenantConnection;
+import static org.apache.phoenix.util.UpgradeUtil.UPSERT_UPDATE_CACHE_FREQUENCY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class GlobalConnectionTenantTable2IT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "SCHEMA2";
+    private static final String TABLE_NAME = generateUniqueName();
+    private static final String TENANT_NAME = "TENANT2";
+    private static final String VIEW1_NAME = "VIEW1";
+    private static final String VIEW1_INDEX_NAME = "INDEX1";
+    private static final String VIEW_INDEX_COL = "v2";
+    public static final String TABLE_INDEX = "TABLE_INDEX";
+    public static final String VIEW2_NAME = "VIEW2";
+    public static final String VIEW2_INDEX_NAME = "INDEX2";
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        createBaseTable(SCHEMA_NAME, TABLE_NAME, true, null, null);
+        createViewIndex(getConnection(), SCHEMA_NAME, TABLE_INDEX, TABLE_NAME, VIEW_INDEX_COL);
+        try (Connection conn = getTenantConnection(TENANT_NAME)) {
+            createView(conn, SCHEMA_NAME, VIEW1_NAME, TABLE_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW1_INDEX_NAME, VIEW1_NAME, VIEW_INDEX_COL);
+            createView(conn, SCHEMA_NAME, VIEW2_NAME, VIEW1_NAME);
+            createViewIndex(conn, SCHEMA_NAME, VIEW2_INDEX_NAME, VIEW2_NAME, VIEW_INDEX_COL);
+        }
+    }
+
+    @Test
+    public void testSyncCacheFreqWithTenantView() throws Exception {
+        try (Connection conn = getConnection()) {
+            ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_NAME + "'");
+            rs.next();
+            long cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, null, TABLE_NAME, 500);
+            updateCacheFreq(conn, null, TABLE_INDEX, 400);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + TABLE_INDEX + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(400, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG "
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_NAME, 999);
+            updateCacheFreq(conn, TENANT_NAME, VIEW1_INDEX_NAME, 888);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(888, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG WHERE TABLE_NAME='"
+                    + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            cacheFreq = rs.getLong(1);
+            assertEquals(0, cacheFreq);
+
+            // clear the server-side cache to get the latest built PTables
+            conn.unwrap(PhoenixConnection.class).getQueryServices().clearCache();
+            PhoenixConnection pcon = conn.unwrap(PhoenixConnection.class);
+            pcon.setRunningUpgrade(true);
+
+            UpgradeUtil.syncUpdateCacheFreqAllIndexes(pcon,
+                PhoenixRuntime.getTableNoCache(conn, SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME)));
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM"
+                    + " SYSTEM.CATALOG WHERE TABLE_NAME='" + TABLE_NAME + "'");
+            rs.next();
+            String tenantId = rs.getString(1);
+            String schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + TABLE_INDEX + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertNull(tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME, tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(999, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW1_INDEX_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW2_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(0, cacheFreq);
+
+            rs = conn.createStatement().executeQuery(
+                "SELECT TENANT_ID,TABLE_SCHEM,UPDATE_CACHE_FREQUENCY FROM SYSTEM.CATALOG"
+                    + " WHERE TABLE_NAME='" + VIEW2_INDEX_NAME + "'");
+            rs.next();
+            tenantId = rs.getString(1);
+            schemaName = rs.getString(2);
+            cacheFreq = rs.getLong(3);
+            assertEquals(TENANT_NAME,tenantId);
+            assertEquals(SCHEMA_NAME, schemaName);
+            assertEquals(500, cacheFreq);

Review comment:
       Similarly, shouldn't this be 0 since VIEW2 has 0 set?




----------------------------------------------------------------
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