You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by "palashc (via GitHub)" <gi...@apache.org> on 2023/05/30 22:28:13 UTC

[GitHub] [phoenix] palashc opened a new pull request, #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

palashc opened a new pull request, #1612:
URL: https://github.com/apache/phoenix/pull/1612

   (no comment)


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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] shahrs87 commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "shahrs87 (via GitHub)" <gi...@apache.org>.
shahrs87 commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1211968504


##########
phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java:
##########
@@ -2857,23 +2859,32 @@ public static void updateViewIndexIdColumnDataTypeFromShortToLong(
         }
     }
 
-    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's DDL timestamp equal to its
+    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's/view's DDL timestamp equal to its
     // last updated row timestamp.
-    public static void bootstrapLastDDLTimestamp(Connection metaConnection) throws SQLException  {
+    public static void bootstrapLastDDLTimestampForTablesAndViews(Connection metaConnection) throws SQLException  {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.TABLE.getSerializedValue(), PTableType.VIEW.getSerializedValue()});
+    }
+
+    //When upgrading to Phoenix 5.2, make each existing index's DDL timestamp equal to its last updated row timestamp.
+    public static void bootstrapLastDDLTimestampForIndexes(Connection metaConnection) throws SQLException {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.INDEX.getSerializedValue()});
+    }
+
+    private static void bootstrapLastDDLTimestamp(Connection metaConnection, String[] tableTypes) throws SQLException {

Review Comment:
   Add a javadoc for this method.



##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);
+
+            // bootstrap last ddl timestamp for indexes
+            UpgradeUtil.bootstrapLastDDLTimestampForIndexes(conn.unwrap(PhoenixConnection.class));
+            actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
+            assertEquals(indexTS, actualIndexTS);

Review Comment:
   Maybe I am missing something. 
   `indexTS` is the timestamp after we set LAST_DDL_TIMESTAMP to null via `nullDDLTimestamps` method. So it should be NULL.
   `actualIndexTS` is the timestamp after we call `UpgradeUtil.bootstrapLastDDLTimestampForIndexes`. So it should the LAST_ROW_TIMESTAMP() for this index.
   So how are they supposed to be equal? 



##########
phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java:
##########
@@ -4122,6 +4122,7 @@ protected PhoenixConnection upgradeSystemCatalogIfRequired(PhoenixConnection met
             metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                 MIN_SYSTEM_TABLE_TIMESTAMP_5_2_0,
                 PhoenixDatabaseMetaData.STREAMING_TOPIC_NAME + " " + PVarchar.INSTANCE.getSqlTypeName());
+            UpgradeUtil.bootstrapLastDDLTimestampForIndexes(metaConnection);

Review Comment:
   I don't know how this UpgradeUtil code works and when it is executed? If it is executed after the upgrade then the server version would be equal to MIN_SYSTEM_TABLE_TIMESTAMP_5_2_0. But adding DDL timestamps to table/view happened in 4.16.0 version and there also we have similar condition `currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP_4_16_0` so this code clearly works.



##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);

Review Comment:
   Can we assert that indexTS is not 0L? Also we can capture the start time of the test and assert that indexTS > start time of the test. 



##########
phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java:
##########
@@ -2857,23 +2859,32 @@ public static void updateViewIndexIdColumnDataTypeFromShortToLong(
         }
     }
 
-    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's DDL timestamp equal to its
+    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's/view's DDL timestamp equal to its
     // last updated row timestamp.
-    public static void bootstrapLastDDLTimestamp(Connection metaConnection) throws SQLException  {
+    public static void bootstrapLastDDLTimestampForTablesAndViews(Connection metaConnection) throws SQLException  {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.TABLE.getSerializedValue(), PTableType.VIEW.getSerializedValue()});
+    }
+
+    //When upgrading to Phoenix 5.2, make each existing index's DDL timestamp equal to its last updated row timestamp.
+    public static void bootstrapLastDDLTimestampForIndexes(Connection metaConnection) throws SQLException {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.INDEX.getSerializedValue()});
+    }
+
+    private static void bootstrapLastDDLTimestamp(Connection metaConnection, String[] tableTypes) throws SQLException {
+        String tableTypesString = Stream.of(tableTypes).collect(Collectors.joining("','", "'", "'")).toString();

Review Comment:
   learned something new on how to use prefix and suffix along with delimiter. Thank you.



##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);

Review Comment:
   should actualIndexTS be NULL or 0? I think it should be null.



##########
phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java:
##########
@@ -2857,23 +2859,32 @@ public static void updateViewIndexIdColumnDataTypeFromShortToLong(
         }
     }
 
