You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2017/05/31 01:52:36 UTC

phoenix git commit: PHOENIX-3897 Unspecified columns in upsert are writing as nulls in Phoenix-Caclite(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/calcite 39ca90e54 -> 48332b223


PHOENIX-3897 Unspecified columns in upsert are writing as nulls in Phoenix-Caclite(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 48332b2237dfe5ae4c670dd3ec8f26adf99fa437
Parents: 39ca90e
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed May 31 07:22:26 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Wed May 31 07:22:26 2017 +0530

----------------------------------------------------------------------
 .../apache/phoenix/rpc/PhoenixClientRpcIT.java  |  5 +-
 .../apache/phoenix/rpc/PhoenixServerRpcIT.java  |  3 +-
 .../apache/phoenix/calcite/CalciteUtils.java    |  6 +--
 .../phoenix/calcite/ImplicitNullLiteral.java    | 49 ++++++++++++++++++++
 .../apache/phoenix/calcite/PhoenixTable.java    |  6 ++-
 .../jdbc/PhoenixCalciteEmbeddedDriver.java      | 13 ++++--
 .../calcite/rel/PhoenixAbstractProject.java     | 20 +++++++-
 .../calcite/rel/PhoenixRelImplementor.java      |  2 +
 .../calcite/rel/PhoenixRelImplementorImpl.java  | 11 +++++
 .../phoenix/calcite/rel/PhoenixTableModify.java | 30 ++++++++----
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |  5 +-
 11 files changed, 129 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
index 8b7d5f2..54782d3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
@@ -15,6 +15,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -67,7 +68,7 @@ public class PhoenixClientRpcIT extends BaseUniqueNamesOwnClusterIT {
     @Test
     public void testIndexQos() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = driver.connect(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);
         try {
             // create the table
             conn.createStatement().execute(
@@ -111,7 +112,7 @@ public class PhoenixClientRpcIT extends BaseUniqueNamesOwnClusterIT {
     @Test
     public void testMetadataQos() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = driver.connect(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);;
         try {
             // create the table
             conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
index 410f02c..9f1ad77 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -89,7 +90,7 @@ public class PhoenixServerRpcIT extends BaseUniqueNamesOwnClusterIT {
     @Test
     public void testIndexQos() throws Exception { 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = driver.connect(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);;
         try {
             // create the table 
             conn.createStatement().execute(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
index 71775af..906fd71 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
@@ -1370,9 +1370,9 @@ public class CalciteUtils {
         }
     }
 
-    public static SQLException unwrapSqlException(SQLException root){
+    public static SQLException unwrapSqlException(Exception root){
         Exception e = root;
-        while(e.getCause() instanceof Exception){
+        while (e.getCause() instanceof Exception) {
             e = (Exception) e.getCause();
             if(e instanceof RuntimeException && e.getCause() instanceof SQLException) {
                 return (SQLException) e.getCause();
@@ -1381,7 +1381,7 @@ public class CalciteUtils {
                 return (SQLException) e;
             }
         }
-        return root;
+        return new SQLException(root);
     }
 
     public static Expression parseExpressionFromStr(String expressionStr, PhoenixConnection pc) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/ImplicitNullLiteral.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/ImplicitNullLiteral.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/ImplicitNullLiteral.java
new file mode 100644
index 0000000..ec8c8fc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/ImplicitNullLiteral.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.calcite;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBiVisitor;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexVisitor;
+
+/*
+ * Rex node used to represent implicit nulls. This helps to detect unspecified columns in upsert or
+ * upsert select.
+ */
+public class ImplicitNullLiteral extends RexNode {
+
+    final RexNode delegate;
+    public ImplicitNullLiteral(RexNode delegate) {
+        this.delegate = delegate;
+    }
+    @Override
+    public RelDataType getType() {
+        return delegate.getType();
+    }
+
+    @Override
+    public <R> R accept(RexVisitor<R> visitor) {
+        return delegate.accept(visitor);
+    }
+
+    @Override
+    public <R, P> R accept(RexBiVisitor<R, P> visitor, P arg) {
+        return delegate.accept(visitor, arg);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
index ad96a54..4175a0e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
@@ -231,7 +231,11 @@ public class PhoenixTable extends AbstractTable
             PColumn column = tableMapping.getMappedColumns().get(iColumn);
             String expressionStr = column.getExpressionStr();
             if(expressionStr == null) {
-                return super.newColumnDefaultValue(table, iColumn, context);
+                RelDataType pDataTypeToRelDataType =
+                        CalciteUtils.pDataTypeToRelDataType(rexBuilder.getTypeFactory(),
+                            column.getDataType(), column.getMaxLength(),
+                            column.getScale(), column.getArraySize());
+                return new ImplicitNullLiteral(this.rexBuilder.makeNullLiteral(pDataTypeToRelDataType));
             }
             Expression defaultExpression = CalciteUtils.parseExpressionFromStr(expressionStr, pc);
             return CalciteUtils.convertColumnExpressionToLiteral(column, defaultExpression,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
index 025ab58..2a5e567 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/jdbc/PhoenixCalciteEmbeddedDriver.java
@@ -30,8 +30,11 @@ import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.jdbc.Driver;
 import org.apache.calcite.linq4j.function.Function0;
+import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.phoenix.calcite.CalciteUtils;
 import org.apache.phoenix.calcite.PhoenixPrepareImpl;
 import org.apache.phoenix.calcite.PhoenixSchema;
 import org.apache.phoenix.calcite.rules.PhoenixConverterRules;
@@ -115,9 +118,13 @@ public abstract class PhoenixCalciteEmbeddedDriver extends Driver implements SQL
         final String phoenixUrl = url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL_CALCITE, PhoenixRuntime.JDBC_PROTOCOL);
         operand.put("url", phoenixUrl);
         SchemaPlus rootSchema = connection.getRootSchema();
-        Schema schema = PhoenixSchema.FACTORY.create(rootSchema, "phoenix", operand);
-        ((PhoenixSchema)schema).setTypeFactory(connection.getTypeFactory());
-        rootSchema.add("phoenix",schema);
+        try {
+            Schema schema = PhoenixSchema.FACTORY.create(rootSchema, "phoenix", operand);
+            ((PhoenixSchema)schema).setTypeFactory(connection.getTypeFactory());
+            rootSchema.add("phoenix",schema);
+        } catch(Exception e) {
+            CalciteUtils.unwrapSqlException(e);
+        }
         connection.setSchema("phoenix");
         
         return connection;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixAbstractProject.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixAbstractProject.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixAbstractProject.java
index d353dec..6ea1b3d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixAbstractProject.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixAbstractProject.java
@@ -11,10 +11,13 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.phoenix.calcite.CalciteUtils;
+import org.apache.phoenix.calcite.ImplicitNullLiteral;
 import org.apache.phoenix.calcite.TableMapping;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
@@ -51,12 +54,27 @@ abstract public class PhoenixAbstractProject extends Project implements PhoenixQ
     
     protected TupleProjector project(PhoenixRelImplementor implementor) {        
         List<Expression> exprs = Lists.newArrayList();
-        for (RexNode project : getProjects()) {
+        List<Integer> unspecifiedColumnPositions = Lists.newArrayList();
+        List<RexNode> projects = getProjects();
+        boolean bindVariablesPresent = false;
+        for(RexNode project: projects) {
+            if(project instanceof RexDynamicParam) {
+                bindVariablesPresent = true;
+                break;
+            }
+        }
+        for (int i = 0; i < projects.size(); i++) {
+            RexNode project = projects.get(i);
+            if((bindVariablesPresent && RexLiteral.isNullLiteral(project)) || project instanceof ImplicitNullLiteral) {
+                unspecifiedColumnPositions.add(new Integer(i));
+                continue;
+            }
             exprs.add(CalciteUtils.toExpression(project, implementor));
         }
         TupleProjector tupleProjector = implementor.project(exprs);
         PTable projectedTable = implementor.getTableMapping().createProjectedTable(implementor.getCurrentContext().retainPKColumns);
         implementor.setTableMapping(new TableMapping(projectedTable));
+        implementor.setUnspecifiedColumnPositions(unspecifiedColumnPositions);
 
         return tupleProjector;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
index db77254..d0b28d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementor.java
@@ -32,6 +32,8 @@ public interface PhoenixRelImplementor {
     ImplementorContext popContext();
     ImplementorContext getCurrentContext();
     TupleProjector project(List<Expression> exprs);
+    void setUnspecifiedColumnPositions(List<Integer> positions);
+    List<Integer> getUnspecifiedColumnPoistions();
     
     class ImplementorContext {
         public final boolean retainPKColumns;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
index c4a7d49..3b33c88 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
@@ -32,6 +32,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.types.PDataType;
+
 import com.google.common.collect.Lists;
 
 public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
@@ -39,6 +40,7 @@ public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
     private final RuntimeContext runtimeContext;
 	private Stack<ImplementorContext> contextStack;
 	private TableMapping tableMapping;
+	private List<Integer> unspecifiedColumnPoistions;
 	
 	public PhoenixRelImplementorImpl(
 	        StatementContext statementContext, RuntimeContext runtimeContext) {
@@ -142,4 +144,13 @@ public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
         return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));        
     }
 
+    @Override
+    public void setUnspecifiedColumnPositions(List<Integer> positions) {
+        this.unspecifiedColumnPoistions = positions;
+    }
+
+    @Override
+    public List<Integer> getUnspecifiedColumnPoistions() {
+        return unspecifiedColumnPoistions;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
index 335facf..6aeb638 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
@@ -4,6 +4,7 @@ import static org.apache.phoenix.execute.MutationState.RowTimestampColInfo.NULL_
 
 import java.sql.ParameterMetaData;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
@@ -83,17 +84,31 @@ public class PhoenixTableModify extends TableModify implements PhoenixRel {
         
         final QueryPlan queryPlan = implementor.visitInput(0, (PhoenixQueryRel) input);
         RowProjector projector;
+        final List<PColumn> targetColumns;
         try {
-            final List<PColumn> targetColumns =
+            final List<PColumn> mappedColumns =
                     getOperation() == Operation.DELETE
                     ? null : targetTable.tableMapping.getMappedColumns();
+            List<Integer> unspecifiedColumnPoistions = implementor.getUnspecifiedColumnPoistions();
+            if (unspecifiedColumnPoistions != null && !unspecifiedColumnPoistions.isEmpty()
+                    && getOperation() != Operation.DELETE) {
+                targetColumns =  new ArrayList<PColumn>(
+                        mappedColumns.size());
+                for(int i = 0; i < mappedColumns.size(); i++) {
+                    if(unspecifiedColumnPoistions.indexOf(i)==-1) {
+                        targetColumns.add(mappedColumns.get(i));
+                    }
+                }
+            } else {
+                targetColumns = mappedColumns;
+            }
             projector = implementor.getTableMapping().createRowProjector(targetColumns);
         } catch (SQLException e) {
             throw new RuntimeException(e);
         }
 
         if (getOperation() == Operation.INSERT) {
-            return upsert(connection, targetTable, targetTableRef, queryPlan, projector);
+            return upsert(connection, targetColumns, targetTableRef, queryPlan, projector);
         }
         
         // delete
@@ -101,19 +116,18 @@ public class PhoenixTableModify extends TableModify implements PhoenixRel {
     }
     
     private static MutationPlan upsert(final PhoenixConnection connection,
-            final PhoenixTable targetTable, final TableRef targetTableRef,
+            final List<PColumn> targetColumns, final TableRef targetTableRef,
             final QueryPlan queryPlan, final RowProjector projector) {
         try (PhoenixStatement stmt = new PhoenixStatement(connection)) {
             final ColumnResolver resolver = FromCompiler.getResolver(targetTableRef);
             final StatementContext context = new StatementContext(stmt, resolver, new Scan(), new SequenceManager(stmt));
 
             // TODO TenantId, ViewIndexId, UpdatableViewColumns
-            final List<PColumn> mappedColumns = targetTable.tableMapping.getMappedColumns();
-            final int[] columnIndexes = new int[mappedColumns.size()];
-            final int[] pkSlotIndexes = new int[mappedColumns.size()];
+            final int[] columnIndexes = new int[targetColumns.size()];
+            final int[] pkSlotIndexes = new int[targetColumns.size()];
             int pkColPosition = -1;
-            for (int i = 0; i < columnIndexes.length; i++) {
-                PColumn column = mappedColumns.get(i);
+            for (int i = 0; i < targetColumns.size(); i++) {
+                PColumn column = targetColumns.get(i);
                 if (SchemaUtil.isPKColumn(column)) {
                     if (pkColPosition == -1) {
                         pkColPosition = column.getPosition();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/48332b22/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
index f3850cb..e1f9373 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
@@ -46,7 +46,7 @@ public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
         final String tenantId = "00Dxx0000001234";
         props.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
 
-        Connection connection = new PhoenixTestDriver().connect(getUrl(), props);
+        Connection connection = DriverManager.getConnection(getUrl(), props);
         assertEquals(tenantId, connection.getClientInfo(PhoenixRuntime.TENANT_ID_ATTRIB));
     }
 
@@ -86,7 +86,8 @@ public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
     public void testMaxMutationSizeInBytesSetCorrectly() throws Exception {
         Properties connectionProperties = new Properties();
         connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB,"100");
-        PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        PhoenixConnection connection = DriverManager.getConnection(getUrl(), connectionProperties)
+                        .unwrap(PhoenixConnection.class);
         assertEquals(100L, connection.getMutateBatchSizeBytes());
         assertEquals(100L, connection.getMutationState().getMaxSizeBytes());
     }