diff --git a/CHANGES.txt b/CHANGES.txt index b7e50ac43df..ef86a65993c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -109,6 +109,8 @@ Release 0.91.0 - Unreleased HBASE-3898 TestSplitTransactionOnCluster broke in TRUNK HBASE-3826 Minor compaction needs to check if still over compactionThreshold after compacting (Nicolas Spiegelberg) + HBASE-3905 HBaseAdmin.createTableAsync() should check for invalid split + keys (Ted Yu) IMPROVEMENTS HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack) diff --git a/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 7aafe6b6509..0f478473db5 100644 --- a/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -286,19 +286,6 @@ public class HBaseAdmin implements Abortable, Closeable { public void createTable(HTableDescriptor desc, byte [][] splitKeys) throws IOException { HTableDescriptor.isLegalTableName(desc.getName()); - if(splitKeys != null && splitKeys.length > 1) { - Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR); - // Verify there are no duplicate split keys - byte [] lastKey = null; - for(byte [] splitKey : splitKeys) { - if(lastKey != null && Bytes.equals(splitKey, lastKey)) { - throw new IllegalArgumentException("All split keys must be unique, " + - "found duplicate: " + Bytes.toStringBinary(splitKey) + - ", " + Bytes.toStringBinary(lastKey)); - } - lastKey = splitKey; - } - } createTableAsync(desc, splitKeys); for (int tries = 0; tries < numRetries; tries++) { try { @@ -336,6 +323,19 @@ public class HBaseAdmin implements Abortable, Closeable { public void createTableAsync(HTableDescriptor desc, byte [][] splitKeys) throws IOException { HTableDescriptor.isLegalTableName(desc.getName()); + if(splitKeys != null && splitKeys.length > 1) { + Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR); + // Verify there are no duplicate split keys + byte [] lastKey = null; + for(byte [] splitKey : splitKeys) { + if(lastKey != null && Bytes.equals(splitKey, lastKey)) { + throw new IllegalArgumentException("All split keys must be unique, " + + "found duplicate: " + Bytes.toStringBinary(splitKey) + + ", " + Bytes.toStringBinary(lastKey)); + } + lastKey = splitKey; + } + } try { getMaster().createTable(desc, splitKeys); } catch (RemoteException e) {