Repository: phoenix Updated Branches: refs/heads/5.x-HBase-2.0 0454e4211 -> c82cc18d8
http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java index 3a6de6a..56080f8 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java @@ -25,7 +25,9 @@ import java.util.HashSet; import java.util.Set; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.util.Bytes; @@ -90,8 +92,7 @@ public class TestApplyAndFilterDeletesFilter { byte[] laterFamily = Bytes.toBytes("zfamily"); filter = new ApplyAndFilterDeletesFilter(asSet(laterFamily)); assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv)); - @SuppressWarnings("deprecation") - KeyValue expected = KeyValue.createFirstOnRow(kv.getRow(), laterFamily, new byte[0]); + KeyValue expected = KeyValueUtil.createFirstOnRow(CellUtil.cloneRow(kv), laterFamily, new byte[0]); assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(next)); assertEquals("Didn't get correct next key with a next family", expected, http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java index dbf67fc..0204cd1 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java @@ -34,8 +34,9 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -123,14 +124,14 @@ public class IndexMaintainerTest extends BaseConnectionlessQueryTest { stmt.setObject(i+1, values[i]); } stmt.execute(); - Iterator<Pair<byte[],List<KeyValue>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> dataKeyValues = iterator.next().getSecond(); + Iterator<Pair<byte[],List<Cell>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> dataKeyValues = iterator.next().getSecond(); Map<ColumnReference,byte[]> valueMap = Maps.newHashMapWithExpectedSize(dataKeyValues.size()); - byte[] row = dataKeyValues.get(0).getRow(); - ImmutableBytesWritable rowKeyPtr = new ImmutableBytesWritable(row); - Put dataMutation = new Put(rowKeyPtr.copyBytes()); - for (KeyValue kv : dataKeyValues) { - valueMap.put(new ColumnReference(kv.getFamily(),kv.getQualifier()), kv.getValue()); + ImmutableBytesWritable rowKeyPtr = new ImmutableBytesWritable(dataKeyValues.get(0).getRowArray(), dataKeyValues.get(0).getRowOffset(), dataKeyValues.get(0).getRowLength()); + byte[] row = rowKeyPtr.copyBytes(); + Put dataMutation = new Put(row); + for (Cell kv : dataKeyValues) { + valueMap.put(new ColumnReference(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()), CellUtil.cloneValue(kv)); dataMutation.add(kv); } ValueGetter valueGetter = newValueGetter(row, valueMap); @@ -148,7 +149,7 @@ public class IndexMaintainerTest extends BaseConnectionlessQueryTest { valueMap.get(ref); } byte[] dataRowKey = im1.buildDataRowKey(indexKeyPtr, null); - assertArrayEquals(dataRowKey, dataKeyValues.get(0).getRow()); + assertArrayEquals(dataRowKey, CellUtil.cloneRow(dataKeyValues.get(0))); } finally { try { conn.createStatement().execute("DROP TABLE " + fullTableName); http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java index 33c72a8..761aa23 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java @@ -27,7 +27,7 @@ import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Cell; import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; import org.junit.Test; @@ -94,7 +94,7 @@ public class CsvBulkImportUtilTest { public static class MockProcessor implements ImportPreUpsertKeyValueProcessor { @Override - public List<KeyValue> preUpsert(byte[] rowKey, List<KeyValue> keyValues) { + public List<Cell> preUpsert(byte[] rowKey, List<Cell> keyValues) { throw new UnsupportedOperationException("Not yet implemented"); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapperTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapperTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapperTest.java index 6424976..59a5edc 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapperTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapperTest.java @@ -20,6 +20,7 @@ package org.apache.phoenix.mapreduce; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil; import org.apache.phoenix.schema.types.PInteger; @@ -95,7 +96,7 @@ public class FormatToBytesWritableMapperTest { static class MockUpsertProcessor implements ImportPreUpsertKeyValueProcessor { @Override - public List<KeyValue> preUpsert(byte[] rowKey, List<KeyValue> keyValues) { + public List<Cell> preUpsert(byte[] rowKey, List<Cell> keyValues) { throw new UnsupportedOperationException("Not yet implemented"); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java index 4571115..61163d1 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java @@ -31,7 +31,8 @@ import java.util.Iterator; import java.util.List; import java.util.Properties; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.jdbc.PhoenixConnection; @@ -118,8 +119,8 @@ public class ConnectionlessTest { statement.setDate(5,now); statement.execute(); - Iterator<Pair<byte[],List<KeyValue>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); - Iterator<KeyValue> iterator = dataIterator.next().getSecond().iterator(); + Iterator<Pair<byte[],List<Cell>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); + Iterator<Cell> iterator = dataIterator.next().getSecond().iterator(); byte[] expectedRowKey1 = saltBuckets == null ? unsaltedRowKey1 : saltedRowKey1; byte[] expectedRowKey2 = saltBuckets == null ? unsaltedRowKey2 : saltedRowKey2; @@ -136,34 +137,34 @@ public class ConnectionlessTest { conn.rollback(); // to clear the list of mutations for the next } - private static void assertRow1(Iterator<KeyValue> iterator, byte[] expectedRowKey1) { - KeyValue kv; + private static void assertRow1(Iterator<Cell> iterator, byte[] expectedRowKey1) { + Cell kv; assertTrue(iterator.hasNext()); kv = iterator.next(); - assertArrayEquals(expectedRowKey1, kv.getRow()); - assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey1, CellUtil.cloneRow(kv)); + assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(CellUtil.cloneValue(kv))); kv = iterator.next(); - assertArrayEquals(expectedRowKey1, kv.getRow()); - assertEquals(name1, PVarchar.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey1, CellUtil.cloneRow(kv)); + assertEquals(name1, PVarchar.INSTANCE.toObject(CellUtil.cloneValue(kv))); assertTrue(iterator.hasNext()); kv = iterator.next(); - assertArrayEquals(expectedRowKey1, kv.getRow()); - assertEquals(now, PDate.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey1, CellUtil.cloneRow(kv)); + assertEquals(now, PDate.INSTANCE.toObject(CellUtil.cloneValue(kv))); } - private static void assertRow2(Iterator<KeyValue> iterator, byte[] expectedRowKey2) { - KeyValue kv; + private static void assertRow2(Iterator<Cell> iterator, byte[] expectedRowKey2) { + Cell kv; kv = iterator.next(); - assertArrayEquals(expectedRowKey2, kv.getRow()); - assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey2, CellUtil.cloneRow(kv)); + assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(CellUtil.cloneValue(kv))); assertTrue(iterator.hasNext()); kv = iterator.next(); - assertArrayEquals(expectedRowKey2, kv.getRow()); - assertEquals(name2, PVarchar.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey2, CellUtil.cloneRow(kv)); + assertEquals(name2, PVarchar.INSTANCE.toObject(CellUtil.cloneValue(kv))); assertTrue(iterator.hasNext()); kv = iterator.next(); - assertArrayEquals(expectedRowKey2, kv.getRow()); - assertEquals(now, PDate.INSTANCE.toObject(kv.getValue())); + assertArrayEquals(expectedRowKey2, CellUtil.cloneRow(kv)); + assertEquals(now, PDate.INSTANCE.toObject(CellUtil.cloneValue(kv))); } @Test http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java index 1052184..b9aabfa 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java @@ -33,7 +33,7 @@ import java.util.ListIterator; import java.util.NoSuchElementException; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList; import org.junit.Test; @@ -122,7 +122,7 @@ public class EncodedColumnQualifierCellsListTest { for (Cell c : cells) { assertTrue(list.contains(c)); } - assertFalse(list.contains(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13)))); + assertFalse(list.contains(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13)))); } @Test @@ -154,7 +154,7 @@ public class EncodedColumnQualifierCellsListTest { assertEquals(5, list.size()); assertTrue(list.remove(cells[3])); assertEquals(4, list.size()); - assertFalse(list.remove(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13)))); + assertFalse(list.remove(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13)))); assertEquals(4, list.size()); } @@ -165,10 +165,10 @@ public class EncodedColumnQualifierCellsListTest { EncodedColumnQualiferCellsList list2 = new EncodedColumnQualiferCellsList(11, 16, FOUR_BYTE_QUALIFIERS); populateList(list2); assertTrue(list1.containsAll(list2)); - list2.remove(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); + list2.remove(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); assertTrue(list1.containsAll(list2)); assertFalse(list2.containsAll(list1)); - list2.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13))); + list2.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(13))); assertFalse(list1.containsAll(list2)); assertFalse(list2.containsAll(list1)); List<Cell> arrayList = new ArrayList<>(); @@ -218,7 +218,7 @@ public class EncodedColumnQualifierCellsListTest { populateList(list2); // retainAll won't be modifying the list1 since they both have the same elements equality wise assertFalse(list1.retainAll(list2)); - list2.remove(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); + list2.remove(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); assertTrue(list1.retainAll(list2)); assertEquals(list1.size(), list2.size()); for (Cell c : list1) { @@ -408,7 +408,7 @@ public class EncodedColumnQualifierCellsListTest { i++; try { itr.next(); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); if (i == 2) { fail("ConcurrentModificationException should have been thrown as the list is being modified while being iterated through"); } @@ -426,7 +426,7 @@ public class EncodedColumnQualifierCellsListTest { populateList(list); ListIterator<Cell> itr = list.listIterator(); itr.next(); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); try { itr.next(); fail("ConcurrentModificationException should have been thrown as the list was modified without using iterator"); @@ -440,7 +440,7 @@ public class EncodedColumnQualifierCellsListTest { itr.next(); itr.remove(); itr.next(); - list.remove(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); + list.remove(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); try { itr.next(); fail("ConcurrentModificationException should have been thrown as the list was modified without using iterator"); @@ -451,28 +451,28 @@ public class EncodedColumnQualifierCellsListTest { private void populateListAndArray(List<Cell> list, Cell[] cells) { // add elements in reserved range - list.add(cells[0] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); - list.add(cells[1] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(5))); - list.add(cells[2] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(10))); + list.add(cells[0] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); + list.add(cells[1] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(5))); + list.add(cells[2] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(10))); // add elements in qualifier range - list.add(cells[6] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(16))); - list.add(cells[4] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); - list.add(cells[5] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(14))); - list.add(cells[3] = KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); + list.add(cells[6] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(16))); + list.add(cells[4] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); + list.add(cells[5] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(14))); + list.add(cells[3] = KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); } private void populateList(List<Cell> list) { // add elements in reserved range - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(5))); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(10))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(0))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(5))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(10))); // add elements in qualifier range - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(16))); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(14))); - list.add(KeyValue.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(16))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(12))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(14))); + list.add(KeyValueUtil.createFirstOnRow(row, cf, FOUR_BYTE_QUALIFIERS.encode(11))); } private class DelegateCell implements Cell { @@ -499,11 +499,6 @@ public class EncodedColumnQualifierCellsListTest { } @Override - public byte[] getValue() { - return delegate.getValue(); - } - - @Override public byte getTypeByte() { return delegate.getTypeByte(); } @@ -539,11 +534,6 @@ public class EncodedColumnQualifierCellsListTest { } @Override - public byte[] getRow() { - return delegate.getRow(); - } - - @Override public int getQualifierOffset() { return delegate.getQualifierOffset(); } @@ -559,16 +549,6 @@ public class EncodedColumnQualifierCellsListTest { } @Override - public byte[] getQualifier() { - return delegate.getQualifier(); - } - - @Override - public long getMvccVersion() { - return delegate.getMvccVersion(); - } - - @Override public int getFamilyOffset() { return delegate.getFamilyOffset(); } @@ -584,11 +564,6 @@ public class EncodedColumnQualifierCellsListTest { } @Override - public byte[] getFamily() { - return delegate.getFamily(); - } - - @Override public String toString() { return name; } http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/query/OrderByTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/OrderByTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/OrderByTest.java index abe2df0..c200c40 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/OrderByTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/OrderByTest.java @@ -26,7 +26,8 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.schema.SortOrder; @@ -42,12 +43,12 @@ public class OrderByTest extends BaseConnectionlessQueryTest { conn.createStatement().execute("UPSERT INTO t VALUES ('a')"); conn.createStatement().execute("UPSERT INTO t VALUES ('ab')"); - Iterator<Pair<byte[],List<KeyValue>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> kvs = dataIterator.next().getSecond(); - Collections.sort(kvs, KeyValue.COMPARATOR); - KeyValue first = kvs.get(0); + Iterator<Pair<byte[],List<Cell>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> kvs = dataIterator.next().getSecond(); + Collections.sort(kvs, CellComparatorImpl.COMPARATOR); + Cell first = kvs.get(0); assertEquals("ab", Bytes.toString(SortOrder.invert(first.getRowArray(), first.getRowOffset(), first.getRowLength()-1))); - KeyValue second = kvs.get(1); + Cell second = kvs.get(1); assertEquals("a", Bytes.toString(SortOrder.invert(second.getRowArray(), second.getRowOffset(), second.getRowLength()-1))); } @@ -58,12 +59,12 @@ public class OrderByTest extends BaseConnectionlessQueryTest { conn.createStatement().execute("UPSERT INTO t VALUES ('a')"); conn.createStatement().execute("UPSERT INTO t VALUES ('ab')"); - Iterator<Pair<byte[],List<KeyValue>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> kvs = dataIterator.next().getSecond(); - Collections.sort(kvs, KeyValue.COMPARATOR); - KeyValue first = kvs.get(0); + Iterator<Pair<byte[],List<Cell>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> kvs = dataIterator.next().getSecond(); + Collections.sort(kvs, CellComparatorImpl.COMPARATOR); + Cell first = kvs.get(0); assertEquals("ab", Bytes.toString(SortOrder.invert(first.getRowArray(), first.getRowOffset(), first.getRowLength()-1))); - KeyValue second = kvs.get(1); + Cell second = kvs.get(1); assertEquals("a", Bytes.toString(SortOrder.invert(second.getRowArray(), second.getRowOffset(), second.getRowLength()-1))); } @@ -74,12 +75,12 @@ public class OrderByTest extends BaseConnectionlessQueryTest { conn.createStatement().execute("UPSERT INTO t VALUES ('a','x')"); conn.createStatement().execute("UPSERT INTO t VALUES ('ab', 'x')"); - Iterator<Pair<byte[],List<KeyValue>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> kvs = dataIterator.next().getSecond(); - Collections.sort(kvs, KeyValue.COMPARATOR); - KeyValue first = kvs.get(0); + Iterator<Pair<byte[],List<Cell>>> dataIterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> kvs = dataIterator.next().getSecond(); + Collections.sort(kvs, CellComparatorImpl.COMPARATOR); + Cell first = kvs.get(0); assertEquals("ab", Bytes.toString(SortOrder.invert(first.getRowArray(), first.getRowOffset(), 2))); - KeyValue second = kvs.get(1); + Cell second = kvs.get(1); assertEquals("a", Bytes.toString(SortOrder.invert(second.getRowArray(), second.getRowOffset(), 1))); } @@ -89,11 +90,11 @@ public class OrderByTest extends BaseConnectionlessQueryTest { conn.createStatement().execute("CREATE TABLE t (k TIMESTAMP PRIMARY KEY DESC)"); conn.createStatement().execute("UPSERT INTO t VALUES ('2016-01-04 13:11:51.631')"); - Iterator<Pair<byte[], List<KeyValue>>> dataIterator = PhoenixRuntime + Iterator<Pair<byte[], List<Cell>>> dataIterator = PhoenixRuntime .getUncommittedDataIterator(conn); - List<KeyValue> kvs = dataIterator.next().getSecond(); - Collections.sort(kvs, KeyValue.COMPARATOR); - KeyValue first = kvs.get(0); + List<Cell> kvs = dataIterator.next().getSecond(); + Collections.sort(kvs, CellComparatorImpl.COMPARATOR); + Cell first = kvs.get(0); long millisDeserialized = PDate.INSTANCE.getCodec().decodeLong(first.getRowArray(), first.getRowOffset(), SortOrder.DESC); assertEquals(1451913111631L, millisDeserialized); http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeySchemaTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeySchemaTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeySchemaTest.java index a435ba6..9d689f9 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeySchemaTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeySchemaTest.java @@ -28,7 +28,7 @@ import java.sql.PreparedStatement; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.jdbc.PhoenixConnection; @@ -67,9 +67,9 @@ public class RowKeySchemaTest extends BaseConnectionlessQueryTest { stmt.setObject(i+1, values[i]); } stmt.execute(); - Iterator<Pair<byte[],List<KeyValue>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> dataKeyValues = iterator.next().getSecond(); - KeyValue keyValue = dataKeyValues.get(0); + Iterator<Pair<byte[],List<Cell>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> dataKeyValues = iterator.next().getSecond(); + Cell keyValue = dataKeyValues.get(0); List<SortOrder> sortOrders = Lists.newArrayListWithExpectedSize(table.getPKColumns().size()); for (PColumn col : table.getPKColumns()) { http://git-wip-us.apache.org/repos/asf/phoenix/blob/c82cc18d/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeyValueAccessorTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeyValueAccessorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeyValueAccessorTest.java index 928eb70..dfeaa4c 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeyValueAccessorTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/RowKeyValueAccessorTest.java @@ -26,12 +26,14 @@ import java.sql.PreparedStatement; import java.util.Iterator; import java.util.List; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.query.BaseConnectionlessQueryTest; import org.apache.phoenix.schema.types.PDataType; +import org.apache.phoenix.util.PhoenixKeyValueUtil; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.SchemaUtil; import org.junit.Test; @@ -63,9 +65,9 @@ public class RowKeyValueAccessorTest extends BaseConnectionlessQueryTest { stmt.setObject(i+1, values[i]); } stmt.execute(); - Iterator<Pair<byte[],List<KeyValue>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); - List<KeyValue> dataKeyValues = iterator.next().getSecond(); - KeyValue keyValue = dataKeyValues.get(0); + Iterator<Pair<byte[],List<Cell>>> iterator = PhoenixRuntime.getUncommittedDataIterator(conn); + List<Cell> dataKeyValues = iterator.next().getSecond(); + KeyValue keyValue = PhoenixKeyValueUtil.maybeCopyCell(dataKeyValues.get(0)); List<PColumn> pkColumns = table.getPKColumns(); RowKeyValueAccessor accessor = new RowKeyValueAccessor(pkColumns, 3);