-    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's DDL timestamp equal to its
+    //When upgrading to Phoenix 4.16 or 5.1, make each existing table's/view's DDL timestamp equal to its
     // last updated row timestamp.
-    public static void bootstrapLastDDLTimestamp(Connection metaConnection) throws SQLException  {
+    public static void bootstrapLastDDLTimestampForTablesAndViews(Connection metaConnection) throws SQLException  {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.TABLE.getSerializedValue(), PTableType.VIEW.getSerializedValue()});
+    }
+
+    //When upgrading to Phoenix 5.2, make each existing index's DDL timestamp equal to its last updated row timestamp.
+    public static void bootstrapLastDDLTimestampForIndexes(Connection metaConnection) throws SQLException {
+        bootstrapLastDDLTimestamp(metaConnection, new String[]{PTableType.INDEX.getSerializedValue()});
+    }
+
+    private static void bootstrapLastDDLTimestamp(Connection metaConnection, String[] tableTypes) throws SQLException {
+        String tableTypesString = Stream.of(tableTypes).collect(Collectors.joining("','", "'", "'")).toString();
         String pkCols = TENANT_ID + ", " + TABLE_SCHEM +
-            ", " + TABLE_NAME + ", " + COLUMN_NAME + ", " + COLUMN_FAMILY;
+                ", " + TABLE_NAME + ", " + COLUMN_NAME + ", " + COLUMN_FAMILY;
         final String upsertSql =
-            "UPSERT INTO " + SYSTEM_CATALOG_NAME + " (" + pkCols + ", " +
-        LAST_DDL_TIMESTAMP + ")" + " " +
-            "SELECT " + pkCols + ", PHOENIX_ROW_TIMESTAMP() FROM " + SYSTEM_CATALOG_NAME + " " +
-                "WHERE " + TABLE_TYPE + " " + " in " + "('" + PTableType.TABLE.getSerializedValue()
-                + "', '" + PTableType.VIEW.getSerializedValue() + "')";
-        LOGGER.info("Setting DDL timestamps for tables and views to row timestamps");
+                "UPSERT INTO " + SYSTEM_CATALOG_NAME + " (" + pkCols + ", " +
+                        LAST_DDL_TIMESTAMP + ")" + " " +
+                        "SELECT " + pkCols + ", PHOENIX_ROW_TIMESTAMP() FROM " + SYSTEM_CATALOG_NAME + " " +
+                        "WHERE " + TABLE_TYPE + " " + " in " + "(" + tableTypesString + ")";
+        LOGGER.info(String.format("Setting DDL timestamps for table_type=%s to row timestamps", tableTypesString));
         try (PreparedStatement stmt = metaConnection.prepareStatement(upsertSql)) {
             stmt.execute();
             metaConnection.commit();
         }
-        LOGGER.info("Setting DDL timestamps for tables and views is complete");
+        LOGGER.info(String.format("Setting DDL timestamps for table_type=%s is complete", tableTypesString));

Review Comment:
   ```suggestion
           LOGGER.info("Setting DDL timestamps for table_type={} is complete", tableTypesString);
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] tkhurana merged pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "tkhurana (via GitHub)" <gi...@apache.org>.
tkhurana merged PR #1612:
URL: https://github.com/apache/phoenix/pull/1612


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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] palashc commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "palashc (via GitHub)" <gi...@apache.org>.
palashc commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1212071303


##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);

Review Comment:
   It is returned using a `rs.getLong()` in `getLastTimestampForMetadata` call which returns 0 if the actual value was sql null. 



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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] shahrs87 commented on pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "shahrs87 (via GitHub)" <gi...@apache.org>.
shahrs87 commented on PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#issuecomment-1570676613

   I don't have commit rights, @tkhurana  can you please review this patch too? Thank you.


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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] palashc commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "palashc (via GitHub)" <gi...@apache.org>.
palashc commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1212079283


##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);
+
+            // bootstrap last ddl timestamp for indexes
+            UpgradeUtil.bootstrapLastDDLTimestampForIndexes(conn.unwrap(PhoenixConnection.class));
+            actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
+            assertEquals(indexTS, actualIndexTS);

Review Comment:
   @shahrs87 `indexTS` is the PHOENIX_ROW_TIMESTAMP() of the header row of the index in syscat (it is the timestamp of the empty column) and indicates the last time the row was modified. After we call `UpgradeUtil.bootstrapLastDDLTimestampForIndexes`, this same value for the row is used to populate the `LAST_DDL_TIMESTAMP` column which is captured in `actualIndexTS`. 



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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] shahrs87 commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "shahrs87 (via GitHub)" <gi...@apache.org>.
shahrs87 commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1212092105


##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -683,10 +683,12 @@ private void verifyExpectedCellValue(byte[] rowKey, byte[] syscatBytes,
 
     @Test
     public void testLastDDLTimestampBootstrap() throws Exception {
+        Long testStartTime = System.currentTimeMillis();

Review Comment:
   replace this with `EnvironmentEdgeManager.currentTimeMillis`



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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] shahrs87 commented on pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "shahrs87 (via GitHub)" <gi...@apache.org>.
shahrs87 commented on PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#issuecomment-1570675876

   In the last jenkins run, there were around 16 checkstyle errors. If those are related to you patch please fix it otherwise +1 ltgm.


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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] shahrs87 commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "shahrs87 (via GitHub)" <gi...@apache.org>.
shahrs87 commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1212088186


##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);
+
+            // bootstrap last ddl timestamp for indexes
+            UpgradeUtil.bootstrapLastDDLTimestampForIndexes(conn.unwrap(PhoenixConnection.class));
+            actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
+            assertEquals(indexTS, actualIndexTS);

Review Comment:
   Oh my bad, I thought indexTS is calculated from `getLastTimestampForMetadata` 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.

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [phoenix] palashc commented on a diff in pull request #1612: PHOENIX-6963 : Bootstrap last ddl timestamp for indexes

Posted by "palashc (via GitHub)" <gi...@apache.org>.
palashc commented on code in PR #1612:
URL: https://github.com/apache/phoenix/pull/1612#discussion_r1212071303


##########
phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java:
##########
@@ -697,21 +698,33 @@ public void testLastDDLTimestampBootstrap() throws Exception {
             conn.createStatement().execute(
                 "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
 
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (KV1) ASYNC");
+
             //Now we null out any existing last ddl timestamps
             nullDDLTimestamps(conn);
 
             //now get the row timestamps for each header row
             long tableTS = getRowTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long viewTS = getRowTimestampForMetadata(conn, schemaName, viewName, PTableType.VIEW);
+            long indexTS = getRowTimestampForMetadata(conn, schemaName, indexName, PTableType.INDEX);
 
-            UpgradeUtil.bootstrapLastDDLTimestamp(conn.unwrap(PhoenixConnection.class));
+            // bootstrap last ddl timestamp for tables and views
+            UpgradeUtil.bootstrapLastDDLTimestampForTablesAndViews(conn.unwrap(PhoenixConnection.class));
             long actualTableTS = getLastTimestampForMetadata(conn, schemaName, tableName,
                 PTableType.TABLE);
             long actualViewTS = getLastTimestampForMetadata(conn, schemaName, viewName,
                 PTableType.VIEW);
+            long actualIndexTS = getLastTimestampForMetadata(conn, schemaName, indexName,
+                    PTableType.INDEX);
             assertEquals(tableTS, actualTableTS);
             assertEquals(viewTS, actualViewTS);
+            assertEquals(0L, actualIndexTS);

Review Comment:
   It is returned using a `rs.getLong()` call which returns 0 if the actual value was sql null. 



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

To unsubscribe, e-mail: issues-unsubscribe@phoenix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org