hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Haibo Chen (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (YARN-7213) [Umbrella] Test and validate HBase-2.0.x with Atsv2
Date Fri, 17 Nov 2017 04:43:00 GMT

    [ https://issues.apache.org/jira/browse/YARN-7213?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16256466#comment-16256466
] 

Haibo Chen edited comment on YARN-7213 at 11/17/17 4:42 AM:
------------------------------------------------------------

[~openinx], [~tedyu@apache.org] Here is a test that you can add to TestTimelineReaderWebServicesHBaseStorage
to reproduce the behavior
{code:java}
  @Test
  public void testDirectScan() throws Exception {
    Table table = util.getConnection().getTable(TableName.valueOf("prod.timelineservice.entity"));

    Scan scan = new Scan();

    SingleColumnValueFilter svf1 = new SingleColumnValueFilter(
        Bytes.toBytes("c"), Bytes.toBytes("config_param1"),
        CompareFilter.CompareOp.EQUAL, new BinaryComparator(GenericConverter.getInstance().encodeValue("value1")));

    SingleColumnValueFilter svf2 = new SingleColumnValueFilter(
        Bytes.toBytes("c"), Bytes.toBytes("config_param1"),
        CompareFilter.CompareOp.EQUAL, new BinaryComparator(GenericConverter.getInstance().encodeValue("value3")));

    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
    filterList.addFilter(
        new FilterList(FilterList.Operator.MUST_PASS_ONE, Arrays.asList(svf1, svf2)));
    filterList.addFilter(
        new FilterList(FilterList.Operator.MUST_PASS_ONE, Arrays.asList(
            new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator("i".getBytes()))
        ))
    );
    scan.setFilter(filterList);

    Scan scan2 = new Scan(scan); // copy the scan

    int i = 0;
    for (Result result : table.getScanner(scan)) { i++; }
    assertEquals("first scan", 2, i); // this passes

    svf1.setFilterIfMissing(true);
    svf2.setFilterIfMissing(true);
    i = 0;
    for (Result result : table.getScanner(scan2)) { i++; }
    assertEquals("second scan", 2, i); // this fails
  }
{code}

[~openinx] Did you apply the wip patch attached?

Copying the internal discussion from our HBase folks.
> Say you have row1:
> r1 c1 q1 v1
> r1 c1 q2 v2
> r1 c2 -- --
> With filters: SCVF(=c1/q2/v2) AND FamilyFilter(=c2)
> The filters will first get applied to r1/c1/q1/v1: the SCVF doesn't match so it says
to skip this cell and go to the next row. The FamilyFilter >says it doesn't match either
and it wants to go to the next family. In this combination, the FamilyFilter "wins" and the
next cell evaluated is >r1/c2.
> Because the SCVF never saw r1/c1/q2, the filterIfMissing kicks in and then the whole
row is dropped.
> An important thing to realize (which I was struggling with) is that filters will narrow
what returns, they are not general predicates that >determine whether the whole cell is
filtered out or not.
> Or if this is in terms of SQL operations, HBase doesn't support something like SELECT
x WHERE y = "foo", so you have to make do with >SELECT x,y WHERE y = "foo" instead.
> This was actually working before due to a bug in the filter processing that got fixed
in 2.0


was (Author: haibochen):
[~openinx], [~tedyu@apache.org] Here is a test that you can add to TestTimelineReaderWebServicesHBaseStorage
to reproduce the behavior
{code:java}
  @Test
  public void testDirectScan() throws Exception {
    Table table = util.getConnection().getTable(TableName.valueOf("prod.timelineservice.entity"));

    Scan scan = new Scan();

    SingleColumnValueFilter svf1 = new SingleColumnValueFilter(
        Bytes.toBytes("c"), Bytes.toBytes("config_param1"),
        CompareFilter.CompareOp.EQUAL, new BinaryComparator(GenericConverter.getInstance().encodeValue("value1")));

    SingleColumnValueFilter svf2 = new SingleColumnValueFilter(
        Bytes.toBytes("c"), Bytes.toBytes("config_param1"),
        CompareFilter.CompareOp.EQUAL, new BinaryComparator(GenericConverter.getInstance().encodeValue("value3")));

    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
    filterList.addFilter(
        new FilterList(FilterList.Operator.MUST_PASS_ONE, Arrays.asList(svf1, svf2)));
    filterList.addFilter(
        new FilterList(FilterList.Operator.MUST_PASS_ONE, Arrays.asList(
            new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator("i".getBytes()))
        ))
    );
    scan.setFilter(filterList);

    Scan scan2 = new Scan(scan); // copy the scan

    int i = 0;
    for (Result result : table.getScanner(scan)) { i++; }
    assertEquals("first scan", 2, i); // this passes

    svf1.setFilterIfMissing(true);
    svf2.setFilterIfMissing(true);
    i = 0;
    for (Result result : table.getScanner(scan2)) { i++; }
    assertEquals("second scan", 2, i); // this fails
  }
{code}

[~openinx] Did you apply the wip patch attached?

Copying the internal discussion from our HBase folks.
>Say you have row1:
>r1 c1 q1 v1
>r1 c1 q2 v2
>r1 c2 -- --
>With filters: SCVF(=c1/q2/v2) AND FamilyFilter(=c2)
>The filters will first get applied to r1/c1/q1/v1: the SCVF doesn't match so it says to
skip this cell and go to the next row. The FamilyFilter >says it doesn't match either and
it wants to go to the next family. In this combination, the FamilyFilter "wins" and the next
cell evaluated is >r1/c2.
>Because the SCVF never saw r1/c1/q2, the filterIfMissing kicks in and then the whole row
is dropped.
>An important thing to realize (which I was struggling with) is that filters will narrow
what returns, they are not general predicates that >determine whether the whole cell is
filtered out or not.
>Or if this is in terms of SQL operations, HBase doesn't support something like SELECT
x WHERE y = "foo", so you have to make do with >SELECT x,y WHERE y = "foo" instead.
>This was actually working before due to a bug in the filter processing that got fixed
in 2.0

> [Umbrella] Test and validate HBase-2.0.x with Atsv2
> ---------------------------------------------------
>
>                 Key: YARN-7213
>                 URL: https://issues.apache.org/jira/browse/YARN-7213
>             Project: Hadoop YARN
>          Issue Type: Task
>            Reporter: Rohith Sharma K S
>            Assignee: Rohith Sharma K S
>         Attachments: YARN-7213.prelim.patch, YARN-7213.wip.patch
>
>
> Hbase-2.0.x officially support hadoop-alpha compilations. And also they are getting ready
for Hadoop-beta release so that HBase can release their versions compatible with Hadoop-beta.
So, this JIRA is to keep track of HBase-2.0 integration issues. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: yarn-issues-help@hadoop.apache.org


Mime
View raw message