You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/12/03 00:08:28 UTC

phoenix git commit: PHOENIX-2468 Add dynamic column to track asynchronous index creation date

Repository: phoenix
Updated Branches:
  refs/heads/master 0701e06d4 -> 30ef0cc5d


PHOENIX-2468 Add dynamic column to track asynchronous index creation date


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/30ef0cc5
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/30ef0cc5
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/30ef0cc5

Branch: refs/heads/master
Commit: 30ef0cc5d3a86002af45b5fb0a7345c31225cc9b
Parents: 0701e06
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Dec 2 15:08:10 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Dec 2 15:08:29 2015 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/IndexMetadataIT.java  | 40 ++++++++++++++++++++
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  2 +
 .../apache/phoenix/schema/MetaDataClient.java   | 30 ++++++++++++---
 3 files changed, 67 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/30ef0cc5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index e72ad0c..1af15a1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -28,6 +28,7 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
@@ -470,4 +473,41 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+    
+    @Test
+    public void testAsyncCreatedDate() throws Exception {
+        Date d0 = new Date(System.currentTimeMillis());
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        String ddl = "create table test_table (k varchar primary key, v1 varchar, v2 varchar, v3 varchar)";
+        PreparedStatement stmt = conn.prepareStatement(ddl);
+        stmt.execute();
+        
+        ddl = "CREATE INDEX IDX1 ON test_table (v1) ASYNC";
+        stmt = conn.prepareStatement(ddl);
+        stmt.execute();
+        ddl = "CREATE INDEX IDX2 ON test_table (v2) ASYNC";
+        stmt = conn.prepareStatement(ddl);
+        stmt.execute();
+        ddl = "CREATE INDEX IDX3 ON test_table (v3)";
+        stmt = conn.prepareStatement(ddl);
+        stmt.execute();
+        
+        ResultSet rs = conn.createStatement().executeQuery(
+            "select table_name, " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " +
+            "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") " +
+            "where " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " is not null " +
+            "order by " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE
+        );
+        assertTrue(rs.next());
+        assertEquals("IDX1", rs.getString(1));
+        Date d1 = rs.getDate(2);
+        assertTrue(d1.after(d0));
+        assertTrue(rs.next());
+        assertEquals("IDX2", rs.getString(1));
+        Date d2 = rs.getDate(2);
+        assertTrue(d2.after(d1));
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/30ef0cc5/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index d6d1230..940ca52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -304,6 +304,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final String TRANSACTIONAL = "TRANSACTIONAL";
     public static final byte[] TRANSACTIONAL_BYTES = Bytes.toBytes(TRANSACTIONAL);
 
+    public static final String ASYNC_CREATED_DATE = "ASYNC_CREATED_DATE";
+
     private final PhoenixConnection connection;
     private final ResultSet emptyResultSet;
     public static final int MAX_LOCAL_SI_VERSION_DISALLOW = VersionUtil.encodeVersion("0", "98", "8");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/30ef0cc5/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index cabfd04..ccfb1cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
 import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
@@ -80,15 +81,14 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.TRANSACTIONS_ENABLED;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
-import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -182,6 +182,7 @@ import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.stats.PTableStats;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PTimestamp;
@@ -216,6 +217,13 @@ public class MetaDataClient {
     private static final Logger logger = LoggerFactory.getLogger(MetaDataClient.class);
 
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
+    private static final String SET_ASYNC_CREATED_DATE =
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+            TENANT_ID + "," +
+            TABLE_SCHEM + "," +
+            TABLE_NAME + "," +
+            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+            ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
             TENANT_ID + "," +
@@ -827,7 +835,7 @@ public class MetaDataClient {
     }
 
     public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
-        PTable table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null);
+        PTable table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null);
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1172,6 +1180,10 @@ public class MetaDataClient {
             try {
                 ColumnResolver resolver = FromCompiler.getResolver(statement, connection, statement.getUdfParseNodes());
                 tableRef = resolver.getTables().get(0);
+                Date asyncCreatedDate = null;
+                if (statement.isAsync()) {
+                    asyncCreatedDate = new Date(tableRef.getTimeStamp());
+                }
                 PTable dataTable = tableRef.getTable();
                 boolean isTenantConnection = connection.getTenantId() != null;
                 if (isTenantConnection) {
@@ -1346,7 +1358,7 @@ public class MetaDataClient {
                 }
                 PrimaryKeyConstraint pk = FACTORY.primaryKey(null, allPkColumns);
                 CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount());
-                table = createTableInternal(tableStatement, splits, dataTable, null, null, null, null, indexId, statement.getIndexType());
+                table = createTableInternal(tableStatement, splits, dataTable, null, null, null, null, indexId, statement.getIndexType(), asyncCreatedDate);
                 break;
             } catch (ConcurrentTableMutationException e) { // Can happen if parent data table changes while above is in progress
                 if (retry) {
@@ -1526,7 +1538,7 @@ public class MetaDataClient {
         return false;
     }
     
-    private PTable createTableInternal(CreateTableStatement statement, byte[][] splits, final PTable parent, String viewStatement, ViewType viewType, final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId, IndexType indexType) throws SQLException {
+    private PTable createTableInternal(CreateTableStatement statement, byte[][] splits, final PTable parent, String viewStatement, ViewType viewType, final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId, IndexType indexType, Date asyncCreatedDate) throws SQLException {
         final PTableType tableType = statement.getTableType();
         boolean wasAutoCommit = connection.getAutoCommit();
         connection.rollback();
@@ -2099,6 +2111,14 @@ public class MetaDataClient {
             tableUpsert.setBoolean(21, transactional);
             tableUpsert.execute();
 
+            if (asyncCreatedDate != null) {
+                PreparedStatement setAsync = connection.prepareStatement(SET_ASYNC_CREATED_DATE);
+                setAsync.setString(1, tenantIdStr);
+                setAsync.setString(2, schemaName);
+                setAsync.setString(3, tableName);
+                setAsync.setDate(4, asyncCreatedDate);
+                setAsync.execute();
+            }
             tableMetaData.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
             connection.rollback();