If the number of splits is greater than 1, you can use FilterList with
two ColumnRangeFilters
when needed.

Cheers

On Sun, Jan 18, 2015 at 4:37 PM, Taeyun Kim <[email protected]>
wrote:

> Thanks.
>
> But in my case it is unlikely that the FirstColumnName would be included
> in the range. (If it is included, it would cause a problem.)
>
> Instead, since the number of splits is mostly 1, I will include the name
> of the first split to the first Get with addColumn(). With that, most
> queries can be satisfied with single Get.
>
> Thanks again.
>
> -----Original Message-----
> From: Ted Yu [mailto:[email protected]]
> Sent: Saturday, January 17, 2015 6:31 AM
> To: [email protected]
> Subject: Re: Get addColumn + ColumnRangeFilter
>
> To clarify what I meant, the test passes with the following change:
>
>       Get g = new Get(RowKey);
>
>       byte[] minColumn = new byte[]{(byte)0};
>
>       int cmpMin = Bytes.compareTo(FirstColumnNameBytes, 0,
> FirstColumnNameBytes.length,
>
>         minColumn, 0, minColumn.length);
>
>       byte[] maxColumn = Bytes.toBytes("~");
>
>       int cmpMax = Bytes.compareTo(FirstColumnNameBytes, 0,
> FirstColumnNameBytes.length,
>
>         maxColumn, 0, maxColumn.length);
>
>       if (cmpMin <= 0 || cmpMax >= 0) {
>
>         g.addColumn(ColumnFamilyNameBytes, FirstColumnNameBytes);  //
> should be redundant...
>
>       }
>
>       g.setFilter(new ColumnRangeFilter(minColumn, false,
>
>         maxColumn, false));  // ...since this includes the first column
>
> FYI
>
> On Fri, Jan 16, 2015 at 7:23 AM, Ted Yu <[email protected]> wrote:
>
> > Thanks for the background information.
> >
> > For your last question, the columns given by addColumn() calls
> > (ColumnTracker
> > uses) are checked first.
> > So yes.
> >
> > Relaxing this limitation may take some effort - ScanQueryMatcher can
> > take Filter user passes into account. But the filter may not be
> > ColumnRangeFilter. It can be FilterList involving ColumnRangeFilter.
> > To add such logic into ScanQueryMatcher#match() makes the code less
> > maintainable.
> >
> > Can you check whether the column in addColumn() is covered by the
> > ColumnRangeFilter and if so, do not call addColumn() ?
> >
> > Cheers
> >
> > On Thu, Jan 15, 2015 at 11:35 PM, Taeyun Kim
> > <[email protected]>
> > wrote:
> >
> >> It's a somewhat long story.
> >> Maybe I use HBase some weird way.
> >>
> >> My use case is as follows:
> >>
> >> I didn't want to put many small file into HDFS. (Since it is bad for
> >> HDFS, both for scalability and performance)
> >>
> >> The small files are grouped by a test log, since the files are many
> >> facets of the result of the analysis of one test log. So, they could
> >> be the members of one SequentialFile.
> >> But I felt SequentialFile (or other similar ones) not attractive,
> >> since anyway I would get many not-so-big(about ~20MB, except for rare
> >> cases) Sequential files since the analysis result files are not so
> >> big and the test log files are continually generated.
> >> So some manual file management and merge could be a must.
> >>
> >> So, I decided to use a HBase record as a kind of 'directory' to avoid
> >> the manual file management. (directory = file group) By this, the
> >> 'files' are automatically 'merged' into appropriately sized HFiles,
> >> and as a bonus that 'files' can be automatically deleted when it's
> >> lifetime is done.
> >>
> >> The 'directory' has the following files.
> >>
> >> - 'm': meta file. (to check the version of the 'directory' format)
> >> - 'Result.csv.0'
> >> - 'Result.csv.1'
> >> - ...
> >> - 'Result.csv.p': parts file. (has the split count and each size. 'p'
> >> is for 'parts')
> >> - 'AnotherResultA.csv.0'
> >> - 'AnotherResultA.csv.1'
> >> - ...
> >> - 'AnotherResultA.csv.p'
> >> - 'TestEnvironment.txt'
> >>
> >> Each 'file' is saved as a column.
> >>
> >> Result files are split for the following reasons:
> >> - To handle extreme case the file is too big to be processed by one
> task.
> >> - To save the task process memory: the split size is actually smaller
> >> than 64MB(size for one task) and individually compressed. By this, a
> >> task process can have at most one column uncompressed. A task is
> >> assigned multiple 'splits'.
> >>
> >> For this, I've written an InputFormat class.
> >>
> >> Now, the InputFormat class can first Get both 'm' and a parts file to
> >> get the inputSplit information. This is not a problem. Single Get
> >> with 2
> >> addColumn() is sufficient.
> >> But when the whole content of a file must be read(like
> >> Files.readAllBytes()), must Get 'm' and unknown number of splits that
> >> has a name range(Result.csv.0 ~ Result.csv.7) to Get the whole
> >> content by single Get. (addColumn() + ColumnRangeFilter) But for the
> >> current HBase status, it seems that I have to invoke 2 Gets, or
> >> disable the version check. (Maybe not a big deal?)
> >>
> >> That's all.
> >>
> >> If you think that this Record is not efficient, or there is better
> >> solution, please let me know.
> >>
> >> BTW, for the current status, when both addColumn() and
> >> ColumnRangeFilter are applied, they are practically combined by 'AND'
> operator. Right?
> >>
> >> -----Original Message-----
> >> From: Ted Yu [mailto:[email protected]]
> >> Sent: Friday, January 16, 2015 3:39 PM
> >> To: [email protected]
> >> Subject: Re: Get addColumn + ColumnRangeFilter
> >>
> >> I reproduced the failed test (testAddColumnWithColumnRangeFilter)
> >> after modifying your test case to fit master branch.
> >>
> >> The reason for one Cell being returned is that ExplicitColumnTracker
> >> is used by ScanQueryMatcher to first check if the column is part of
> >> the requested columns (f:fc in your case). The other columns don't
> >> pass this check, hence they're not included in the result.
> >>
> >> Before this part of code is changed, can I ask why you need to call
> >> g.addColumn() when g has ColumnRangeFilter associated with it.
> >>
> >> Cheers
> >>
> >> On Thu, Jan 15, 2015 at 6:22 PM, Taeyun Kim
> >> <[email protected]>
> >> wrote:
> >>
> >> > (Sorry if this mail is a duplicate)
> >> >
> >> > Hi Ted,
> >> >
> >> > I've attached 2 unit test classes.
> >> >
> >> > Both have one failed test.
> >> >
> >> > -
> >> >
> >>
> HBaseAddColumnWithColumnRangeFilterTest1.testAddColumnWithColumnRangeFilter():
> >> > Expected: 10, Actual 1
> >> > -
> >> >
> >>
> HBaseAddColumnWithColumnRangeFilterTest2.testAddColumnWithColumnRangeFilter():
> >> > Result is empty
> >> >
> >> > If the tests have problems, please let me know.
> >> >
> >> >
> >> > -----Original Message-----
> >> > From: Ted Yu [mailto:[email protected]]
> >> > Sent: Thursday, January 15, 2015 6:59 PM
> >> > To: [email protected]
> >> > Subject: Re: Get addColumn + ColumnRangeFilter
> >> >
> >> > Can you write a unit test which shows this behavior?
> >> >
> >> > Thanks
> >> >
> >> >
> >> >
> >> > > On Jan 14, 2015, at 9:09 PM, Taeyun Kim <
> >> > [email protected]> wrote:
> >> > >
> >> > > Hi,
> >> > >
> >> > >
> >> > >
> >> > > I have a situation that both Get.addColumn() and
> >> > > Get.setFilter(new
> >> > > ColumnRangeFilter(…)) needed to Get.
> >> > >
> >> > > The source code snippet is as follows:
> >> > >
> >> > >
> >> > >
> >> > >        Get g = new Get(getRowKey(lfileId));
> >> > >
> >> > >        g.addColumn(Schema.ColumnFamilyNameBytes,
> >> > > MetaColumnNameBytes);
> >> > >
> >> > >        g.setFilter(new ColumnRangeFilter(Bytes.toBytes(name),
> >> > > false,
> >> > >
> >> > >            Bytes.toBytes(name + "~"), false));
> >> > >
> >> > >        Result r = table.get(g);
> >> > >
> >> > >
> >> > >
> >> > >        if (r.isEmpty())
> >> > >
> >> > >            throw new FileNotFoundException(
> >> > >
> >> > >                String.format("%d:%d:%s", projectId, lfileId,
> >> > > name));
> >> > >
> >> > >
> >> > >
> >> > > When g.addColumn() is commented out, the Result is not empty,
> >> > > while with g.addColumn the Result is empty(FileNotFoundException is
> thrown).
> >> > >
> >> > > Is it illegal to use both methods?
> >> > >
> >> > >
> >> > >
> >> > > BTW, ther version of HBase used is 0.98. (Hortonworks HDP 2.1)
> >> > >
> >> > >
> >> > >
> >> > > Thanks.
> >> >
> >>
> >>
> >
>
>

Reply via email to