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

    https://github.com/apache/flink/pull/3149#discussion_r97700808
  
    --- Diff: 
flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java
 ---
    @@ -22,54 +22,63 @@
     import org.apache.flink.api.java.ExecutionEnvironment;
     import org.apache.flink.api.java.typeutils.RowTypeInfo;
     import org.apache.flink.table.sources.BatchTableSource;
    -import org.apache.flink.table.sources.ProjectableTableSource;
     import org.apache.flink.types.Row;
     import org.apache.flink.util.Preconditions;
     import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.util.Pair;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
     
     /**
      * Creates a table source that helps to scan data from an hbase table
      *
      * Note : the colNames are specified along with a familyName and they are 
seperated by a ':'
      * For eg, cf1:q1 - where cf1 is the familyName and q1 is the qualifier 
name
      */
    -public class HBaseTableSource implements BatchTableSource<Row>, 
ProjectableTableSource<Row> {
    +// TODO : Implement ProjectableTableSource?
    +public class HBaseTableSource implements BatchTableSource<Row> {
     
        private Configuration conf;
        private String tableName;
    -   private byte[] rowKey;
    -   private String[] colNames;
    -   private TypeInformation<?>[] colTypes;
    +   private HBaseTableSchema schema;
    +   private String[] famNames;
     
    -   public HBaseTableSource(Configuration conf, String tableName, byte[] 
rowKey, String[] colNames,
    -                                                   TypeInformation<?>[] 
colTypes) {
    +   public HBaseTableSource(Configuration conf, String tableName, 
HBaseTableSchema schema) {
                this.conf = conf;
                this.tableName = Preconditions.checkNotNull(tableName, "Table  
name");
    -           this.rowKey = Preconditions.checkNotNull(rowKey, "Rowkey");
    -           this.colNames = Preconditions.checkNotNull(colNames, "Field 
names");
    -           this.colTypes = Preconditions.checkNotNull(colTypes, "Field 
types");
    +           this.schema = Preconditions.checkNotNull(schema, "Schema");
    +           Map<String, List<Pair>> familyMap = schema.getFamilyMap();
    +           famNames = familyMap.keySet().toArray(new 
String[familyMap.size()]);
        }
     
        @Override
        public TypeInformation<Row> getReturnType() {
    -           return new RowTypeInfo(colTypes);
    -   }
    -
    -   @Override
    -   public DataSet<Row> getDataSet(ExecutionEnvironment execEnv) {
    -           return execEnv.createInput(new 
HBaseTableSourceInputFormat(conf, tableName, colNames, colTypes), 
getReturnType());
    -   }
    +           // split the fieldNames
    +           Map<String, List<Pair>> famMap = schema.getFamilyMap();
     
    -   @Override
    -   public ProjectableTableSource<Row> projectFields(int[] fields) {
    -           String[] newColNames = new String[fields.length];
    -           TypeInformation<?>[] newColTypes =  new 
TypeInformation<?>[fields.length];
    +           List<String> qualNames = new ArrayList<String>();
    --- End diff --
    
    We can move the code of creating `typeInfos` into `HBaseTableSchema` , 
named `TypeInformation<?>[] getColumnTypes()`. 
    
    And also the code of creating family names called `String[] 
getFamilyNames()`.
    
    This can reduce the redundant code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to