From a2cbe7a97c2f65bd1f2a21eaba0c8868b315d55d Mon Sep 17 00:00:00 2001 From: Rahul Kesharwani <42969463+rahulKQL@users.noreply.github.com> Date: Fri, 17 Apr 2020 20:35:18 +0530 Subject: [PATCH] fix: updated assertions and scan for firstKeyOnlyFilter test (#2483) * chore(test): updated assertions for firstKeyOnlyFilter This PR updates integration tests assertions to verify firstKeyOnlyFilter. * chore: reverting back FirstKeyOnlyFilterAdapter to return cell with value This commit partially reverts changes from #1996 to have consistent behavior with HBase FirstKeyOnlyFilter. * to trigger failed CI Job --- .../bigtable/hbase/AbstractTestFilters.java | 34 ++++++++++++++----- .../filters/FirstKeyOnlyFilterAdapter.java | 3 +- .../TestFirstKeyOnlyFilterAdapter.java | 8 +---- 3 files changed, 27 insertions(+), 18 deletions(-) diff --git a/bigtable-client-core-parent/bigtable-hbase-integration-tests-common/src/test/java/com/google/cloud/bigtable/hbase/AbstractTestFilters.java b/bigtable-client-core-parent/bigtable-hbase-integration-tests-common/src/test/java/com/google/cloud/bigtable/hbase/AbstractTestFilters.java index 9e8546b407..b3e350afdf 100644 --- a/bigtable-client-core-parent/bigtable-hbase-integration-tests-common/src/test/java/com/google/cloud/bigtable/hbase/AbstractTestFilters.java +++ b/bigtable-client-core-parent/bigtable-hbase-integration-tests-common/src/test/java/com/google/cloud/bigtable/hbase/AbstractTestFilters.java @@ -1363,22 +1363,38 @@ public void testValueFilter() throws IOException { @Test public void testFirstKeyFilter() throws IOException { // Initialize + int rowCount = 5; int numCols = 5; - String columnValue = "includeThisValue"; + String rowPrefix = dataHelper.randomString("testFirstKeyValue-"); + byte[] columnValue = Bytes.toBytes("includeThisValue"); Table table = getDefaultTable(); - byte[] rowKey = dataHelper.randomData("testRow-"); - Put put = new Put(rowKey); - for (int i = 0; i < numCols; ++i) { - put.addColumn(COLUMN_FAMILY, dataHelper.randomData(""), Bytes.toBytes(columnValue)); + + List puts = new ArrayList<>(rowCount); + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + Put put = new Put(Bytes.toBytes(rowPrefix + rowIndex)); + for (int i = 0; i < numCols; ++i) { + put.addColumn(COLUMN_FAMILY, dataHelper.randomData(""), columnValue); + } + puts.add(put); } - table.put(put); + table.put(puts); // Filter for results Filter filter = new FirstKeyOnlyFilter(); - Get get = new Get(rowKey).setFilter(filter); - Result result = table.get(get); - Assert.assertEquals("Should only return 1 keyvalue", 1, result.size()); + Scan scan = new Scan().setRowPrefixFilter(Bytes.toBytes(rowPrefix)).setFilter(filter); + try (ResultScanner resultScanner = table.getScanner(scan)) { + int rowIndex = 0; + for (Result result : resultScanner) { + Assert.assertArrayEquals(Bytes.toBytes(rowPrefix + rowIndex), result.getRow()); + Assert.assertEquals("Should only return 1 keyvalue", 1, result.size()); + Assert.assertTrue( + "Should contains column value", + CellUtil.matchingValue(result.rawCells()[0], columnValue)); + + rowIndex++; + } + } table.close(); } diff --git a/bigtable-client-core-parent/bigtable-hbase/src/main/java/com/google/cloud/bigtable/hbase/adapters/filters/FirstKeyOnlyFilterAdapter.java b/bigtable-client-core-parent/bigtable-hbase/src/main/java/com/google/cloud/bigtable/hbase/adapters/filters/FirstKeyOnlyFilterAdapter.java index 4f920d5881..e68b62e33a 100644 --- a/bigtable-client-core-parent/bigtable-hbase/src/main/java/com/google/cloud/bigtable/hbase/adapters/filters/FirstKeyOnlyFilterAdapter.java +++ b/bigtable-client-core-parent/bigtable-hbase/src/main/java/com/google/cloud/bigtable/hbase/adapters/filters/FirstKeyOnlyFilterAdapter.java @@ -29,8 +29,7 @@ @InternalApi("For internal usage only") public class FirstKeyOnlyFilterAdapter extends TypedFilterAdapterBase { - private static Filters.Filter LIMIT_ONE = - FILTERS.chain().filter(FILTERS.limit().cellsPerRow(1)).filter(FILTERS.value().strip()); + private static Filters.Filter LIMIT_ONE = FILTERS.limit().cellsPerRow(1); /** {@inheritDoc} */ @Override diff --git a/bigtable-client-core-parent/bigtable-hbase/src/test/java/com/google/cloud/bigtable/hbase/adapters/filters/TestFirstKeyOnlyFilterAdapter.java b/bigtable-client-core-parent/bigtable-hbase/src/test/java/com/google/cloud/bigtable/hbase/adapters/filters/TestFirstKeyOnlyFilterAdapter.java index 8805fb50a4..783e49ecb5 100644 --- a/bigtable-client-core-parent/bigtable-hbase/src/test/java/com/google/cloud/bigtable/hbase/adapters/filters/TestFirstKeyOnlyFilterAdapter.java +++ b/bigtable-client-core-parent/bigtable-hbase/src/test/java/com/google/cloud/bigtable/hbase/adapters/filters/TestFirstKeyOnlyFilterAdapter.java @@ -35,12 +35,6 @@ public class TestFirstKeyOnlyFilterAdapter { public void onlyTheFirstKeyFromEachRowIsEmitted() throws IOException { Filters.Filter adaptedFilter = adapter.adapt(new FilterAdapterContext(new Scan(), null), new FirstKeyOnlyFilter()); - Assert.assertEquals( - FILTERS - .chain() - .filter(FILTERS.limit().cellsPerRow(1)) - .filter(FILTERS.value().strip()) - .toProto(), - adaptedFilter.toProto()); + Assert.assertEquals(FILTERS.limit().cellsPerRow(1).toProto(), adaptedFilter.toProto()); } }