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 2016/08/23 17:28:28 UTC

phoenix git commit: PHOENIX-930 duplicated columns cause query exception and drop table exception (Junegunn Choi, Kalyan Hadoop)

Repository: phoenix
Updated Branches:
  refs/heads/master 5fecde8d8 -> b36bb31fe


PHOENIX-930 duplicated columns cause query exception and drop table exception (Junegunn Choi, Kalyan Hadoop)


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

Branch: refs/heads/master
Commit: b36bb31fe42cb7bf5b6de5bfab63bcad424998a4
Parents: 5fecde8
Author: James Taylor <ja...@apache.org>
Authored: Fri Aug 19 16:34:28 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Tue Aug 23 10:29:40 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/index/IndexMetadataIT.java  | 40 ++++++++++
 .../apache/phoenix/schema/MetaDataClient.java   | 78 +++++---------------
 .../org/apache/phoenix/schema/PTableImpl.java   | 12 +--
 .../compile/CreateTableCompilerTest.java        | 47 ++++++++++++
 4 files changed, 113 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/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 1af15a1..a48cc4b 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
@@ -43,6 +43,7 @@ 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.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -401,6 +402,45 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+
+    @Test
+    public void testTableWithSameColumnNames() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        try {
+            String ddl = "create table test_table (char_pk varchar not null,"
+        		+ " int_col integer, long_col integer, int_col integer"
+        		+ " constraint pk primary key (char_pk))";
+            PreparedStatement stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+            fail("Should have caught exception");
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals(SQLExceptionCode.COLUMN_EXIST_IN_DEF.getErrorCode(), e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testTableWithSameColumnNamesWithFamily() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        try {
+            String ddl = "create table test_table (char_pk varchar not null,"
+        		+ " a.int_col integer, a.long_col integer,"
+        		+ " a.int_col integer, b.long_col integer"
+        		+ " constraint pk primary key (char_pk))";
+            PreparedStatement stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+            fail("Should have caught exception");
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals(SQLExceptionCode.COLUMN_EXIST_IN_DEF.getErrorCode(), e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
     
     @Test
     public void testIndexDefinitionWithSameColumnNamesInTwoFamily() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/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 1264e32..7f97f4a 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
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.schema;
 
-import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
 import static com.google.common.collect.Sets.newLinkedHashSet;
 import static com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
 import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
@@ -111,6 +110,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -120,7 +120,6 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -216,7 +215,6 @@ import org.apache.tephra.TxConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
@@ -1886,7 +1884,7 @@ public class MetaDataClient {
             }
 
             List<ColumnDef> colDefs = statement.getColumnDefs();
-            List<PColumn> columns;
+            LinkedHashMap<PColumn,PColumn> columns;
             LinkedHashSet<PColumn> pkColumns;
 
             if (tenantId != null && !sharedTable) {
@@ -1905,7 +1903,7 @@ public class MetaDataClient {
             if (tableType == PTableType.VIEW) {
                 physicalNames = Collections.singletonList(PNameFactory.newName(parent.getPhysicalName().getString()));
                 if (viewType == ViewType.MAPPED) {
-                    columns = newArrayListWithExpectedSize(colDefs.size());
+                    columns = Maps.newLinkedHashMap();
                     pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size());
                 } else {
                     // Propagate property values to VIEW.
@@ -1926,8 +1924,10 @@ public class MetaDataClient {
                     if (saltBucketNum != null) { // Don't include salt column in columns, as it should not have it when created
                         allColumns = allColumns.subList(1, allColumns.size());
                     }
-                    columns = newArrayListWithExpectedSize(allColumns.size() + colDefs.size());
-                    columns.addAll(allColumns);
+                    columns = new LinkedHashMap<PColumn,PColumn>(allColumns.size() + colDefs.size());
+                    for (PColumn column : allColumns) {
+                        columns.put(column, column);
+                    }
                     pkColumns = newLinkedHashSet(parent.getPKColumns());
 
                     // Add row linking from view to its parent table
@@ -1943,7 +1943,7 @@ public class MetaDataClient {
                     linkStatement.execute();
                 }
             } else {
-                columns = newArrayListWithExpectedSize(colDefs.size());
+                columns = new LinkedHashMap<PColumn,PColumn>(colDefs.size());
                 pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size() + 1); // in case salted
             }
 
@@ -2026,11 +2026,9 @@ public class MetaDataClient {
                         throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                     }
                 }
-                if (tableType == PTableType.VIEW && hasColumnWithSameNameAndFamily(columns, column)) {
-                    // we only need to check for dup columns for views because they inherit columns from parent
+                if (columns.put(column, column) != null) {
                     throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                 }
-                columns.add(column);
                 if ((colDef.getDataType() == PVarbinary.INSTANCE || colDef.getDataType().isArrayType())
                         && SchemaUtil.isPKColumn(column)
                         && pkColumnsIterator.hasNext()) {
@@ -2129,7 +2127,7 @@ public class MetaDataClient {
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
                         null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
-                        PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns, null, null,
+                        PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.values(), null, null,
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
@@ -2163,14 +2161,14 @@ public class MetaDataClient {
             
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
-                for (int i = 0; i < columns.size(); i++) {
-                    PColumn column = columns.get(i);
+                for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
+                    PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
                     // set the autoPartition column attributes   
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
-                        columns.set(i, column = new DelegateColumn(column) {
+                        entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
                                 // set to non-null value so that we will generate a Put that 
@@ -2186,7 +2184,7 @@ public class MetaDataClient {
                     }
                     else if (isViewColumnReferenced != null) {
                         if (viewColumnConstants != null && columnPosition < viewColumnConstants.length) {
-                            columns.set(i, column = new DelegateColumn(column) {
+                            entry.setValue(column = new DelegateColumn(column) {
                                 @Override
                                 public byte[] getViewConstant() {
                                     return viewColumnConstants[columnPosition];
@@ -2197,7 +2195,7 @@ public class MetaDataClient {
                                 }
                             });
                         } else {
-                            columns.set(i, column = new DelegateColumn(column) {
+                            entry.setValue(column = new DelegateColumn(column) {
                                 @Override
                                 public boolean isViewReferenced() {
                                     return isViewColumnReferenced.get(columnPosition);
@@ -2337,19 +2335,12 @@ public class MetaDataClient {
                 // If the parent table of the view has the auto partition sequence name attribute,
                 // set the view statement and relevant partition column attributes correctly
                 if (parent!=null && parent.getAutoPartitionSeqName()!=null) {
-                    int autoPartitionColIndex = -1;
-                    PColumn autoPartitionCol = parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent));
-                    for (int i=0; i<columns.size(); ++i) {
-                        if (autoPartitionCol.getName().equals(columns.get(i).getName())) {
-                            autoPartitionColIndex = i;
-                        }
-                    }
+                    final PColumn autoPartitionCol = parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent));
                     final Long autoPartitionNum = Long.valueOf(result.getAutoPartitionNum());
-                    final PColumn column = columns.get(autoPartitionColIndex);
-                    columns.set(autoPartitionColIndex, new DelegateColumn(column) {
+                    columns.put(autoPartitionCol, new DelegateColumn(autoPartitionCol) {
                         @Override
                         public byte[] getViewConstant() {
-                            PDataType dataType = column.getDataType();
+                            PDataType dataType = autoPartitionCol.getDataType();
                             Object val = dataType.toObject(autoPartitionNum, PLong.INSTANCE);
                             byte[] bytes = new byte [dataType.getByteSize() + 1];
                             dataType.toBytes(val, bytes, 0);
@@ -2371,7 +2362,7 @@ public class MetaDataClient {
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 PTable table =  PTableImpl.makePTable(
                         tenantId, newSchemaName, PNameFactory.newName(tableName), tableType, indexState, timestamp!=null ? timestamp : result.getMutationTime(),
-                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns,
+                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         dataTableName == null ? null : newSchemaName, dataTableName == null ? null : PNameFactory.newName(dataTableName), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
                         indexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
@@ -2384,29 +2375,6 @@ public class MetaDataClient {
         }
     }
 
-    private byte[][] getSplitKeys(List<HRegionLocation> allTableRegions) {
-        if(allTableRegions.size() == 1) return null;
-        byte[][] splitKeys = new byte[allTableRegions.size()-1][];
-        int i = 0;
-        for (HRegionLocation region : allTableRegions) {
-            if (region.getRegionInfo().getStartKey().length != 0) {
-                splitKeys[i] = region.getRegionInfo().getStartKey();
-                i++;
-            }
-        }
-        return splitKeys;
-    }
-
-    private static boolean hasColumnWithSameNameAndFamily(Collection<PColumn> columns, PColumn column) {
-        for (PColumn currColumn : columns) {
-           if (Objects.equal(currColumn.getFamilyName(), column.getFamilyName()) &&
-               Objects.equal(currColumn.getName(), column.getName())) {
-               return true;
-           }
-        }
-        return false;
-    }
-
     /**
      * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
      * <p>
@@ -2517,7 +2485,6 @@ public class MetaDataClient {
             Delete tableDelete = new Delete(key, clientTimeStamp);
             tableMetaData.add(tableDelete);
             boolean hasViewIndexTable = false;
-            boolean hasLocalIndexTable = false;
             if (parentTableName != null) {
                 byte[] linkKey = MetaDataUtil.getParentLinkKey(tenantIdStr, schemaName, parentTableName, tableName);
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
@@ -2558,11 +2525,6 @@ public class MetaDataClient {
                         // PName name, PTableType type, long timeStamp, long sequenceNumber, List<PColumn> columns
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
-                            for (PTable index : table.getIndexes()) {
-                                if (index.getIndexType() == IndexType.LOCAL) {
-                                    hasLocalIndexTable = true;
-                                } 
-                            }
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
                                                      // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
@@ -3716,7 +3678,7 @@ public class MetaDataClient {
                 || useSchemaStatement.getSchemaName().toUpperCase().equals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE)) {
             connection.setSchema(null);
         } else {
-            PSchema schema = FromCompiler.getResolverForSchema(useSchemaStatement, connection)
+            FromCompiler.getResolverForSchema(useSchemaStatement, connection)
                     .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 847979a..92c49f9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -299,7 +300,7 @@ public class PTableImpl implements PTable {
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
+            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
@@ -313,7 +314,7 @@ public class PTableImpl implements PTable {
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
+            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
@@ -328,7 +329,7 @@ public class PTableImpl implements PTable {
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
-            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, List<PColumn> columns,
+            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, Collection<PColumn> columns,
             PName parentSchemaName, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
             List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
@@ -367,7 +368,7 @@ public class PTableImpl implements PTable {
     }
 
     private void init(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
-            PName pkName, Integer bucketNum, List<PColumn> columns, PName parentSchemaName, PName parentTableName,
+            PName pkName, Integer bucketNum, Collection<PColumn> columns, PName parentSchemaName, PName parentTableName,
             List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
@@ -423,8 +424,7 @@ public class PTableImpl implements PTable {
             allColumns = new PColumn[columns.size()];
             pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
         }
-        for (int i = 0; i < columns.size(); i++) {
-            PColumn column = columns.get(i);
+        for (PColumn column : columns) {
             allColumns[column.getPosition()] = column;
             PName familyName = column.getFamilyName();
             if (familyName == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
new file mode 100644
index 0000000..ed907c3
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.compile;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+public class CreateTableCompilerTest extends BaseConnectionlessQueryTest {
+    @Test
+    public void testCreateTableWithDuplicateColumns() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        PhoenixConnection conn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
+        String ddl = "CREATE TABLE T (ID INTEGER PRIMARY KEY, DUPE INTEGER, DUPE INTEGER)";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals("DUPE", e.getColumnName());
+        }
+    }
+}