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/27 00:43:31 UTC

[GitHub] [phoenix] ChinmaySKulkarni commented on a change in pull request #935: PHOENIX-6186 - Store last DDL timestamp in System.Catalog

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



##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
##########
@@ -387,6 +388,31 @@ public void testViewUsesTableLocalIndex() throws Exception {
         }
     }
 
+    @Test
+    public void testCreateViewTimestamp() throws Exception {
+        Properties props = new Properties();
+        final String schemaName = "S_" + generateUniqueName();
+        final String tableName = "T_" + generateUniqueName();
+        final String viewName = "V_" + generateUniqueName();
+        final String dataTableFullName = SchemaUtil.getTableName(schemaName, tableName);
+        final String viewFullName = SchemaUtil.getTableName(schemaName, viewName);
+        String tableDDL =
+            "CREATE TABLE " + dataTableFullName + " (\n" + "ID1 VARCHAR(15) NOT NULL,\n"
+                + "ID2 VARCHAR(15) NOT NULL,\n" + "CREATED_DATE DATE,\n"
+                + "CREATION_TIME BIGINT,\n" + "LAST_USED DATE,\n"
+                + "CONSTRAINT PK PRIMARY KEY (ID1, ID2)) ";
+        String viewDDL = "CREATE VIEW " + viewFullName  + " AS SELECT * " +
+            "FROM " + dataTableFullName;
+        long startTS = EnvironmentEdgeManager.currentTimeMillis();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(tableDDL);
+            conn.createStatement().execute(viewDDL);
+            conn.commit();

Review comment:
       nit: Don't need commit() since they are DDL statements.

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
##########
@@ -1216,5 +1217,50 @@ public void testDroppingIndexedColDropsViewIndex() throws Exception {
             assertNull(results.next());
         }
     }
+
+    @Test
+    public void testAddThenDropColumnTableDDLTimestamp() throws Exception {
+        Properties props = new Properties();
+        String schemaName = SCHEMA1;
+        String dataTableName = "T_" + generateUniqueName();
+        String viewName = "V_" + generateUniqueName();
+        String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
+        String viewFullName = SchemaUtil.getTableName(schemaName, viewName);
+
+        String tableDDL = generateDDL("CREATE TABLE IF NOT EXISTS " + dataTableFullName + " ("
+            + " %s ID char(1) NOT NULL,"
+            + " COL1 integer NOT NULL,"
+            + " COL2 bigint NOT NULL,"
+            + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
+            + " ) %s");
+
+        String viewDDL = "CREATE VIEW " + viewFullName + " AS SELECT * FROM " + dataTableFullName;
+
+        String columnAddDDL = "ALTER VIEW " + viewFullName + " ADD COL3 varchar(50) NULL ";
+        String columnDropDDL = "ALTER VIEW " + viewFullName + " DROP COLUMN COL3 ";
+        long startTS = EnvironmentEdgeManager.currentTimeMillis();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(tableDDL);
+            //first get the original DDL timestamp when we created the table
+            long tableDDLTimestamp = CreateTableIT.verifyLastDDLTimestamp(schemaName, dataTableName,
+                dataTableFullName, startTS,
+                conn);
+            Thread.sleep(1);
+            conn.createStatement().execute(viewDDL);
+            tableDDLTimestamp = CreateTableIT.verifyLastDDLTimestamp(schemaName, viewName,
+                viewFullName, tableDDLTimestamp + 1, conn);
+            Thread.sleep(1);
+            //now add a column and make sure the timestamp updates
+            conn.createStatement().execute(columnAddDDL);
+            tableDDLTimestamp = CreateTableIT.verifyLastDDLTimestamp(schemaName, viewName,
+                viewFullName,
+                tableDDLTimestamp + 1, conn);
+            Thread.sleep(1);
+            conn.createStatement().execute(columnDropDDL);
+            CreateTableIT.verifyLastDDLTimestamp(schemaName, viewName,
+                viewFullName,
+                tableDDLTimestamp + 1 , conn);
+        }
+    }

