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

ASF GitHub Bot commented on TEPHRA-272:
---------------------------------------

Github user poornachandra commented on a diff in the pull request:

    https://github.com/apache/incubator-tephra/pull/67#discussion_r168888423
  
    --- Diff: 
tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/FilteredInternalScanner.java
 ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.tephra.hbase.coprocessor;
    +
    +import org.apache.hadoop.hbase.Cell;
    +import org.apache.hadoop.hbase.filter.Filter;
    +import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
    +import org.apache.hadoop.hbase.regionserver.InternalScanner;
    +import org.apache.hadoop.hbase.regionserver.ScannerContext;
    +import 
org.apache.tephra.hbase.coprocessor.TransactionProcessor.IncludeInProgressFilter;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * Wrapper of InternalScanner to apply Transaction visibility filter for 
flush and compact
    + */
    +public class FilteredInternalScanner implements InternalScanner {
    +
    +  private InternalScanner delegate;
    +  private Filter filter;
    +
    +  public FilteredInternalScanner(InternalScanner internalScanner, 
IncludeInProgressFilter filter) {
    +    this.delegate = internalScanner;
    +    this.filter = filter;
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +    this.delegate.close();
    +  }
    +
    +  @Override
    +  public boolean next(List<Cell> result, ScannerContext scannerContext) 
throws IOException {
    +    List<Cell> outResult = new ArrayList<Cell>();
    +    while (true) {
    +      boolean next = delegate.next(outResult, scannerContext);
    +      for (Cell cell : outResult) {
    +        ReturnCode code = filter.filterKeyValue(cell);
    +        switch (code) {
    +        // included, so we are done
    +        case INCLUDE:
    +        case INCLUDE_AND_NEXT_COL:
    --- End diff --
    
    When the filter return code is `INCLUDE_AND_NEXT_COL`, this filter needs to 
skip the remaining cells for that column, right?


> Add HBase 2.0 compatibility module
> ----------------------------------
>
>                 Key: TEPHRA-272
>                 URL: https://issues.apache.org/jira/browse/TEPHRA-272
>             Project: Tephra
>          Issue Type: Improvement
>            Reporter: Ankit Singhal
>            Assignee: Ankit Singhal
>            Priority: Major
>              Labels: HBase-2.0
>             Fix For: 0.14.0-incubating
>
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to