You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2016/07/21 20:37:43 UTC
svn commit: r1753732 - in /pig/trunk: CHANGES.txt
src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
test/org/apache/pig/test/TestHBaseStorage.java
Author: daijy
Date: Thu Jul 21 20:37:43 2016
New Revision: 1753732
URL: http://svn.apache.org/viewvc?rev=1753732&view=rev
Log:
PIG-4947: LOAD with HBaseStorage using a mix of pure wildcards and prefixed wildcards results in empty maps for the pure wildcarded column families
Modified:
pig/trunk/CHANGES.txt
pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1753732&r1=1753731&r2=1753732&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Jul 21 20:37:43 2016
@@ -60,6 +60,9 @@ OPTIMIZATIONS
BUG FIXES
+PIG-4947: LOAD with HBaseStorage using a mix of pure wildcards and prefixed wildcards results
+ in empty maps for the pure wildcarded column families (daijy)
+
PIG-4948: Pig on Tez AM use too much memory on a small cluster (daijy)
PIG-4949: Fix registering jar in S3 which was broken by PIG-4417 in Pig 0.16 (yangyishan0901m via daijy)
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java?rev=1753732&r1=1753731&r2=1753732&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java Thu Jul 21 20:37:43 2016
@@ -590,7 +590,9 @@ public class HBaseStorage extends LoadFu
new BinaryComparator(colInfo.getColumnName())));
}
}
- thisColumnGroupFilter.addFilter(columnFilters);
+ if (columnFilters.getFilters().size() != 0) {
+ thisColumnGroupFilter.addFilter(columnFilters);
+ }
allColumnFilters.addFilter(thisColumnGroupFilter);
}
if (allColumnFilters != null) {
Modified: pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java?rev=1753732&r1=1753731&r2=1753732&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java Thu Jul 21 20:37:43 2016
@@ -69,12 +69,16 @@ public class TestHBaseStorage {
private static final String TESTTABLE_1 = "pigtable_1";
private static final String TESTTABLE_2 = "pigtable_2";
private static final byte[] COLUMNFAMILY = Bytes.toBytes("pig");
+ private static final byte[] COLUMNFAMILY2 = Bytes.toBytes("pig2");
private static final String TESTCOLUMN_A = "pig:col_a";
private static final String TESTCOLUMN_B = "pig:col_b";
private static final String TESTCOLUMN_C = "pig:col_c";
private static final int TEST_ROW_COUNT = 100;
+ private enum TableType {ONE_CF, TWO_CF};
+ private TableType lastTableType;
+
@BeforeClass
public static void setUp() throws Exception {
// This is needed by Pig
@@ -311,13 +315,13 @@ public class TestHBaseStorage {
*/
@Test
public void testLoadWithMap_3_col_prefix() throws IOException {
- prepareTable(TESTTABLE_1, true, DataFormat.UTF8PlainText);
+ prepareTable(TESTTABLE_1, true, DataFormat.UTF8PlainText, TableType.TWO_CF);
pig.registerQuery("a = load 'hbase://"
+ TESTTABLE_1
+ "' using "
+ "org.apache.pig.backend.hadoop.hbase.HBaseStorage('"
- + "pig:col_* pig:prefixed_col_*"
+ + "pig2:* pig:prefixed_col_*"
+ "','-loadKey') as (rowKey:chararray, pig_cf_map:map[], pig_prefix_cf_map:map[]);");
Iterator<Tuple> it = pig.openIterator("a");
int count = 0;
@@ -326,24 +330,18 @@ public class TestHBaseStorage {
Tuple t = it.next();
LOG.info("LoadFromHBase " + t);
String rowKey = t.get(0).toString();
- Map pig_cf_map = (Map) t.get(1);
+ Map pig_secondery_cf_map = (Map) t.get(1);
Map pig_prefix_cf_map = (Map) t.get(2);
Assert.assertEquals(3, t.size());
Assert.assertEquals("00".substring((count + "").length()) + count,
rowKey);
+ Assert.assertEquals(count,
+ Integer.parseInt(pig_secondery_cf_map.get("col_x").toString()));
Assert.assertEquals("PrefixedText_" + count,
((DataByteArray) pig_prefix_cf_map.get("prefixed_col_d")).toString());
Assert.assertEquals(1, pig_prefix_cf_map.size());
- Assert.assertEquals(count,
- Integer.parseInt(pig_cf_map.get("col_a").toString()));
- Assert.assertEquals(count + 0.0,
- Double.parseDouble(pig_cf_map.get("col_b").toString()), 1e-6);
- Assert.assertEquals("Text_" + count,
- ((DataByteArray) pig_cf_map.get("col_c")).toString());
- Assert.assertEquals(3, pig_cf_map.size());
-
count++;
}
Assert.assertEquals(TEST_ROW_COUNT, count);
@@ -432,6 +430,39 @@ public class TestHBaseStorage {
LOG.info("LoadFromHBase done");
}
+ public void testLoadWithFixedAndPrefixedCols3() throws IOException {
+ prepareTable(TESTTABLE_1, true, DataFormat.UTF8PlainText);
+
+ pig.registerQuery("a = load 'hbase://"
+ + TESTTABLE_1
+ + "' using "
+ + "org.apache.pig.backend.hadoop.hbase.HBaseStorage('"
+ + "pig:* pig:prefixed_col_*"
+ + "','-loadKey') as (rowKey:chararray, pig_cf_map:map[], pig_prefix_cf_map:map[]);");
+ Iterator<Tuple> it = pig.openIterator("a");
+ int count = 0;
+ LOG.info("LoadFromHBase Starting");
+ while (it.hasNext()) {
+ Tuple t = it.next();
+ LOG.info("LoadFromHBase " + t);
+ String rowKey = (String) t.get(0);
+ Map pig_cf_map = (Map) t.get(1);
+ Map pig_prefix_cf_map = (Map) t.get(2);
+ Assert.assertEquals(3, t.size());
+
+ Assert.assertEquals("00".substring((count + "").length()) + count,
+ rowKey);
+ Assert.assertEquals("PrefixedText_" + count,
+ ((DataByteArray) pig_cf_map.get("prefixed_col_d")).toString());
+ Assert.assertEquals(1, pig_cf_map.size());
+ Assert.assertEquals(1, pig_prefix_cf_map.size());
+
+ count++;
+ }
+ Assert.assertEquals(TEST_ROW_COUNT, count);
+ LOG.info("LoadFromHBase done");
+ }
+
/**
* * Test Load from hbase with map parameters and with a
* static column in different order
@@ -1487,22 +1518,36 @@ public class TestHBaseStorage {
+ "') as (rowKey:chararray,col_a:int, col_b:double, col_c:chararray);");
}
+ private HTable prepareTable(String tableName, boolean initData,
+ DataFormat format) throws IOException {
+ return prepareTable(tableName, initData, format, TableType.ONE_CF);
+ }
/**
* Prepare a table in hbase for testing.
*
*/
private HTable prepareTable(String tableName, boolean initData,
- DataFormat format) throws IOException {
+ DataFormat format, TableType type) throws IOException {
// define the table schema
HTable table = null;
try {
- deleteAllRows(tableName);
+ if (lastTableType == type) {
+ deleteAllRows(tableName);
+ } else {
+ util.deleteTable(tableName);
+ }
} catch (Exception e) {
// It's ok, table might not exist.
}
try {
- table = util.createTable(Bytes.toBytesBinary(tableName),
- COLUMNFAMILY);
+ if (type == TableType.TWO_CF) {
+ table = util.createTable(Bytes.toBytesBinary(tableName),
+ new byte[][]{COLUMNFAMILY, COLUMNFAMILY2});
+ } else {
+ table = util.createTable(Bytes.toBytesBinary(tableName),
+ COLUMNFAMILY);
+ }
+ lastTableType = type;
} catch (Exception e) {
table = new HTable(conf, Bytes.toBytesBinary(tableName));
}
@@ -1529,6 +1574,11 @@ public class TestHBaseStorage {
// prefixed_col_d: string type
put.add(COLUMNFAMILY, Bytes.toBytes("prefixed_col_d"),
Bytes.toBytes("PrefixedText_" + i));
+ // another cf
+ if (type == TableType.TWO_CF) {
+ put.add(COLUMNFAMILY2, Bytes.toBytes("col_x"),
+ Bytes.toBytes(i));
+ }
table.put(put);
} else {
// row key: string type
@@ -1549,6 +1599,11 @@ public class TestHBaseStorage {
// prefixed_col_d: string type
put.add(COLUMNFAMILY, Bytes.toBytes("prefixed_col_d"),
("PrefixedText_" + i).getBytes());
+ // another cf
+ if (type == TableType.TWO_CF) {
+ put.add(COLUMNFAMILY2, Bytes.toBytes("col_x"),
+ (i + "").getBytes());
+ }
table.put(put);
}
}