You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/04 22:13:46 UTC

[19/50] [abbrv] phoenix git commit: PHOENIX-3424 Backward compatibility failure: 4.8 -> 4.9 upgrade

PHOENIX-3424 Backward compatibility failure: 4.8 -> 4.9 upgrade


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

Branch: refs/heads/encodecolumns2
Commit: 3c804320c9a64f01aedc6e73fc3df75938ba30e2
Parents: 6ef3a3f
Author: James Taylor <ja...@apache.org>
Authored: Sat Oct 29 17:32:29 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Sat Oct 29 17:34:34 2016 -0700

----------------------------------------------------------------------
 .../phoenix/coprocessor/MetaDataProtocol.java   |  2 +-
 .../query/ConnectionQueryServicesImpl.java      | 69 +++++++-------------
 .../org/apache/phoenix/util/SchemaUtil.java     | 15 +++++
 3 files changed, 41 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3c804320/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 4f0a34c..83290db 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -84,7 +84,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 = MIN_TABLE_TIMESTAMP + 15;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 = MIN_TABLE_TIMESTAMP + 18;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_8_1 = MIN_TABLE_TIMESTAMP + 18;
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0 = MIN_TABLE_TIMESTAMP + 19;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0 = MIN_TABLE_TIMESTAMP + 20;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0;
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3c804320/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 1773175..62ee2bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -25,7 +25,6 @@ import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MAJOR_VERS
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MINOR_VERSION;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_PATCH_NUMBER;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.getVersion;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
@@ -38,10 +37,8 @@ import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
 import java.lang.ref.WeakReference;
-import java.sql.PreparedStatement;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
-import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -197,6 +194,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.JDBCUtil;
+import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixContextExecutor;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -2273,46 +2271,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     }
 
-    private PhoenixConnection removeNotNullConstraint(PhoenixConnection oldMetaConnection, String schemaName, String tableName, long timestamp, String columnName) throws SQLException {
-        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-        // Cannot go through DriverManager or you end up in an infinite loop because it'll call init again
-        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-        SQLException sqlE = null;
-        try {
-            String dml = "UPSERT INTO " + SYSTEM_CATALOG_NAME + " (" + PhoenixDatabaseMetaData.TENANT_ID + ","
-                    + PhoenixDatabaseMetaData.TABLE_SCHEM + "," + PhoenixDatabaseMetaData.TABLE_NAME + ","
-                    + PhoenixDatabaseMetaData.COLUMN_NAME + "," + PhoenixDatabaseMetaData.COLUMN_FAMILY + ","
-                    + PhoenixDatabaseMetaData.NULLABLE + ") VALUES (" + "?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = metaConnection.prepareStatement(dml);
-            stmt.setNull(1, Types.VARCHAR);
-            stmt.setString(2, schemaName);
-            stmt.setString(3, tableName);
-            stmt.setString(4, columnName);
-            stmt.setString(5, QueryConstants.DEFAULT_COLUMN_FAMILY);
-            stmt.setInt(6, ResultSetMetaData.columnNullable);
-            stmt.executeUpdate();
-            metaConnection.commit();
-        } catch (NewerTableAlreadyExistsException e) {
-            logger.warn("Table already modified at this timestamp, so assuming column already nullable: " + columnName);
-        } catch (SQLException e) {
-            logger.warn("Add column failed due to:" + e);
-            sqlE = e;
-        } finally {
-            try {
-                oldMetaConnection.close();
-            } catch (SQLException e) {
-                if (sqlE != null) {
-                    sqlE.setNextException(e);
-                } else {
-                    sqlE = e;
-                }
-            }
-            if (sqlE != null) {
-                throw sqlE;
-            }
+    private void removeNotNullConstraint(String schemaName, String tableName, long timestamp, String columnName) throws SQLException {
+        try (HTableInterface htable = this.getTable(SYSTEM_CATALOG_NAME_BYTES)) {
+            byte[] tableRowKey = SchemaUtil.getTableKey(null, schemaName, tableName);
+            Put tableHeader = new Put(tableRowKey);
+            tableHeader.add(KeyValueUtil.newKeyValue(tableRowKey, 
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
+                    QueryConstants.EMPTY_COLUMN_BYTES, 
+                    timestamp, 
+                    QueryConstants.EMPTY_COLUMN_VALUE_BYTES));
+            byte[] columnRowKey = SchemaUtil.getColumnKey(null, schemaName, tableName, columnName, null);
+            Put tableColumn = new Put(columnRowKey);
+            tableColumn.add(KeyValueUtil.newKeyValue(columnRowKey,
+                    QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                    PhoenixDatabaseMetaData.NULLABLE_BYTES,
+                    timestamp,
+                    PInteger.INSTANCE.toBytes(ResultSetMetaData.columnNullable)));
+            List<Mutation> mutations = Lists.<Mutation>newArrayList(tableHeader, tableColumn);
+            htable.batch(mutations, new Object[mutations.size()]);
+        } catch (IOException e) {
+            throw new SQLException(e);
+        } catch (InterruptedException e) {
+            Thread.currentThread().isInterrupted();
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION).setRootCause(e).build().buildException();
         }
-        return metaConnection;
     }
     /**
      * This closes the passed connection.
@@ -2766,8 +2748,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0) {
                     // The COLUMN_FAMILY column should be nullable as we create a row in it without
                     // any column family to mark when guideposts were last collected.
-                    metaConnection = removeNotNullConstraint(metaConnection,
-                            PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
+                    removeNotNullConstraint(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
                             PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE,
                             MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0,
                             PhoenixDatabaseMetaData.COLUMN_FAMILY);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3c804320/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 5fc7564..003fd73 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -262,6 +262,21 @@ public class SchemaUtil {
         return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
     }
 
+    public static byte[] getColumnKey(String tenantId, String schemaName, String tableName, String columnName, String familyName) {
+        Preconditions.checkNotNull(columnName,"Column name cannot be null");
+        if (familyName == null) {
+            return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
+                    QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
+                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName));
+        }
+        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
+                QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
+                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName),
+                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(familyName));
+    }
+
     public static String getTableName(String schemaName, String tableName) {
         return getName(schemaName,tableName, false);
     }