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 2014/12/03 20:13:22 UTC

phoenix git commit: PHOENIX-1484 Index creation failed due to specifying DEFAULT_COLUMN_FAMILY option (Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/4.2 e42e8f214 -> 8a6db7e45


PHOENIX-1484 Index creation failed due to specifying
 DEFAULT_COLUMN_FAMILY option (Rajeshbabu)


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

Branch: refs/heads/4.2
Commit: 8a6db7e450ecb30762d11e3f858b492c13f4e31e
Parents: e42e8f2
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Dec 3 11:13:06 2014 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Dec 3 11:13:06 2014 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/LocalIndexIT.java     | 19 ++++++
 .../apache/phoenix/schema/MetaDataClient.java   | 64 +++++++++++++++-----
 .../phoenix/compile/QueryCompilerTest.java      | 19 ++++++
 3 files changed, 88 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a6db7e4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 3986985..4db5279 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -29,6 +29,7 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
@@ -649,6 +650,24 @@ public class LocalIndexIT extends BaseIndexIT {
     }
 
     @Test
+    public void testLocalIndexCreationWithDefaultFamilyOption() throws Exception {
+        Connection conn1 = DriverManager.getConnection(getUrl());
+        try{
+            Statement statement = conn1.createStatement();
+            statement.execute("create table example (id integer not null,fn varchar,"
+                    + "ln varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
+            statement.execute("upsert into example values(1,'fn','ln')");
+            statement
+                    .execute("create local index my_idx on example (fn)");
+            statement.execute("upsert into example values(2,'fn1','ln1')");
+            ResultSet rs = statement.executeQuery("SELECT COUNT(*) FROM my_idx");
+            assertTrue(rs.next());
+       } finally {
+            conn1.close();
+        }
+    }
+
+    @Test
     public void testLocalIndexScanAfterRegionSplit() throws Exception {
         createBaseTable(DATA_TABLE_NAME, null, "('e','j','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a6db7e4/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 0085470..af699e4 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
@@ -66,6 +66,7 @@ import static org.apache.phoenix.schema.PDataType.VARCHAR;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.ParameterMetaData;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -97,11 +98,13 @@ import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.ExplainPlan;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.MutationPlan;
 import org.apache.phoenix.compile.PostDDLCompiler;
 import org.apache.phoenix.compile.PostIndexDDLCompiler;
 import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
@@ -113,6 +116,7 @@ import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AlterIndexStatement;
@@ -724,6 +728,7 @@ public class MetaDataClient {
         try {
             connection.setAutoCommit(true);
             MutationState state;
+            MutationPlan mutationPlan;
             
             // For local indexes, we optimize the initial index population by *not* sending Puts over
             // the wire for the index rows, as we don't need to do that. Instead, we tap into our
@@ -732,7 +737,7 @@ public class MetaDataClient {
                 final PhoenixStatement statement = new PhoenixStatement(connection);
                 String tableName = getFullTableName(dataTableRef);
                 String query = "SELECT count(*) FROM " + tableName;
-                QueryPlan plan = statement.compileQuery(query);
+                final QueryPlan plan = statement.compileQuery(query);
                 TableRef tableRef = plan.getContext().getResolver().getTables().get(0);
                 // Set attribute on scan that UngroupedAggregateRegionObserver will switch on.
                 // We'll detect that this attribute was set the server-side and write the index
@@ -775,24 +780,54 @@ public class MetaDataClient {
                 for (ColumnReference columnRef : indexMaintainer.getAllColumns()) {
                     scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
                 }
-                Cell kv = plan.iterator().next().getValue(0);
-                ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
-                // A single Cell will be returned with the count(*) - we decode that here
-                long rowCount = PDataType.LONG.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
-                // The contract is to return a MutationState that contains the number of rows modified. In this
-                // case, it's the number of rows in the data table which corresponds to the number of index
-                // rows that were added.
-                state = new MutationState(0, connection, rowCount);
+                
+                // Go through MutationPlan abstraction so that we can create local indexes
+                // with a connectionless connection (which makes testing easier).
+                mutationPlan = new MutationPlan() {
+
+                    @Override
+                    public StatementContext getContext() {
+                        return plan.getContext();
+                    }
+
+                    @Override
+                    public ParameterMetaData getParameterMetaData() {
+                        return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                    }
+
+                    @Override
+                    public ExplainPlan getExplainPlan() throws SQLException {
+                        return ExplainPlan.EMPTY_PLAN;
+                    }
+
+                    @Override
+                    public PhoenixConnection getConnection() {
+                        return connection;
+                    }
+
+                    @Override
+                    public MutationState execute() throws SQLException {
+                        Cell kv = plan.iterator().next().getValue(0);
+                        ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+                        // A single Cell will be returned with the count(*) - we decode that here
+                        long rowCount = PDataType.LONG.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
+                        // The contract is to return a MutationState that contains the number of rows modified. In this
+                        // case, it's the number of rows in the data table which corresponds to the number of index
+                        // rows that were added.
+                        return new MutationState(0, connection, rowCount);
+                    }
+                    
+                };
             } else {
                 PostIndexDDLCompiler compiler = new PostIndexDDLCompiler(connection, dataTableRef);
-                MutationPlan plan = compiler.compile(index);
+                mutationPlan = compiler.compile(index);
                 try {
-                    plan.getContext().setScanTimeRange(new TimeRange(dataTableRef.getLowerBoundTimeStamp(), Long.MAX_VALUE));
+                    mutationPlan.getContext().setScanTimeRange(new TimeRange(dataTableRef.getLowerBoundTimeStamp(), Long.MAX_VALUE));
                 } catch (IOException e) {
                     throw new SQLException(e);
                 }
-                state = connection.getQueryServices().updateData(plan);
             }            
+            state = connection.getQueryServices().updateData(mutationPlan);
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null, 
                 TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
                 dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
@@ -1203,7 +1238,7 @@ public class MetaDataClient {
             
             boolean removedProp = false;
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-            if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
+            if ((tableType != PTableType.INDEX || indexId != null) && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.remove(PhoenixDatabaseMetaData.MULTI_TENANT);
                 multiTenant = Boolean.TRUE.equals(multiTenantProp);
                 // Remove, but add back after our check below
@@ -1218,7 +1253,8 @@ public class MetaDataClient {
             }
             // Delay this check as it is supported to have IMMUTABLE_ROWS and SALT_BUCKETS defined on views
             if ((statement.getTableType() == PTableType.VIEW || indexId != null) && !tableProps.isEmpty()) {
-                throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build().buildException();
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
+                        .buildException();
             }
             if (removedProp) {
                 tableProps.put(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME, defaultFamilyName);  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a6db7e4/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 3857736..bb864d9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1452,4 +1452,23 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             assertFalse("Did not expected to find GROUP BY limit optimization in: " + query, QueryUtil.getExplainPlan(rs).contains(" LIMIT 3 GROUPS"));
         }
     }
+    
+    @Test
+    public void testLocalIndexCreationWithDefaultFamilyOption() throws Exception {
+        Connection conn1 = DriverManager.getConnection(getUrl());
+        try{
+            Statement statement = conn1.createStatement();
+            statement.execute("create table example (id integer not null,fn varchar,"
+                    + "ln varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
+            try {
+                statement.execute("create local index my_idx on example (fn) DEFAULT_COLUMN_FAMILY='F'");
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.VIEW_WITH_PROPERTIES.getErrorCode(),e.getErrorCode());
+            }
+            statement.execute("create local index my_idx on example (fn)");
+       } finally {
+            conn1.close();
+        }
+    }
+
 }