Review comment:
       Once we converge on the expected behavior for column inheritance from a parent to its child views, we should add some tests for those scenarios too.

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2100,6 +2115,13 @@ public void createTable(RpcController controller, CreateTableRequest request,
                     builder.setViewIndexIdType(PLong.INSTANCE.getSqlType());
                 }
                 builder.setMutationTime(currentTimeStamp);
+                //send the newly built table back because we generated the DDL timestamp server
+                // side and the client doesn't have it.
+                PTable newTable = buildTable(tableKey, cacheKey, region,
+                    clientTimeStamp, clientVersion);
+                if (newTable != null) {

Review comment:
       It might be better to just send the DDL timestamp from the server instead of the entire PTable. This is because on the client, we set certain properties which may be derived from the parent, etc. so we probably can't blindly cache the PTable that is returned from the server anyhow. 

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
##########
@@ -4416,10 +4423,15 @@ else if (columnToDrop.isViewReferenced()) {
                     // client-side cache as it would be too painful. Just let it pull it over from
                     // the server when needed.
                     if (tableColumnsToDrop.size() > 0) {
-                        if (removedIndexTableOrColumn)
+                        if (removedIndexTableOrColumn) {
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else
-                            connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
+                        }
+                        else {
+                            //replace the cache of this table with the updated one we got back
+                            // from the server
+                            connection.addTable(result.getTable(),

Review comment:
       Similar kind of concern here?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
##########
@@ -3048,7 +3048,13 @@ public boolean isViewReferenced() {
              * the counter as NULL_COUNTER for extra safety.
              */
             EncodedCQCounter cqCounterToBe = tableType == PTableType.VIEW ? NULL_COUNTER : cqCounter;
-            PTable table = new PTableImpl.Builder()
+            PTable table;
+            //better to use the table sent back from the server so we get an accurate DDL
+            // timestamp, which is server-generated.
+            if (result.getTable() != null ) {

Review comment:
       This seems risky since we are relying on the fact that the PTable returned from the server has all the necessary attributes set as the PTable we create on the client-side. There are some that we set explicitly inside `MetaDataClient` which depend on the parent so I'm not sure we still have those set as expected.
   
   Instead, to be safe we can maybe `getDDLTimestamp()` from this returned PTable and set that in the builder. Better yet, we could just send the DDL timestamp in the server response rather than the entire PTable. We'd also have to add a setter and getter to the PTable builder for this new attribute.

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
##########
@@ -1326,7 +1327,40 @@ public void testAddingColumnsToTablesAndViews() throws Exception {
             assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
         }
     }
-	
+
+    @Test
+    public void testAddThenDropColumnTableDDLTimestamp() throws Exception {
+        Properties props = new Properties();
+        String tableDDL = "CREATE TABLE IF NOT EXISTS " + dataTableFullName + " ("
+            + " ENTITY_ID integer NOT NULL,"
+            + " COL1 integer NOT NULL,"
+            + " COL2 bigint NOT NULL,"
+            + " CONSTRAINT NAME_PK PRIMARY KEY (ENTITY_ID, COL1, COL2)"
+            + " ) " + generateDDLOptions("");
+
+        String columnAddDDL = "ALTER TABLE " + dataTableFullName + " ADD COL3 varchar(50) NULL ";
+        String columnDropDDL = "ALTER TABLE " + dataTableFullName + " DROP COLUMN COL3 ";
+        long startTS = EnvironmentEdgeManager.currentTimeMillis();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(tableDDL);
+            //first get the original DDL timestamp when we created the table
+            long tableDDLTimestamp = CreateTableIT.verifyLastDDLTimestamp(schemaName, dataTableName,
+                dataTableFullName, startTS,
+                conn);
+            Thread.sleep(1);
+            //now add a column and make sure the timestamp updates
+            conn.createStatement().execute(columnAddDDL);
+            tableDDLTimestamp = CreateTableIT.verifyLastDDLTimestamp(schemaName, dataTableName,
+                dataTableFullName,
+                tableDDLTimestamp + 1, conn);
+            Thread.sleep(1);
+            conn.createStatement().execute(columnDropDDL);
+            CreateTableIT.verifyLastDDLTimestamp(schemaName, dataTableName,
+                dataTableFullName,
+                tableDDLTimestamp + 1 , conn);
+        }
+    }

Review comment:
       Can we add a test which confirms that ALTER SET <properties> doesn't modify the timestamp?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
##########
@@ -910,6 +911,48 @@ public void testTableDescriptorPriority() throws SQLException, IOException {
         }
     }
 
+    @Test
+    public void testCreateTableDDLTimestamp() throws Exception {
+        Properties props = new Properties();
+        final String schemaName = generateUniqueName();
+        final String tableName = generateUniqueName();
+        final String dataTableFullName = SchemaUtil.getTableName(schemaName, tableName);
+        String ddl =
+            "CREATE TABLE " + dataTableFullName + " (\n" + "ID1 VARCHAR(15) NOT NULL,\n"
+                + "ID2 VARCHAR(15) NOT NULL,\n" + "CREATED_DATE DATE,\n"
+                + "CREATION_TIME BIGINT,\n" + "LAST_USED DATE,\n"
+                + "CONSTRAINT PK PRIMARY KEY (ID1, ID2)) ";
+        long startTS = EnvironmentEdgeManager.currentTimeMillis();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(ddl);
+            verifyLastDDLTimestamp(schemaName, tableName, dataTableFullName, startTS, conn);
+        }
+    }
+
+    public static long verifyLastDDLTimestamp(String schemaName, String tableName,
+                                              String dataTableFullName, long startTS, Connection conn) throws SQLException {
+        long endTS = EnvironmentEdgeManager.currentTimeMillis();
+        //First try the JDBC metadata API
+        PhoenixDatabaseMetaData metadata = (PhoenixDatabaseMetaData) conn.getMetaData();
+        ResultSet rs = metadata.getTables("", schemaName, tableName, null);
+        assertTrue("No metadata returned", rs.next());
+        Long ddlTimestamp = rs.getLong(PhoenixDatabaseMetaData.LAST_DDL_TIMESTAMP);
+        assertNotNull("JDBC DDL timestamp is null!", ddlTimestamp);
+        assertTrue("JDBC DDL Timestamp not in the right range!",
+            ddlTimestamp >= startTS && ddlTimestamp <= endTS);
+        //Now try the PTable API
+        PTable table = PhoenixRuntime.getTableNoCache(conn, dataTableFullName);

Review comment:
       Do we need some tests to confirm tenant-view behavior? And then subsequently make this helper method work for resolving those?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/AddColumnMutator.java
##########
@@ -399,6 +399,10 @@ public MetaDataMutationResult validateAndAddMetadata(PTable table, byte[][] rowK
                                 rowKeyMetaData[TABLE_NAME_INDEX])));
             }
         }
+        //We're changing the application-facing schema by adding a column, so update the DDL
+        // timestamp
+        additionalTableMetadataMutations.add(MetaDataUtil.getLastDDLTimestampUpdate(tableHeaderRowKey,

Review comment:
       This will get triggered even for `ALTER TABLE/VIEW SET <property>`. I thought we didn't want to update the ddl timestamp in those cases?

##########
File path: pom.xml
##########
@@ -318,7 +318,7 @@
             <execution>
               <id>ParallelStatsDisabledTest</id>
               <configuration>
-                <reuseForks>true</reuseForks>
+                <reuseForks>false</reuseForks>

Review comment:
       What is this change for?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
##########
@@ -107,6 +107,18 @@
             HColumnDescriptor.KEEP_DELETED_CELLS,
             HColumnDescriptor.REPLICATION_SCOPE);
 
+    public static Mutation getLastDDLTimestampUpdate(byte[] tableHeaderRowKey,

Review comment:
       Can we add a simple unit test for this inside `MetaDataUtilTest`?




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